mirror of https://github.com/milvus-io/milvus.git
				
				
				
			feat: timestamptz expression & index & timezone (#44080)
issue: https://github.com/milvus-io/milvus/issues/27467 >My plan is as follows. >- [x] M1 Create collection with timestamptz field >- [x] M2 Insert timestamptz field data >- [x] M3 Retrieve timestamptz field data >- [x] M4 Implement handoff >- [x] M5 Implement compare operator >- [x] M6 Implement extract operator >- [x] M8 Support database/collection level default timezone >- [x] M7 Support STL-SORT index for datatype timestamptz --- The third PR of issue: https://github.com/milvus-io/milvus/issues/27467, which completes M5, M6, M7, M8 described above. ## M8 Default Timezone We will be able to use alter_collection() and alter_database() in a future Python SDK release to modify the default timezone at the collection or database level. For insert requests, the timezone will be resolved using the following order of precedence: String Literal-> Collection Default -> Database Default. For retrieval requests, the timezone will be resolved in this order: Query Parameters -> Collection Default -> Database Default. In both cases, the final fallback timezone is UTC. ## M5: Comparison Operators We can now use the following expression format to filter on the timestamptz field: - `timestamptz_field [+/- INTERVAL 'interval_string'] {comparison_op} ISO 'iso_string' ` - The interval_string follows the ISO 8601 duration format, for example: P1Y2M3DT1H2M3S. - The iso_string follows the ISO 8601 timestamp format, for example: 2025-01-03T00:00:00+08:00. - Example expressions: "tsz + INTERVAL 'P0D' != ISO '2025-01-03T00:00:00+08:00'" or "tsz != ISO '2025-01-03T00:00:00+08:00'". ## M6: Extract We will be able to extract sepecific time filed by kwargs in a future Python SDK release. The key is `time_fields`, and value should be one or more of "year, month, day, hour, minute, second, microsecond", seperated by comma or space. Then the result of each record would be an array of int64. ## M7: Indexing Support Expressions without interval arithmetic can be accelerated using an STL-SORT index. However, expressions that include interval arithmetic cannot be indexed. This is because the result of an interval calculation depends on the specific timestamp value. For example, adding one month to a date in February results in a different number of added days than adding one month to a date in March. --- After this PR, the input / output type of timestamptz would be iso string. Timestampz would be stored as timestamptz data, which is int64_t finally. > for more information, see https://en.wikipedia.org/wiki/ISO_8601 --------- Signed-off-by: xtx <xtianx@smail.nju.edu.cn>pull/44394/head
							parent
							
								
									338ed2fed4
								
							
						
					
					
						commit
						2c0c5ef41e
					
				| 
						 | 
				
			
			@ -114,3 +114,6 @@ pkg/streaming/proto/**/*.pb.go
 | 
			
		|||
 | 
			
		||||
#AI rules
 | 
			
		||||
WARP.md
 | 
			
		||||
 | 
			
		||||
# Antlr
 | 
			
		||||
.antlr
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -97,7 +97,6 @@ FieldMeta::ParseFrom(const milvus::proto::schema::FieldSchema& schema_proto) {
 | 
			
		|||
        return schema_proto.default_value();
 | 
			
		||||
    }();
 | 
			
		||||
 | 
			
		||||
    LOG_DEBUG("Get datatype {}", GetDataTypeName(data_type));
 | 
			
		||||
    if (data_type == DataType::VECTOR_ARRAY) {
 | 
			
		||||
        // todo(SpadeA): revisit the code when index build for vector array is ready
 | 
			
		||||
        int64_t dim = 0;
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1834,5 +1834,9 @@ PhyBinaryArithOpEvalRangeExpr::ExecRangeVisitorImplForData(
 | 
			
		|||
    return res_vec;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
template VectorPtr
 | 
			
		||||
PhyBinaryArithOpEvalRangeExpr::ExecRangeVisitorImpl<int64_t>(
 | 
			
		||||
    OffsetVector*);  // add this for macos
 | 
			
		||||
 | 
			
		||||
}  //namespace exec
 | 
			
		||||
}  // namespace milvus
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -450,7 +450,10 @@ struct ArithOpIndexFunc {
 | 
			
		|||
    }
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
class PhyTimestamptzArithCompareExpr;
 | 
			
		||||
class PhyBinaryArithOpEvalRangeExpr : public SegmentExpr {
 | 
			
		||||
    friend class PhyTimestamptzArithCompareExpr;
 | 
			
		||||
 | 
			
		||||
 public:
 | 
			
		||||
    PhyBinaryArithOpEvalRangeExpr(
 | 
			
		||||
        const std::vector<std::shared_ptr<Expr>>& input,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -32,6 +32,7 @@
 | 
			
		|||
#include "exec/expression/TermExpr.h"
 | 
			
		||||
#include "exec/expression/UnaryExpr.h"
 | 
			
		||||
#include "exec/expression/ValueExpr.h"
 | 
			
		||||
#include "exec/expression/TimestamptzArithCompareExpr.h"
 | 
			
		||||
#include "expr/ITypeExpr.h"
 | 
			
		||||
#include "monitor/Monitor.h"
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -251,6 +252,17 @@ CompileExpression(const expr::TypedExprPtr& expr,
 | 
			
		|||
            context->get_active_count(),
 | 
			
		||||
            context->query_config()->get_expr_batch_size(),
 | 
			
		||||
            context->get_consistency_level());
 | 
			
		||||
    } else if (auto casted_expr = std::dynamic_pointer_cast<
 | 
			
		||||
                   const milvus::expr::TimestamptzArithCompareExpr>(expr)) {
 | 
			
		||||
        result = std::make_shared<PhyTimestamptzArithCompareExpr>(
 | 
			
		||||
            compiled_inputs,
 | 
			
		||||
            casted_expr,
 | 
			
		||||
            "PhyTimestamptzArithCompareExpr",
 | 
			
		||||
            op_ctx,
 | 
			
		||||
            context->get_segment(),
 | 
			
		||||
            context->get_active_count(),
 | 
			
		||||
            context->query_config()->get_expr_batch_size(),
 | 
			
		||||
            context->get_consistency_level());
 | 
			
		||||
    } else if (auto casted_expr =
 | 
			
		||||
                   std::dynamic_pointer_cast<const milvus::expr::CompareExpr>(
 | 
			
		||||
                       expr)) {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -0,0 +1,174 @@
 | 
			
		|||
#include "TimestamptzArithCompareExpr.h"
 | 
			
		||||
#include <cstddef>
 | 
			
		||||
#include <memory>
 | 
			
		||||
#include "absl/time/civil_time.h"
 | 
			
		||||
#include "absl/time/time.h"
 | 
			
		||||
#include "common/EasyAssert.h"
 | 
			
		||||
#include "common/Types.h"
 | 
			
		||||
#include "common/Vector.h"
 | 
			
		||||
#include "exec/expression/EvalCtx.h"
 | 
			
		||||
#include "exec/expression/Expr.h"
 | 
			
		||||
#include "expr/ITypeExpr.h"
 | 
			
		||||
#include "pb/plan.pb.h"
 | 
			
		||||
#include "BinaryArithOpEvalRangeExpr.h"
 | 
			
		||||
 | 
			
		||||
namespace milvus {
 | 
			
		||||
namespace exec {
 | 
			
		||||
 | 
			
		||||
std::string
 | 
			
		||||
PhyTimestamptzArithCompareExpr::ToString() const {
 | 
			
		||||
    return expr_->ToString();
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
void
 | 
			
		||||
PhyTimestamptzArithCompareExpr::Eval(EvalCtx& context, VectorPtr& result) {
 | 
			
		||||
    auto input = context.get_offset_input();
 | 
			
		||||
    SetHasOffsetInput((input != nullptr));
 | 
			
		||||
    result = ExecCompareVisitorImpl<int64_t>(input);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
template <typename T>
 | 
			
		||||
VectorPtr
 | 
			
		||||
PhyTimestamptzArithCompareExpr::ExecCompareVisitorImpl(OffsetVector* input) {
 | 
			
		||||
    // We can not use index by transforming ts_col + interval > iso_string to ts_col > iso_string - interval
 | 
			
		||||
    // Because year / month interval is not fixed days, it depends on the specific date.
 | 
			
		||||
    // So, currently, we only support the data scanning path.
 | 
			
		||||
    // In the future, one could add a switch here to check for index availability.
 | 
			
		||||
    return ExecCompareVisitorImplForAll<T>(input);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
template <typename T>
 | 
			
		||||
VectorPtr
 | 
			
		||||
PhyTimestamptzArithCompareExpr::ExecCompareVisitorImplForAll(
 | 
			
		||||
    OffsetVector* input) {
 | 
			
		||||
    if (!arg_inited_) {
 | 
			
		||||
        interval_ = expr_->interval_;
 | 
			
		||||
        compare_value_.SetValue<T>(expr_->compare_value_);
 | 
			
		||||
        arg_inited_ = true;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    auto arith_op = expr_->arith_op_;
 | 
			
		||||
    auto compare_op = expr_->compare_op_;
 | 
			
		||||
    auto compare_value = this->compare_value_.GetValue<T>();
 | 
			
		||||
    auto interval = interval_;
 | 
			
		||||
 | 
			
		||||
    if (arith_op == proto::plan::ArithOpType::Unknown) {
 | 
			
		||||
        if (!helperPhyExpr_) {  // reconstruct helper expr would cause an error
 | 
			
		||||
            proto::plan::GenericValue zeroRightOperand;
 | 
			
		||||
            zeroRightOperand.set_int64_val(0);
 | 
			
		||||
            auto helperExpr =
 | 
			
		||||
                std::make_shared<milvus::expr::BinaryArithOpEvalRangeExpr>(
 | 
			
		||||
                    expr_->timestamp_column_,
 | 
			
		||||
                    expr_->compare_op_,
 | 
			
		||||
                    proto::plan::ArithOpType::Add,
 | 
			
		||||
                    expr_->compare_value_,
 | 
			
		||||
                    zeroRightOperand);
 | 
			
		||||
            helperPhyExpr_ = std::make_shared<PhyBinaryArithOpEvalRangeExpr>(
 | 
			
		||||
                inputs_,
 | 
			
		||||
                helperExpr,
 | 
			
		||||
                "PhyTimestamptzArithCompareExprHelper",
 | 
			
		||||
                op_ctx_,
 | 
			
		||||
                segment_,
 | 
			
		||||
                active_count_,
 | 
			
		||||
                batch_size_,
 | 
			
		||||
                consistency_level_);
 | 
			
		||||
        }
 | 
			
		||||
        return helperPhyExpr_->ExecRangeVisitorImpl<T>(input);
 | 
			
		||||
    }
 | 
			
		||||
    auto real_batch_size =
 | 
			
		||||
        has_offset_input_ ? input->size() : GetNextBatchSize();
 | 
			
		||||
    if (real_batch_size == 0) {
 | 
			
		||||
        return nullptr;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    auto res_vec =
 | 
			
		||||
        std::make_shared<ColumnVector>(TargetBitmap(real_batch_size, false),
 | 
			
		||||
                                       TargetBitmap(real_batch_size, true));
 | 
			
		||||
 | 
			
		||||
    TargetBitmapView res(res_vec->GetRawData(), real_batch_size);
 | 
			
		||||
    TargetBitmapView valid_res(res_vec->GetValidRawData(), real_batch_size);
 | 
			
		||||
    auto exec_sub_batch =
 | 
			
		||||
        [ arith_op,
 | 
			
		||||
          compare_op ]<FilterType filter_type = FilterType::sequential>(
 | 
			
		||||
            const T* data,
 | 
			
		||||
            const bool* valid_data,
 | 
			
		||||
            const int32_t* offsets,
 | 
			
		||||
            const int size,
 | 
			
		||||
            TargetBitmapView res,
 | 
			
		||||
            TargetBitmapView valid_res,
 | 
			
		||||
            T compare_value,
 | 
			
		||||
            proto::plan::Interval interval) {
 | 
			
		||||
        absl::TimeZone utc = absl::UTCTimeZone();
 | 
			
		||||
        absl::Time compare_t = absl::FromUnixMicros(compare_value);
 | 
			
		||||
        for (int i = 0; i < size; ++i) {
 | 
			
		||||
            auto offset = (offsets) ? offsets[i] : i;
 | 
			
		||||
            const int64_t current_ts_us = data[i];
 | 
			
		||||
            const int op_sign =
 | 
			
		||||
                (arith_op == proto::plan::ArithOpType::Add) ? 1 : -1;
 | 
			
		||||
            absl::Time t = absl::FromUnixMicros(current_ts_us);
 | 
			
		||||
            // CivilSecond can handle calendar time for us
 | 
			
		||||
            absl::CivilSecond cs = absl::ToCivilSecond(t, utc);
 | 
			
		||||
            absl::CivilSecond new_cs(
 | 
			
		||||
                cs.year() + (interval.years() * op_sign),
 | 
			
		||||
                cs.month() + (interval.months() * op_sign),
 | 
			
		||||
                cs.day() + (interval.days() * op_sign),
 | 
			
		||||
                cs.hour() + (interval.hours() * op_sign),
 | 
			
		||||
                cs.minute() + (interval.minutes() * op_sign),
 | 
			
		||||
                cs.second() + (interval.seconds() * op_sign));
 | 
			
		||||
            absl::Time final_time = absl::FromCivil(new_cs, utc);
 | 
			
		||||
            bool match = false;
 | 
			
		||||
            switch (compare_op) {
 | 
			
		||||
                case proto::plan::OpType::Equal:
 | 
			
		||||
                    match = (final_time == compare_t);
 | 
			
		||||
                    break;
 | 
			
		||||
                case proto::plan::OpType::NotEqual:
 | 
			
		||||
                    match = (final_time != compare_t);
 | 
			
		||||
                    break;
 | 
			
		||||
                case proto::plan::OpType::GreaterThan:
 | 
			
		||||
                    match = (final_time > compare_t);
 | 
			
		||||
                    break;
 | 
			
		||||
                case proto::plan::OpType::GreaterEqual:
 | 
			
		||||
                    match = (final_time >= compare_t);
 | 
			
		||||
                    break;
 | 
			
		||||
                case proto::plan::OpType::LessThan:
 | 
			
		||||
                    match = (final_time < compare_t);
 | 
			
		||||
                    break;
 | 
			
		||||
                case proto::plan::OpType::LessEqual:
 | 
			
		||||
                    match = (final_time <= compare_t);
 | 
			
		||||
                    break;
 | 
			
		||||
                default:  // Should not happen
 | 
			
		||||
                    ThrowInfo(OpTypeInvalid,
 | 
			
		||||
                              "Unsupported compare op for "
 | 
			
		||||
                              "timestamptz_arith_compare_expr");
 | 
			
		||||
            }
 | 
			
		||||
            res[i] = match;
 | 
			
		||||
        }
 | 
			
		||||
    };
 | 
			
		||||
    int64_t processed_size;
 | 
			
		||||
    if (has_offset_input_) {
 | 
			
		||||
        processed_size = ProcessDataByOffsets<T>(exec_sub_batch,
 | 
			
		||||
                                                 std::nullptr_t{},
 | 
			
		||||
                                                 input,
 | 
			
		||||
                                                 res,
 | 
			
		||||
                                                 valid_res,
 | 
			
		||||
                                                 compare_value,
 | 
			
		||||
                                                 interval);
 | 
			
		||||
    } else {
 | 
			
		||||
        processed_size = ProcessDataChunks<T>(exec_sub_batch,
 | 
			
		||||
                                              std::nullptr_t{},
 | 
			
		||||
                                              res,
 | 
			
		||||
                                              valid_res,
 | 
			
		||||
                                              compare_value,
 | 
			
		||||
                                              interval);
 | 
			
		||||
    }
 | 
			
		||||
    AssertInfo(processed_size == real_batch_size,
 | 
			
		||||
               "internal error: expr processed rows {} not equal "
 | 
			
		||||
               "expect batch size {}",
 | 
			
		||||
               processed_size,
 | 
			
		||||
               real_batch_size);
 | 
			
		||||
 | 
			
		||||
    return res_vec;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
}  // namespace exec
 | 
			
		||||
}  // namespace milvus
 | 
			
		||||
| 
						 | 
				
			
			@ -0,0 +1,77 @@
 | 
			
		|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
 | 
			
		||||
//
 | 
			
		||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
 | 
			
		||||
// with the License. You may obtain a copy of the License at
 | 
			
		||||
//
 | 
			
		||||
// http://www.apache.org/licenses/LICENSE-2.0
 | 
			
		||||
//
 | 
			
		||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
 | 
			
		||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
 | 
			
		||||
// or implied. See the License for the specific language governing permissions and limitations under the License
 | 
			
		||||
 | 
			
		||||
#pragma once
 | 
			
		||||
 | 
			
		||||
#include <memory>
 | 
			
		||||
#include "common/Vector.h"
 | 
			
		||||
#include "exec/expression/BinaryArithOpEvalRangeExpr.h"
 | 
			
		||||
#include "exec/expression/Element.h"
 | 
			
		||||
#include "exec/expression/EvalCtx.h"
 | 
			
		||||
#include "exec/expression/Expr.h"
 | 
			
		||||
#include "pb/plan.pb.h"
 | 
			
		||||
 | 
			
		||||
namespace milvus::exec {
 | 
			
		||||
 | 
			
		||||
class PhyTimestamptzArithCompareExpr : public SegmentExpr {
 | 
			
		||||
 public:
 | 
			
		||||
    PhyTimestamptzArithCompareExpr(
 | 
			
		||||
        const std::vector<std::shared_ptr<Expr>>& input,
 | 
			
		||||
        const std::shared_ptr<const milvus::expr::TimestamptzArithCompareExpr>&
 | 
			
		||||
            expr,
 | 
			
		||||
        const std::string& name,
 | 
			
		||||
        milvus::OpContext* op_ctx,
 | 
			
		||||
        const segcore::SegmentInternalInterface* segment,
 | 
			
		||||
        int64_t active_count,
 | 
			
		||||
        int64_t batch_size,
 | 
			
		||||
        int32_t consistency_level)
 | 
			
		||||
        : SegmentExpr(std::move(input),
 | 
			
		||||
                      name,
 | 
			
		||||
                      op_ctx,
 | 
			
		||||
                      segment,
 | 
			
		||||
                      expr->timestamp_column_.field_id_,
 | 
			
		||||
                      expr->timestamp_column_.nested_path_,
 | 
			
		||||
                      DataType::TIMESTAMPTZ,
 | 
			
		||||
                      active_count,
 | 
			
		||||
                      batch_size,
 | 
			
		||||
                      consistency_level),
 | 
			
		||||
          expr_(expr) {
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    void
 | 
			
		||||
    Eval(EvalCtx& context, VectorPtr& result) override;
 | 
			
		||||
 | 
			
		||||
    std::string
 | 
			
		||||
    ToString() const override;
 | 
			
		||||
 | 
			
		||||
    bool
 | 
			
		||||
    IsSource() const override {
 | 
			
		||||
        return true;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
 private:
 | 
			
		||||
    template <typename T>
 | 
			
		||||
    VectorPtr
 | 
			
		||||
    ExecCompareVisitorImpl(OffsetVector* input);
 | 
			
		||||
 | 
			
		||||
    template <typename T>
 | 
			
		||||
    VectorPtr
 | 
			
		||||
    ExecCompareVisitorImplForAll(OffsetVector* input);
 | 
			
		||||
 | 
			
		||||
 private:
 | 
			
		||||
    std::shared_ptr<PhyBinaryArithOpEvalRangeExpr> helperPhyExpr_;
 | 
			
		||||
    std::shared_ptr<const milvus::expr::TimestamptzArithCompareExpr> expr_;
 | 
			
		||||
    bool arg_inited_{false};
 | 
			
		||||
    proto::plan::Interval interval_;
 | 
			
		||||
    SingleElement compare_value_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
}  // namespace milvus::exec
 | 
			
		||||
| 
						 | 
				
			
			@ -673,6 +673,40 @@ class BinaryArithOpEvalRangeExpr : public ITypeFilterExpr {
 | 
			
		|||
    const proto::plan::GenericValue value_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
class TimestamptzArithCompareExpr : public ITypeFilterExpr {
 | 
			
		||||
 public:
 | 
			
		||||
    TimestamptzArithCompareExpr(const ColumnInfo& timestamp_column,
 | 
			
		||||
                                const proto::plan::ArithOpType arith_op,
 | 
			
		||||
                                const proto::plan::Interval& interval,
 | 
			
		||||
                                const proto::plan::OpType compare_op,
 | 
			
		||||
                                const proto::plan::GenericValue& compare_value)
 | 
			
		||||
        : timestamp_column_(timestamp_column),
 | 
			
		||||
          arith_op_(arith_op),
 | 
			
		||||
          interval_(interval),
 | 
			
		||||
          compare_op_(compare_op),
 | 
			
		||||
          compare_value_(compare_value) {
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    std::string
 | 
			
		||||
    ToString() const override {
 | 
			
		||||
        std::stringstream ss;
 | 
			
		||||
        ss << "TimestamptzArithCompareExpr:[Column: "
 | 
			
		||||
           << timestamp_column_.ToString()
 | 
			
		||||
           << ", ArithOp: " << milvus::proto::plan::ArithOpType_Name(arith_op_)
 | 
			
		||||
           << ", Interval: " << interval_.DebugString()
 | 
			
		||||
           << ", CompareOp: " << milvus::proto::plan::OpType_Name(compare_op_)
 | 
			
		||||
           << ", CompareValue: " << compare_value_.DebugString() << "]";
 | 
			
		||||
        return ss.str();
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
 public:
 | 
			
		||||
    const ColumnInfo timestamp_column_;
 | 
			
		||||
    const proto::plan::ArithOpType arith_op_;
 | 
			
		||||
    const proto::plan::Interval interval_;
 | 
			
		||||
    const proto::plan::OpType compare_op_;
 | 
			
		||||
    const proto::plan::GenericValue compare_value_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
class NullExpr : public ITypeFilterExpr {
 | 
			
		||||
 public:
 | 
			
		||||
    explicit NullExpr(const ColumnInfo& column, NullExprType op)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -378,6 +378,7 @@ IndexFactory::CreatePrimitiveScalarIndex(
 | 
			
		|||
            return CreatePrimitiveScalarIndex<int32_t>(create_index_info,
 | 
			
		||||
                                                       file_manager_context);
 | 
			
		||||
        case DataType::INT64:
 | 
			
		||||
        case DataType::TIMESTAMPTZ:
 | 
			
		||||
            return CreatePrimitiveScalarIndex<int64_t>(create_index_info,
 | 
			
		||||
                                                       file_manager_context);
 | 
			
		||||
        case DataType::FLOAT:
 | 
			
		||||
| 
						 | 
				
			
			@ -495,6 +496,7 @@ IndexFactory::CreateScalarIndex(
 | 
			
		|||
        case DataType::DOUBLE:
 | 
			
		||||
        case DataType::VARCHAR:
 | 
			
		||||
        case DataType::STRING:
 | 
			
		||||
        case DataType::TIMESTAMPTZ:
 | 
			
		||||
            return CreatePrimitiveScalarIndex(
 | 
			
		||||
                data_type, create_index_info, file_manager_context);
 | 
			
		||||
        case DataType::ARRAY: {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -62,6 +62,7 @@ class IndexFactory {
 | 
			
		|||
            case DataType::STRING:
 | 
			
		||||
            case DataType::ARRAY:
 | 
			
		||||
            case DataType::JSON:
 | 
			
		||||
            case DataType::TIMESTAMPTZ:
 | 
			
		||||
                return CreateScalarIndex(type, config, context);
 | 
			
		||||
 | 
			
		||||
            case DataType::VECTOR_FLOAT:
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -398,6 +398,21 @@ ProtoParser::ParseBinaryRangeExprs(
 | 
			
		|||
        expr_pb.upper_inclusive());
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
expr::TypedExprPtr
 | 
			
		||||
ProtoParser::ParseTimestamptzArithCompareExprs(
 | 
			
		||||
    const proto::plan::TimestamptzArithCompareExpr& expr_pb) {
 | 
			
		||||
    auto& columnInfo = expr_pb.timestamptz_column();
 | 
			
		||||
    auto field_id = FieldId(columnInfo.field_id());
 | 
			
		||||
    auto data_type = schema->operator[](field_id).get_data_type();
 | 
			
		||||
    Assert(data_type == (DataType)columnInfo.data_type());
 | 
			
		||||
    return std::make_shared<expr::TimestamptzArithCompareExpr>(
 | 
			
		||||
        columnInfo,
 | 
			
		||||
        expr_pb.arith_op(),
 | 
			
		||||
        expr_pb.interval(),
 | 
			
		||||
        expr_pb.compare_op(),
 | 
			
		||||
        expr_pb.compare_value());
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
expr::TypedExprPtr
 | 
			
		||||
ProtoParser::ParseCallExprs(const proto::plan::CallExpr& expr_pb) {
 | 
			
		||||
    std::vector<expr::TypedExprPtr> parameters;
 | 
			
		||||
| 
						 | 
				
			
			@ -593,6 +608,11 @@ ProtoParser::ParseExprs(const proto::plan::Expr& expr_pb,
 | 
			
		|||
            result = ParseNullExprs(expr_pb.null_expr());
 | 
			
		||||
            break;
 | 
			
		||||
        }
 | 
			
		||||
        case ppe::kTimestamptzArithCompareExpr: {
 | 
			
		||||
            result = ParseTimestamptzArithCompareExprs(
 | 
			
		||||
                expr_pb.timestamptz_arith_compare_expr());
 | 
			
		||||
            break;
 | 
			
		||||
        }
 | 
			
		||||
        default: {
 | 
			
		||||
            std::string s;
 | 
			
		||||
            google::protobuf::TextFormat::PrintToString(expr_pb, &s);
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -98,6 +98,10 @@ class ProtoParser {
 | 
			
		|||
    expr::TypedExprPtr
 | 
			
		||||
    ParseUnaryRangeExprs(const proto::plan::UnaryRangeExpr& expr_pb);
 | 
			
		||||
 | 
			
		||||
    expr::TypedExprPtr
 | 
			
		||||
    ParseTimestamptzArithCompareExprs(
 | 
			
		||||
        const proto::plan::TimestamptzArithCompareExpr& expr_pb);
 | 
			
		||||
 | 
			
		||||
    expr::TypedExprPtr
 | 
			
		||||
    ParseValueExprs(const proto::plan::ValueExpr& expr_pb);
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -26,7 +26,6 @@
 | 
			
		|||
#include "common/Schema.h"
 | 
			
		||||
#include "common/TrackingStdAllocator.h"
 | 
			
		||||
#include "common/Types.h"
 | 
			
		||||
#include "log/Log.h"
 | 
			
		||||
#include "mmap/ChunkedColumn.h"
 | 
			
		||||
#include "segcore/AckResponder.h"
 | 
			
		||||
#include "segcore/ConcurrentVector.h"
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -694,6 +694,8 @@ WriteOptFieldIvfData(
 | 
			
		|||
            return GENERATE_OPT_FIELD_IVF_IMPL(DataType::INT16);
 | 
			
		||||
        case DataType::INT32:
 | 
			
		||||
            return GENERATE_OPT_FIELD_IVF_IMPL(DataType::INT32);
 | 
			
		||||
        case DataType::TIMESTAMPTZ:
 | 
			
		||||
            return GENERATE_OPT_FIELD_IVF_IMPL(DataType::TIMESTAMPTZ);
 | 
			
		||||
        case DataType::INT64:
 | 
			
		||||
            return GENERATE_OPT_FIELD_IVF_IMPL(DataType::INT64);
 | 
			
		||||
        case DataType::FLOAT:
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -269,6 +269,8 @@ GetOptFieldIvfData(const DataType& dt,
 | 
			
		|||
            return GetOptFieldIvfDataImpl<DataType::INT16>(field_datas);
 | 
			
		||||
        case DataType::INT32:
 | 
			
		||||
            return GetOptFieldIvfDataImpl<DataType::INT32>(field_datas);
 | 
			
		||||
        case DataType::TIMESTAMPTZ:
 | 
			
		||||
            return GetOptFieldIvfDataImpl<DataType::TIMESTAMPTZ>(field_datas);
 | 
			
		||||
        case DataType::INT64:
 | 
			
		||||
            return GetOptFieldIvfDataImpl<DataType::INT64>(field_datas);
 | 
			
		||||
        case DataType::FLOAT:
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -4087,8 +4087,8 @@ dependencies = [
 | 
			
		|||
 "tokio",
 | 
			
		||||
 "tonic",
 | 
			
		||||
 "tonic-build",
 | 
			
		||||
 "url",
 | 
			
		||||
 "unicode-general-category",
 | 
			
		||||
 "url",
 | 
			
		||||
 "whatlang",
 | 
			
		||||
 "zstd-sys",
 | 
			
		||||
]
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1,7 +1,8 @@
 | 
			
		|||
grammar Plan;
 | 
			
		||||
 | 
			
		||||
expr:
 | 
			
		||||
	IntegerConstant											                     # Integer
 | 
			
		||||
  Identifier (op1=(ADD | SUB) INTERVAL interval_string=StringLiteral)? op2=(LT | LE | GT | GE | EQ | NE) ISO compare_string=StringLiteral # TimestamptzCompare
 | 
			
		||||
	| IntegerConstant											                     # Integer
 | 
			
		||||
	| FloatingConstant										                     # Floating
 | 
			
		||||
	| BooleanConstant										                     # Boolean
 | 
			
		||||
	| StringLiteral											                     # String
 | 
			
		||||
| 
						 | 
				
			
			@ -64,6 +65,8 @@ EXISTS: 'exists' | 'EXISTS';
 | 
			
		|||
TEXTMATCH: 'text_match'|'TEXT_MATCH';
 | 
			
		||||
PHRASEMATCH: 'phrase_match'|'PHRASE_MATCH';
 | 
			
		||||
RANDOMSAMPLE: 'random_sample' | 'RANDOM_SAMPLE';
 | 
			
		||||
INTERVAL: 'interval' | 'INTERVAL';
 | 
			
		||||
ISO: 'iso' | 'ISO';
 | 
			
		||||
 | 
			
		||||
ADD: '+';
 | 
			
		||||
SUB: '-';
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
										
											
												File diff suppressed because one or more lines are too long
											
										
									
								
							| 
						 | 
				
			
			@ -16,41 +16,43 @@ EXISTS=15
 | 
			
		|||
TEXTMATCH=16
 | 
			
		||||
PHRASEMATCH=17
 | 
			
		||||
RANDOMSAMPLE=18
 | 
			
		||||
ADD=19
 | 
			
		||||
SUB=20
 | 
			
		||||
MUL=21
 | 
			
		||||
DIV=22
 | 
			
		||||
MOD=23
 | 
			
		||||
POW=24
 | 
			
		||||
SHL=25
 | 
			
		||||
SHR=26
 | 
			
		||||
BAND=27
 | 
			
		||||
BOR=28
 | 
			
		||||
BXOR=29
 | 
			
		||||
AND=30
 | 
			
		||||
OR=31
 | 
			
		||||
ISNULL=32
 | 
			
		||||
ISNOTNULL=33
 | 
			
		||||
BNOT=34
 | 
			
		||||
NOT=35
 | 
			
		||||
IN=36
 | 
			
		||||
EmptyArray=37
 | 
			
		||||
JSONContains=38
 | 
			
		||||
JSONContainsAll=39
 | 
			
		||||
JSONContainsAny=40
 | 
			
		||||
ArrayContains=41
 | 
			
		||||
ArrayContainsAll=42
 | 
			
		||||
ArrayContainsAny=43
 | 
			
		||||
ArrayLength=44
 | 
			
		||||
BooleanConstant=45
 | 
			
		||||
IntegerConstant=46
 | 
			
		||||
FloatingConstant=47
 | 
			
		||||
Identifier=48
 | 
			
		||||
Meta=49
 | 
			
		||||
StringLiteral=50
 | 
			
		||||
JSONIdentifier=51
 | 
			
		||||
Whitespace=52
 | 
			
		||||
Newline=53
 | 
			
		||||
INTERVAL=19
 | 
			
		||||
ISO=20
 | 
			
		||||
ADD=21
 | 
			
		||||
SUB=22
 | 
			
		||||
MUL=23
 | 
			
		||||
DIV=24
 | 
			
		||||
MOD=25
 | 
			
		||||
POW=26
 | 
			
		||||
SHL=27
 | 
			
		||||
SHR=28
 | 
			
		||||
BAND=29
 | 
			
		||||
BOR=30
 | 
			
		||||
BXOR=31
 | 
			
		||||
AND=32
 | 
			
		||||
OR=33
 | 
			
		||||
ISNULL=34
 | 
			
		||||
ISNOTNULL=35
 | 
			
		||||
BNOT=36
 | 
			
		||||
NOT=37
 | 
			
		||||
IN=38
 | 
			
		||||
EmptyArray=39
 | 
			
		||||
JSONContains=40
 | 
			
		||||
JSONContainsAll=41
 | 
			
		||||
JSONContainsAny=42
 | 
			
		||||
ArrayContains=43
 | 
			
		||||
ArrayContainsAll=44
 | 
			
		||||
ArrayContainsAny=45
 | 
			
		||||
ArrayLength=46
 | 
			
		||||
BooleanConstant=47
 | 
			
		||||
IntegerConstant=48
 | 
			
		||||
FloatingConstant=49
 | 
			
		||||
Identifier=50
 | 
			
		||||
Meta=51
 | 
			
		||||
StringLiteral=52
 | 
			
		||||
JSONIdentifier=53
 | 
			
		||||
Whitespace=54
 | 
			
		||||
Newline=55
 | 
			
		||||
'('=1
 | 
			
		||||
')'=2
 | 
			
		||||
'['=3
 | 
			
		||||
| 
						 | 
				
			
			@ -64,16 +66,16 @@ Newline=53
 | 
			
		|||
'>='=11
 | 
			
		||||
'=='=12
 | 
			
		||||
'!='=13
 | 
			
		||||
'+'=19
 | 
			
		||||
'-'=20
 | 
			
		||||
'*'=21
 | 
			
		||||
'/'=22
 | 
			
		||||
'%'=23
 | 
			
		||||
'**'=24
 | 
			
		||||
'<<'=25
 | 
			
		||||
'>>'=26
 | 
			
		||||
'&'=27
 | 
			
		||||
'|'=28
 | 
			
		||||
'^'=29
 | 
			
		||||
'~'=34
 | 
			
		||||
'$meta'=49
 | 
			
		||||
'+'=21
 | 
			
		||||
'-'=22
 | 
			
		||||
'*'=23
 | 
			
		||||
'/'=24
 | 
			
		||||
'%'=25
 | 
			
		||||
'**'=26
 | 
			
		||||
'<<'=27
 | 
			
		||||
'>>'=28
 | 
			
		||||
'&'=29
 | 
			
		||||
'|'=30
 | 
			
		||||
'^'=31
 | 
			
		||||
'~'=36
 | 
			
		||||
'$meta'=51
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
										
											
												File diff suppressed because one or more lines are too long
											
										
									
								
							| 
						 | 
				
			
			@ -16,41 +16,43 @@ EXISTS=15
 | 
			
		|||
TEXTMATCH=16
 | 
			
		||||
PHRASEMATCH=17
 | 
			
		||||
RANDOMSAMPLE=18
 | 
			
		||||
ADD=19
 | 
			
		||||
SUB=20
 | 
			
		||||
MUL=21
 | 
			
		||||
DIV=22
 | 
			
		||||
MOD=23
 | 
			
		||||
POW=24
 | 
			
		||||
SHL=25
 | 
			
		||||
SHR=26
 | 
			
		||||
BAND=27
 | 
			
		||||
BOR=28
 | 
			
		||||
BXOR=29
 | 
			
		||||
AND=30
 | 
			
		||||
OR=31
 | 
			
		||||
ISNULL=32
 | 
			
		||||
ISNOTNULL=33
 | 
			
		||||
BNOT=34
 | 
			
		||||
NOT=35
 | 
			
		||||
IN=36
 | 
			
		||||
EmptyArray=37
 | 
			
		||||
JSONContains=38
 | 
			
		||||
JSONContainsAll=39
 | 
			
		||||
JSONContainsAny=40
 | 
			
		||||
ArrayContains=41
 | 
			
		||||
ArrayContainsAll=42
 | 
			
		||||
ArrayContainsAny=43
 | 
			
		||||
ArrayLength=44
 | 
			
		||||
BooleanConstant=45
 | 
			
		||||
IntegerConstant=46
 | 
			
		||||
FloatingConstant=47
 | 
			
		||||
Identifier=48
 | 
			
		||||
Meta=49
 | 
			
		||||
StringLiteral=50
 | 
			
		||||
JSONIdentifier=51
 | 
			
		||||
Whitespace=52
 | 
			
		||||
Newline=53
 | 
			
		||||
INTERVAL=19
 | 
			
		||||
ISO=20
 | 
			
		||||
ADD=21
 | 
			
		||||
SUB=22
 | 
			
		||||
MUL=23
 | 
			
		||||
DIV=24
 | 
			
		||||
MOD=25
 | 
			
		||||
POW=26
 | 
			
		||||
SHL=27
 | 
			
		||||
SHR=28
 | 
			
		||||
BAND=29
 | 
			
		||||
BOR=30
 | 
			
		||||
BXOR=31
 | 
			
		||||
AND=32
 | 
			
		||||
OR=33
 | 
			
		||||
ISNULL=34
 | 
			
		||||
ISNOTNULL=35
 | 
			
		||||
BNOT=36
 | 
			
		||||
NOT=37
 | 
			
		||||
IN=38
 | 
			
		||||
EmptyArray=39
 | 
			
		||||
JSONContains=40
 | 
			
		||||
JSONContainsAll=41
 | 
			
		||||
JSONContainsAny=42
 | 
			
		||||
ArrayContains=43
 | 
			
		||||
ArrayContainsAll=44
 | 
			
		||||
ArrayContainsAny=45
 | 
			
		||||
ArrayLength=46
 | 
			
		||||
BooleanConstant=47
 | 
			
		||||
IntegerConstant=48
 | 
			
		||||
FloatingConstant=49
 | 
			
		||||
Identifier=50
 | 
			
		||||
Meta=51
 | 
			
		||||
StringLiteral=52
 | 
			
		||||
JSONIdentifier=53
 | 
			
		||||
Whitespace=54
 | 
			
		||||
Newline=55
 | 
			
		||||
'('=1
 | 
			
		||||
')'=2
 | 
			
		||||
'['=3
 | 
			
		||||
| 
						 | 
				
			
			@ -64,16 +66,16 @@ Newline=53
 | 
			
		|||
'>='=11
 | 
			
		||||
'=='=12
 | 
			
		||||
'!='=13
 | 
			
		||||
'+'=19
 | 
			
		||||
'-'=20
 | 
			
		||||
'*'=21
 | 
			
		||||
'/'=22
 | 
			
		||||
'%'=23
 | 
			
		||||
'**'=24
 | 
			
		||||
'<<'=25
 | 
			
		||||
'>>'=26
 | 
			
		||||
'&'=27
 | 
			
		||||
'|'=28
 | 
			
		||||
'^'=29
 | 
			
		||||
'~'=34
 | 
			
		||||
'$meta'=49
 | 
			
		||||
'+'=21
 | 
			
		||||
'-'=22
 | 
			
		||||
'*'=23
 | 
			
		||||
'/'=24
 | 
			
		||||
'%'=25
 | 
			
		||||
'**'=26
 | 
			
		||||
'<<'=27
 | 
			
		||||
'>>'=28
 | 
			
		||||
'&'=29
 | 
			
		||||
'|'=30
 | 
			
		||||
'^'=31
 | 
			
		||||
'~'=36
 | 
			
		||||
'$meta'=51
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -123,6 +123,10 @@ func (v *BasePlanVisitor) VisitUnary(ctx *UnaryContext) interface{} {
 | 
			
		|||
	return v.VisitChildren(ctx)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *BasePlanVisitor) VisitTimestamptzCompare(ctx *TimestamptzCompareContext) interface{} {
 | 
			
		||||
	return v.VisitChildren(ctx)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *BasePlanVisitor) VisitInteger(ctx *IntegerContext) interface{} {
 | 
			
		||||
	return v.VisitChildren(ctx)
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -44,39 +44,41 @@ func planlexerLexerInit() {
 | 
			
		|||
	}
 | 
			
		||||
	staticData.LiteralNames = []string{
 | 
			
		||||
		"", "'('", "')'", "'['", "','", "']'", "'{'", "'}'", "'<'", "'<='",
 | 
			
		||||
		"'>'", "'>='", "'=='", "'!='", "", "", "", "", "", "'+'", "'-'", "'*'",
 | 
			
		||||
		"'/'", "'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "",
 | 
			
		||||
		"", "'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'",
 | 
			
		||||
		"'>'", "'>='", "'=='", "'!='", "", "", "", "", "", "", "", "'+'", "'-'",
 | 
			
		||||
		"'*'", "'/'", "'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "",
 | 
			
		||||
		"", "", "", "'~'", "", "", "", "", "", "", "", "", "", "", "", "", "",
 | 
			
		||||
		"", "'$meta'",
 | 
			
		||||
	}
 | 
			
		||||
	staticData.SymbolicNames = []string{
 | 
			
		||||
		"", "", "", "", "", "", "LBRACE", "RBRACE", "LT", "LE", "GT", "GE",
 | 
			
		||||
		"EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "PHRASEMATCH", "RANDOMSAMPLE",
 | 
			
		||||
		"ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR",
 | 
			
		||||
		"BXOR", "AND", "OR", "ISNULL", "ISNOTNULL", "BNOT", "NOT", "IN", "EmptyArray",
 | 
			
		||||
		"JSONContains", "JSONContainsAll", "JSONContainsAny", "ArrayContains",
 | 
			
		||||
		"ArrayContainsAll", "ArrayContainsAny", "ArrayLength", "BooleanConstant",
 | 
			
		||||
		"IntegerConstant", "FloatingConstant", "Identifier", "Meta", "StringLiteral",
 | 
			
		||||
		"JSONIdentifier", "Whitespace", "Newline",
 | 
			
		||||
		"INTERVAL", "ISO", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL",
 | 
			
		||||
		"SHR", "BAND", "BOR", "BXOR", "AND", "OR", "ISNULL", "ISNOTNULL", "BNOT",
 | 
			
		||||
		"NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll", "JSONContainsAny",
 | 
			
		||||
		"ArrayContains", "ArrayContainsAll", "ArrayContainsAny", "ArrayLength",
 | 
			
		||||
		"BooleanConstant", "IntegerConstant", "FloatingConstant", "Identifier",
 | 
			
		||||
		"Meta", "StringLiteral", "JSONIdentifier", "Whitespace", "Newline",
 | 
			
		||||
	}
 | 
			
		||||
	staticData.RuleNames = []string{
 | 
			
		||||
		"T__0", "T__1", "T__2", "T__3", "T__4", "LBRACE", "RBRACE", "LT", "LE",
 | 
			
		||||
		"GT", "GE", "EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "PHRASEMATCH",
 | 
			
		||||
		"RANDOMSAMPLE", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "SHL", "SHR",
 | 
			
		||||
		"BAND", "BOR", "BXOR", "AND", "OR", "ISNULL", "ISNOTNULL", "BNOT", "NOT",
 | 
			
		||||
		"IN", "EmptyArray", "JSONContains", "JSONContainsAll", "JSONContainsAny",
 | 
			
		||||
		"ArrayContains", "ArrayContainsAll", "ArrayContainsAny", "ArrayLength",
 | 
			
		||||
		"BooleanConstant", "IntegerConstant", "FloatingConstant", "Identifier",
 | 
			
		||||
		"Meta", "StringLiteral", "JSONIdentifier", "EncodingPrefix", "DoubleSCharSequence",
 | 
			
		||||
		"SingleSCharSequence", "DoubleSChar", "SingleSChar", "Nondigit", "Digit",
 | 
			
		||||
		"BinaryConstant", "DecimalConstant", "OctalConstant", "HexadecimalConstant",
 | 
			
		||||
		"NonzeroDigit", "OctalDigit", "HexadecimalDigit", "HexQuad", "UniversalCharacterName",
 | 
			
		||||
		"DecimalFloatingConstant", "HexadecimalFloatingConstant", "FractionalConstant",
 | 
			
		||||
		"ExponentPart", "DigitSequence", "HexadecimalFractionalConstant", "HexadecimalDigitSequence",
 | 
			
		||||
		"BinaryExponentPart", "EscapeSequence", "Whitespace", "Newline",
 | 
			
		||||
		"RANDOMSAMPLE", "INTERVAL", "ISO", "ADD", "SUB", "MUL", "DIV", "MOD",
 | 
			
		||||
		"POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "ISNULL", "ISNOTNULL",
 | 
			
		||||
		"BNOT", "NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll",
 | 
			
		||||
		"JSONContainsAny", "ArrayContains", "ArrayContainsAll", "ArrayContainsAny",
 | 
			
		||||
		"ArrayLength", "BooleanConstant", "IntegerConstant", "FloatingConstant",
 | 
			
		||||
		"Identifier", "Meta", "StringLiteral", "JSONIdentifier", "EncodingPrefix",
 | 
			
		||||
		"DoubleSCharSequence", "SingleSCharSequence", "DoubleSChar", "SingleSChar",
 | 
			
		||||
		"Nondigit", "Digit", "BinaryConstant", "DecimalConstant", "OctalConstant",
 | 
			
		||||
		"HexadecimalConstant", "NonzeroDigit", "OctalDigit", "HexadecimalDigit",
 | 
			
		||||
		"HexQuad", "UniversalCharacterName", "DecimalFloatingConstant", "HexadecimalFloatingConstant",
 | 
			
		||||
		"FractionalConstant", "ExponentPart", "DigitSequence", "HexadecimalFractionalConstant",
 | 
			
		||||
		"HexadecimalDigitSequence", "BinaryExponentPart", "EscapeSequence",
 | 
			
		||||
		"Whitespace", "Newline",
 | 
			
		||||
	}
 | 
			
		||||
	staticData.PredictionContextCache = antlr.NewPredictionContextCache()
 | 
			
		||||
	staticData.serializedATN = []int32{
 | 
			
		||||
		4, 0, 53, 892, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2,
 | 
			
		||||
		4, 0, 55, 922, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2,
 | 
			
		||||
		4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2,
 | 
			
		||||
		10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15,
 | 
			
		||||
		7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7,
 | 
			
		||||
| 
						 | 
				
			
			@ -90,395 +92,408 @@ func planlexerLexerInit() {
 | 
			
		|||
		7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7,
 | 
			
		||||
		62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67,
 | 
			
		||||
		2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2,
 | 
			
		||||
		73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 2, 77, 7, 77, 1, 0,
 | 
			
		||||
		1, 0, 1, 1, 1, 1, 1, 2, 1, 2, 1, 3, 1, 3, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6,
 | 
			
		||||
		1, 6, 1, 7, 1, 7, 1, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 10, 1, 10, 1, 10, 1,
 | 
			
		||||
		11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13,
 | 
			
		||||
		1, 13, 1, 13, 1, 13, 3, 13, 196, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1,
 | 
			
		||||
		14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 3, 14, 210, 8, 14,
 | 
			
		||||
		1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1,
 | 
			
		||||
		15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 3, 15,
 | 
			
		||||
		232, 8, 15, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1,
 | 
			
		||||
		73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 2, 77, 7, 77, 2, 78,
 | 
			
		||||
		7, 78, 2, 79, 7, 79, 1, 0, 1, 0, 1, 1, 1, 1, 1, 2, 1, 2, 1, 3, 1, 3, 1,
 | 
			
		||||
		4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 7, 1, 7, 1, 8, 1, 8, 1, 8, 1, 9, 1,
 | 
			
		||||
		9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 13,
 | 
			
		||||
		1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 3, 13, 200, 8, 13, 1,
 | 
			
		||||
		14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14,
 | 
			
		||||
		1, 14, 3, 14, 214, 8, 14, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1,
 | 
			
		||||
		15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15,
 | 
			
		||||
		1, 15, 1, 15, 1, 15, 3, 15, 236, 8, 15, 1, 16, 1, 16, 1, 16, 1, 16, 1,
 | 
			
		||||
		16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16,
 | 
			
		||||
		1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 3, 16, 258, 8, 16, 1, 17, 1, 17, 1,
 | 
			
		||||
		1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 3, 16, 262,
 | 
			
		||||
		8, 16, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1,
 | 
			
		||||
		17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17,
 | 
			
		||||
		1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1,
 | 
			
		||||
		17, 1, 17, 1, 17, 3, 17, 286, 8, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 20,
 | 
			
		||||
		1, 20, 1, 21, 1, 21, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1,
 | 
			
		||||
		24, 1, 25, 1, 25, 1, 25, 1, 26, 1, 26, 1, 27, 1, 27, 1, 28, 1, 28, 1, 29,
 | 
			
		||||
		1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 3, 29, 321, 8, 29, 1,
 | 
			
		||||
		30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 329, 8, 30, 1, 31, 1, 31,
 | 
			
		||||
		1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1,
 | 
			
		||||
		31, 1, 31, 3, 31, 345, 8, 31, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32,
 | 
			
		||||
		1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 1,
 | 
			
		||||
		32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 3, 32, 369, 8, 32, 1, 33, 1, 33,
 | 
			
		||||
		1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 3, 34, 380, 8, 34, 1,
 | 
			
		||||
		35, 1, 35, 1, 35, 1, 35, 3, 35, 386, 8, 35, 1, 36, 1, 36, 1, 36, 5, 36,
 | 
			
		||||
		391, 8, 36, 10, 36, 12, 36, 394, 9, 36, 1, 36, 1, 36, 1, 37, 1, 37, 1,
 | 
			
		||||
		37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37,
 | 
			
		||||
		1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1,
 | 
			
		||||
		37, 1, 37, 1, 37, 3, 37, 424, 8, 37, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38,
 | 
			
		||||
		1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1,
 | 
			
		||||
		38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38,
 | 
			
		||||
		1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 1, 38, 3, 38, 460, 8,
 | 
			
		||||
		38, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39,
 | 
			
		||||
		1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1,
 | 
			
		||||
		1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 3, 17, 290, 8, 17, 1, 18, 1,
 | 
			
		||||
		18, 1, 18, 1, 18, 1, 18, 1, 18, 1, 18, 1, 18, 1, 18, 1, 18, 1, 18, 1, 18,
 | 
			
		||||
		1, 18, 1, 18, 1, 18, 1, 18, 3, 18, 308, 8, 18, 1, 19, 1, 19, 1, 19, 1,
 | 
			
		||||
		19, 1, 19, 1, 19, 3, 19, 316, 8, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, 22,
 | 
			
		||||
		1, 22, 1, 23, 1, 23, 1, 24, 1, 24, 1, 25, 1, 25, 1, 25, 1, 26, 1, 26, 1,
 | 
			
		||||
		26, 1, 27, 1, 27, 1, 27, 1, 28, 1, 28, 1, 29, 1, 29, 1, 30, 1, 30, 1, 31,
 | 
			
		||||
		1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 1, 31, 3, 31, 351, 8, 31, 1,
 | 
			
		||||
		32, 1, 32, 1, 32, 1, 32, 1, 32, 1, 32, 3, 32, 359, 8, 32, 1, 33, 1, 33,
 | 
			
		||||
		1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1,
 | 
			
		||||
		33, 1, 33, 3, 33, 375, 8, 33, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34,
 | 
			
		||||
		1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 1,
 | 
			
		||||
		34, 1, 34, 1, 34, 1, 34, 1, 34, 1, 34, 3, 34, 399, 8, 34, 1, 35, 1, 35,
 | 
			
		||||
		1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 3, 36, 410, 8, 36, 1,
 | 
			
		||||
		37, 1, 37, 1, 37, 1, 37, 3, 37, 416, 8, 37, 1, 38, 1, 38, 1, 38, 5, 38,
 | 
			
		||||
		421, 8, 38, 10, 38, 12, 38, 424, 9, 38, 1, 38, 1, 38, 1, 39, 1, 39, 1,
 | 
			
		||||
		39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39,
 | 
			
		||||
		1, 39, 1, 39, 1, 39, 3, 39, 496, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1,
 | 
			
		||||
		40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40,
 | 
			
		||||
		1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1, 39, 1,
 | 
			
		||||
		39, 1, 39, 1, 39, 3, 39, 454, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40,
 | 
			
		||||
		1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1,
 | 
			
		||||
		40, 1, 40, 1, 40, 3, 40, 526, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41,
 | 
			
		||||
		40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40,
 | 
			
		||||
		1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 3, 40, 490, 8,
 | 
			
		||||
		40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41,
 | 
			
		||||
		1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1,
 | 
			
		||||
		41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41,
 | 
			
		||||
		1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 3,
 | 
			
		||||
		41, 564, 8, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42,
 | 
			
		||||
		1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1,
 | 
			
		||||
		1, 41, 1, 41, 1, 41, 3, 41, 526, 8, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1,
 | 
			
		||||
		42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42,
 | 
			
		||||
		1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 3, 42, 602, 8, 42, 1,
 | 
			
		||||
		43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43,
 | 
			
		||||
		1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 1,
 | 
			
		||||
		42, 1, 42, 1, 42, 3, 42, 556, 8, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43,
 | 
			
		||||
		1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1,
 | 
			
		||||
		43, 1, 43, 1, 43, 3, 43, 628, 8, 43, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44,
 | 
			
		||||
		43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43,
 | 
			
		||||
		1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 3,
 | 
			
		||||
		43, 594, 8, 43, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44,
 | 
			
		||||
		1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1,
 | 
			
		||||
		44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44,
 | 
			
		||||
		1, 44, 3, 44, 657, 8, 44, 1, 45, 1, 45, 1, 45, 1, 45, 3, 45, 663, 8, 45,
 | 
			
		||||
		1, 46, 1, 46, 3, 46, 667, 8, 46, 1, 47, 1, 47, 1, 47, 5, 47, 672, 8, 47,
 | 
			
		||||
		10, 47, 12, 47, 675, 9, 47, 1, 48, 1, 48, 1, 48, 1, 48, 1, 48, 1, 48, 1,
 | 
			
		||||
		49, 3, 49, 684, 8, 49, 1, 49, 1, 49, 3, 49, 688, 8, 49, 1, 49, 1, 49, 1,
 | 
			
		||||
		49, 3, 49, 693, 8, 49, 1, 49, 3, 49, 696, 8, 49, 1, 50, 1, 50, 3, 50, 700,
 | 
			
		||||
		8, 50, 1, 50, 1, 50, 1, 50, 3, 50, 705, 8, 50, 1, 50, 1, 50, 4, 50, 709,
 | 
			
		||||
		8, 50, 11, 50, 12, 50, 710, 1, 51, 1, 51, 1, 51, 3, 51, 716, 8, 51, 1,
 | 
			
		||||
		52, 4, 52, 719, 8, 52, 11, 52, 12, 52, 720, 1, 53, 4, 53, 724, 8, 53, 11,
 | 
			
		||||
		53, 12, 53, 725, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54,
 | 
			
		||||
		735, 8, 54, 1, 55, 1, 55, 1, 55, 1, 55, 1, 55, 1, 55, 1, 55, 3, 55, 744,
 | 
			
		||||
		8, 55, 1, 56, 1, 56, 1, 57, 1, 57, 1, 58, 1, 58, 1, 58, 4, 58, 753, 8,
 | 
			
		||||
		58, 11, 58, 12, 58, 754, 1, 59, 1, 59, 5, 59, 759, 8, 59, 10, 59, 12, 59,
 | 
			
		||||
		762, 9, 59, 1, 59, 3, 59, 765, 8, 59, 1, 60, 1, 60, 5, 60, 769, 8, 60,
 | 
			
		||||
		10, 60, 12, 60, 772, 9, 60, 1, 61, 1, 61, 1, 61, 1, 61, 1, 62, 1, 62, 1,
 | 
			
		||||
		63, 1, 63, 1, 64, 1, 64, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 66, 1, 66,
 | 
			
		||||
		1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 799, 8,
 | 
			
		||||
		66, 1, 67, 1, 67, 3, 67, 803, 8, 67, 1, 67, 1, 67, 1, 67, 3, 67, 808, 8,
 | 
			
		||||
		67, 1, 68, 1, 68, 1, 68, 1, 68, 3, 68, 814, 8, 68, 1, 68, 1, 68, 1, 69,
 | 
			
		||||
		3, 69, 819, 8, 69, 1, 69, 1, 69, 1, 69, 1, 69, 1, 69, 3, 69, 826, 8, 69,
 | 
			
		||||
		1, 70, 1, 70, 3, 70, 830, 8, 70, 1, 70, 1, 70, 1, 71, 4, 71, 835, 8, 71,
 | 
			
		||||
		11, 71, 12, 71, 836, 1, 72, 3, 72, 840, 8, 72, 1, 72, 1, 72, 1, 72, 1,
 | 
			
		||||
		72, 1, 72, 3, 72, 847, 8, 72, 1, 73, 4, 73, 850, 8, 73, 11, 73, 12, 73,
 | 
			
		||||
		851, 1, 74, 1, 74, 3, 74, 856, 8, 74, 1, 74, 1, 74, 1, 75, 1, 75, 1, 75,
 | 
			
		||||
		1, 75, 1, 75, 3, 75, 865, 8, 75, 1, 75, 3, 75, 868, 8, 75, 1, 75, 1, 75,
 | 
			
		||||
		1, 75, 1, 75, 1, 75, 3, 75, 875, 8, 75, 1, 76, 4, 76, 878, 8, 76, 11, 76,
 | 
			
		||||
		12, 76, 879, 1, 76, 1, 76, 1, 77, 1, 77, 3, 77, 886, 8, 77, 1, 77, 3, 77,
 | 
			
		||||
		889, 8, 77, 1, 77, 1, 77, 0, 0, 78, 1, 1, 3, 2, 5, 3, 7, 4, 9, 5, 11, 6,
 | 
			
		||||
		1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 3, 44, 632, 8, 44, 1,
 | 
			
		||||
		45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45,
 | 
			
		||||
		1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1,
 | 
			
		||||
		45, 1, 45, 1, 45, 3, 45, 658, 8, 45, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46,
 | 
			
		||||
		1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1,
 | 
			
		||||
		46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 46,
 | 
			
		||||
		1, 46, 3, 46, 687, 8, 46, 1, 47, 1, 47, 1, 47, 1, 47, 3, 47, 693, 8, 47,
 | 
			
		||||
		1, 48, 1, 48, 3, 48, 697, 8, 48, 1, 49, 1, 49, 1, 49, 5, 49, 702, 8, 49,
 | 
			
		||||
		10, 49, 12, 49, 705, 9, 49, 1, 50, 1, 50, 1, 50, 1, 50, 1, 50, 1, 50, 1,
 | 
			
		||||
		51, 3, 51, 714, 8, 51, 1, 51, 1, 51, 3, 51, 718, 8, 51, 1, 51, 1, 51, 1,
 | 
			
		||||
		51, 3, 51, 723, 8, 51, 1, 51, 3, 51, 726, 8, 51, 1, 52, 1, 52, 3, 52, 730,
 | 
			
		||||
		8, 52, 1, 52, 1, 52, 1, 52, 3, 52, 735, 8, 52, 1, 52, 1, 52, 4, 52, 739,
 | 
			
		||||
		8, 52, 11, 52, 12, 52, 740, 1, 53, 1, 53, 1, 53, 3, 53, 746, 8, 53, 1,
 | 
			
		||||
		54, 4, 54, 749, 8, 54, 11, 54, 12, 54, 750, 1, 55, 4, 55, 754, 8, 55, 11,
 | 
			
		||||
		55, 12, 55, 755, 1, 56, 1, 56, 1, 56, 1, 56, 1, 56, 1, 56, 1, 56, 3, 56,
 | 
			
		||||
		765, 8, 56, 1, 57, 1, 57, 1, 57, 1, 57, 1, 57, 1, 57, 1, 57, 3, 57, 774,
 | 
			
		||||
		8, 57, 1, 58, 1, 58, 1, 59, 1, 59, 1, 60, 1, 60, 1, 60, 4, 60, 783, 8,
 | 
			
		||||
		60, 11, 60, 12, 60, 784, 1, 61, 1, 61, 5, 61, 789, 8, 61, 10, 61, 12, 61,
 | 
			
		||||
		792, 9, 61, 1, 61, 3, 61, 795, 8, 61, 1, 62, 1, 62, 5, 62, 799, 8, 62,
 | 
			
		||||
		10, 62, 12, 62, 802, 9, 62, 1, 63, 1, 63, 1, 63, 1, 63, 1, 64, 1, 64, 1,
 | 
			
		||||
		65, 1, 65, 1, 66, 1, 66, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 68, 1, 68,
 | 
			
		||||
		1, 68, 1, 68, 1, 68, 1, 68, 1, 68, 1, 68, 1, 68, 1, 68, 3, 68, 829, 8,
 | 
			
		||||
		68, 1, 69, 1, 69, 3, 69, 833, 8, 69, 1, 69, 1, 69, 1, 69, 3, 69, 838, 8,
 | 
			
		||||
		69, 1, 70, 1, 70, 1, 70, 1, 70, 3, 70, 844, 8, 70, 1, 70, 1, 70, 1, 71,
 | 
			
		||||
		3, 71, 849, 8, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 3, 71, 856, 8, 71,
 | 
			
		||||
		1, 72, 1, 72, 3, 72, 860, 8, 72, 1, 72, 1, 72, 1, 73, 4, 73, 865, 8, 73,
 | 
			
		||||
		11, 73, 12, 73, 866, 1, 74, 3, 74, 870, 8, 74, 1, 74, 1, 74, 1, 74, 1,
 | 
			
		||||
		74, 1, 74, 3, 74, 877, 8, 74, 1, 75, 4, 75, 880, 8, 75, 11, 75, 12, 75,
 | 
			
		||||
		881, 1, 76, 1, 76, 3, 76, 886, 8, 76, 1, 76, 1, 76, 1, 77, 1, 77, 1, 77,
 | 
			
		||||
		1, 77, 1, 77, 3, 77, 895, 8, 77, 1, 77, 3, 77, 898, 8, 77, 1, 77, 1, 77,
 | 
			
		||||
		1, 77, 1, 77, 1, 77, 3, 77, 905, 8, 77, 1, 78, 4, 78, 908, 8, 78, 11, 78,
 | 
			
		||||
		12, 78, 909, 1, 78, 1, 78, 1, 79, 1, 79, 3, 79, 916, 8, 79, 1, 79, 3, 79,
 | 
			
		||||
		919, 8, 79, 1, 79, 1, 79, 0, 0, 80, 1, 1, 3, 2, 5, 3, 7, 4, 9, 5, 11, 6,
 | 
			
		||||
		13, 7, 15, 8, 17, 9, 19, 10, 21, 11, 23, 12, 25, 13, 27, 14, 29, 15, 31,
 | 
			
		||||
		16, 33, 17, 35, 18, 37, 19, 39, 20, 41, 21, 43, 22, 45, 23, 47, 24, 49,
 | 
			
		||||
		25, 51, 26, 53, 27, 55, 28, 57, 29, 59, 30, 61, 31, 63, 32, 65, 33, 67,
 | 
			
		||||
		34, 69, 35, 71, 36, 73, 37, 75, 38, 77, 39, 79, 40, 81, 41, 83, 42, 85,
 | 
			
		||||
		43, 87, 44, 89, 45, 91, 46, 93, 47, 95, 48, 97, 49, 99, 50, 101, 51, 103,
 | 
			
		||||
		0, 105, 0, 107, 0, 109, 0, 111, 0, 113, 0, 115, 0, 117, 0, 119, 0, 121,
 | 
			
		||||
		52, 105, 53, 107, 0, 109, 0, 111, 0, 113, 0, 115, 0, 117, 0, 119, 0, 121,
 | 
			
		||||
		0, 123, 0, 125, 0, 127, 0, 129, 0, 131, 0, 133, 0, 135, 0, 137, 0, 139,
 | 
			
		||||
		0, 141, 0, 143, 0, 145, 0, 147, 0, 149, 0, 151, 0, 153, 52, 155, 53, 1,
 | 
			
		||||
		0, 16, 3, 0, 76, 76, 85, 85, 117, 117, 4, 0, 10, 10, 13, 13, 34, 34, 92,
 | 
			
		||||
		92, 4, 0, 10, 10, 13, 13, 39, 39, 92, 92, 3, 0, 65, 90, 95, 95, 97, 122,
 | 
			
		||||
		1, 0, 48, 57, 2, 0, 66, 66, 98, 98, 1, 0, 48, 49, 2, 0, 88, 88, 120, 120,
 | 
			
		||||
		1, 0, 49, 57, 1, 0, 48, 55, 3, 0, 48, 57, 65, 70, 97, 102, 2, 0, 69, 69,
 | 
			
		||||
		101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 80, 80, 112, 112, 10, 0, 34, 34,
 | 
			
		||||
		39, 39, 63, 63, 92, 92, 97, 98, 102, 102, 110, 110, 114, 114, 116, 116,
 | 
			
		||||
		118, 118, 2, 0, 9, 9, 32, 32, 940, 0, 1, 1, 0, 0, 0, 0, 3, 1, 0, 0, 0,
 | 
			
		||||
		0, 5, 1, 0, 0, 0, 0, 7, 1, 0, 0, 0, 0, 9, 1, 0, 0, 0, 0, 11, 1, 0, 0, 0,
 | 
			
		||||
		0, 13, 1, 0, 0, 0, 0, 15, 1, 0, 0, 0, 0, 17, 1, 0, 0, 0, 0, 19, 1, 0, 0,
 | 
			
		||||
		0, 0, 21, 1, 0, 0, 0, 0, 23, 1, 0, 0, 0, 0, 25, 1, 0, 0, 0, 0, 27, 1, 0,
 | 
			
		||||
		0, 0, 0, 29, 1, 0, 0, 0, 0, 31, 1, 0, 0, 0, 0, 33, 1, 0, 0, 0, 0, 35, 1,
 | 
			
		||||
		0, 0, 0, 0, 37, 1, 0, 0, 0, 0, 39, 1, 0, 0, 0, 0, 41, 1, 0, 0, 0, 0, 43,
 | 
			
		||||
		1, 0, 0, 0, 0, 45, 1, 0, 0, 0, 0, 47, 1, 0, 0, 0, 0, 49, 1, 0, 0, 0, 0,
 | 
			
		||||
		51, 1, 0, 0, 0, 0, 53, 1, 0, 0, 0, 0, 55, 1, 0, 0, 0, 0, 57, 1, 0, 0, 0,
 | 
			
		||||
		0, 59, 1, 0, 0, 0, 0, 61, 1, 0, 0, 0, 0, 63, 1, 0, 0, 0, 0, 65, 1, 0, 0,
 | 
			
		||||
		0, 0, 67, 1, 0, 0, 0, 0, 69, 1, 0, 0, 0, 0, 71, 1, 0, 0, 0, 0, 73, 1, 0,
 | 
			
		||||
		0, 0, 0, 75, 1, 0, 0, 0, 0, 77, 1, 0, 0, 0, 0, 79, 1, 0, 0, 0, 0, 81, 1,
 | 
			
		||||
		0, 0, 0, 0, 83, 1, 0, 0, 0, 0, 85, 1, 0, 0, 0, 0, 87, 1, 0, 0, 0, 0, 89,
 | 
			
		||||
		1, 0, 0, 0, 0, 91, 1, 0, 0, 0, 0, 93, 1, 0, 0, 0, 0, 95, 1, 0, 0, 0, 0,
 | 
			
		||||
		97, 1, 0, 0, 0, 0, 99, 1, 0, 0, 0, 0, 101, 1, 0, 0, 0, 0, 153, 1, 0, 0,
 | 
			
		||||
		0, 0, 155, 1, 0, 0, 0, 1, 157, 1, 0, 0, 0, 3, 159, 1, 0, 0, 0, 5, 161,
 | 
			
		||||
		1, 0, 0, 0, 7, 163, 1, 0, 0, 0, 9, 165, 1, 0, 0, 0, 11, 167, 1, 0, 0, 0,
 | 
			
		||||
		13, 169, 1, 0, 0, 0, 15, 171, 1, 0, 0, 0, 17, 173, 1, 0, 0, 0, 19, 176,
 | 
			
		||||
		1, 0, 0, 0, 21, 178, 1, 0, 0, 0, 23, 181, 1, 0, 0, 0, 25, 184, 1, 0, 0,
 | 
			
		||||
		0, 27, 195, 1, 0, 0, 0, 29, 209, 1, 0, 0, 0, 31, 231, 1, 0, 0, 0, 33, 257,
 | 
			
		||||
		1, 0, 0, 0, 35, 285, 1, 0, 0, 0, 37, 287, 1, 0, 0, 0, 39, 289, 1, 0, 0,
 | 
			
		||||
		0, 41, 291, 1, 0, 0, 0, 43, 293, 1, 0, 0, 0, 45, 295, 1, 0, 0, 0, 47, 297,
 | 
			
		||||
		1, 0, 0, 0, 49, 300, 1, 0, 0, 0, 51, 303, 1, 0, 0, 0, 53, 306, 1, 0, 0,
 | 
			
		||||
		0, 55, 308, 1, 0, 0, 0, 57, 310, 1, 0, 0, 0, 59, 320, 1, 0, 0, 0, 61, 328,
 | 
			
		||||
		1, 0, 0, 0, 63, 344, 1, 0, 0, 0, 65, 368, 1, 0, 0, 0, 67, 370, 1, 0, 0,
 | 
			
		||||
		0, 69, 379, 1, 0, 0, 0, 71, 385, 1, 0, 0, 0, 73, 387, 1, 0, 0, 0, 75, 423,
 | 
			
		||||
		1, 0, 0, 0, 77, 459, 1, 0, 0, 0, 79, 495, 1, 0, 0, 0, 81, 525, 1, 0, 0,
 | 
			
		||||
		0, 83, 563, 1, 0, 0, 0, 85, 601, 1, 0, 0, 0, 87, 627, 1, 0, 0, 0, 89, 656,
 | 
			
		||||
		1, 0, 0, 0, 91, 662, 1, 0, 0, 0, 93, 666, 1, 0, 0, 0, 95, 668, 1, 0, 0,
 | 
			
		||||
		0, 97, 676, 1, 0, 0, 0, 99, 683, 1, 0, 0, 0, 101, 699, 1, 0, 0, 0, 103,
 | 
			
		||||
		715, 1, 0, 0, 0, 105, 718, 1, 0, 0, 0, 107, 723, 1, 0, 0, 0, 109, 734,
 | 
			
		||||
		1, 0, 0, 0, 111, 743, 1, 0, 0, 0, 113, 745, 1, 0, 0, 0, 115, 747, 1, 0,
 | 
			
		||||
		0, 0, 117, 749, 1, 0, 0, 0, 119, 764, 1, 0, 0, 0, 121, 766, 1, 0, 0, 0,
 | 
			
		||||
		123, 773, 1, 0, 0, 0, 125, 777, 1, 0, 0, 0, 127, 779, 1, 0, 0, 0, 129,
 | 
			
		||||
		781, 1, 0, 0, 0, 131, 783, 1, 0, 0, 0, 133, 798, 1, 0, 0, 0, 135, 807,
 | 
			
		||||
		1, 0, 0, 0, 137, 809, 1, 0, 0, 0, 139, 825, 1, 0, 0, 0, 141, 827, 1, 0,
 | 
			
		||||
		0, 0, 143, 834, 1, 0, 0, 0, 145, 846, 1, 0, 0, 0, 147, 849, 1, 0, 0, 0,
 | 
			
		||||
		149, 853, 1, 0, 0, 0, 151, 874, 1, 0, 0, 0, 153, 877, 1, 0, 0, 0, 155,
 | 
			
		||||
		888, 1, 0, 0, 0, 157, 158, 5, 40, 0, 0, 158, 2, 1, 0, 0, 0, 159, 160, 5,
 | 
			
		||||
		41, 0, 0, 160, 4, 1, 0, 0, 0, 161, 162, 5, 91, 0, 0, 162, 6, 1, 0, 0, 0,
 | 
			
		||||
		163, 164, 5, 44, 0, 0, 164, 8, 1, 0, 0, 0, 165, 166, 5, 93, 0, 0, 166,
 | 
			
		||||
		10, 1, 0, 0, 0, 167, 168, 5, 123, 0, 0, 168, 12, 1, 0, 0, 0, 169, 170,
 | 
			
		||||
		5, 125, 0, 0, 170, 14, 1, 0, 0, 0, 171, 172, 5, 60, 0, 0, 172, 16, 1, 0,
 | 
			
		||||
		0, 0, 173, 174, 5, 60, 0, 0, 174, 175, 5, 61, 0, 0, 175, 18, 1, 0, 0, 0,
 | 
			
		||||
		176, 177, 5, 62, 0, 0, 177, 20, 1, 0, 0, 0, 178, 179, 5, 62, 0, 0, 179,
 | 
			
		||||
		180, 5, 61, 0, 0, 180, 22, 1, 0, 0, 0, 181, 182, 5, 61, 0, 0, 182, 183,
 | 
			
		||||
		5, 61, 0, 0, 183, 24, 1, 0, 0, 0, 184, 185, 5, 33, 0, 0, 185, 186, 5, 61,
 | 
			
		||||
		0, 0, 186, 26, 1, 0, 0, 0, 187, 188, 5, 108, 0, 0, 188, 189, 5, 105, 0,
 | 
			
		||||
		0, 189, 190, 5, 107, 0, 0, 190, 196, 5, 101, 0, 0, 191, 192, 5, 76, 0,
 | 
			
		||||
		0, 192, 193, 5, 73, 0, 0, 193, 194, 5, 75, 0, 0, 194, 196, 5, 69, 0, 0,
 | 
			
		||||
		195, 187, 1, 0, 0, 0, 195, 191, 1, 0, 0, 0, 196, 28, 1, 0, 0, 0, 197, 198,
 | 
			
		||||
		5, 101, 0, 0, 198, 199, 5, 120, 0, 0, 199, 200, 5, 105, 0, 0, 200, 201,
 | 
			
		||||
		5, 115, 0, 0, 201, 202, 5, 116, 0, 0, 202, 210, 5, 115, 0, 0, 203, 204,
 | 
			
		||||
		5, 69, 0, 0, 204, 205, 5, 88, 0, 0, 205, 206, 5, 73, 0, 0, 206, 207, 5,
 | 
			
		||||
		83, 0, 0, 207, 208, 5, 84, 0, 0, 208, 210, 5, 83, 0, 0, 209, 197, 1, 0,
 | 
			
		||||
		0, 0, 209, 203, 1, 0, 0, 0, 210, 30, 1, 0, 0, 0, 211, 212, 5, 116, 0, 0,
 | 
			
		||||
		212, 213, 5, 101, 0, 0, 213, 214, 5, 120, 0, 0, 214, 215, 5, 116, 0, 0,
 | 
			
		||||
		215, 216, 5, 95, 0, 0, 216, 217, 5, 109, 0, 0, 217, 218, 5, 97, 0, 0, 218,
 | 
			
		||||
		219, 5, 116, 0, 0, 219, 220, 5, 99, 0, 0, 220, 232, 5, 104, 0, 0, 221,
 | 
			
		||||
		222, 5, 84, 0, 0, 222, 223, 5, 69, 0, 0, 223, 224, 5, 88, 0, 0, 224, 225,
 | 
			
		||||
		5, 84, 0, 0, 225, 226, 5, 95, 0, 0, 226, 227, 5, 77, 0, 0, 227, 228, 5,
 | 
			
		||||
		65, 0, 0, 228, 229, 5, 84, 0, 0, 229, 230, 5, 67, 0, 0, 230, 232, 5, 72,
 | 
			
		||||
		0, 0, 231, 211, 1, 0, 0, 0, 231, 221, 1, 0, 0, 0, 232, 32, 1, 0, 0, 0,
 | 
			
		||||
		233, 234, 5, 112, 0, 0, 234, 235, 5, 104, 0, 0, 235, 236, 5, 114, 0, 0,
 | 
			
		||||
		236, 237, 5, 97, 0, 0, 237, 238, 5, 115, 0, 0, 238, 239, 5, 101, 0, 0,
 | 
			
		||||
		239, 240, 5, 95, 0, 0, 240, 241, 5, 109, 0, 0, 241, 242, 5, 97, 0, 0, 242,
 | 
			
		||||
		243, 5, 116, 0, 0, 243, 244, 5, 99, 0, 0, 244, 258, 5, 104, 0, 0, 245,
 | 
			
		||||
		246, 5, 80, 0, 0, 246, 247, 5, 72, 0, 0, 247, 248, 5, 82, 0, 0, 248, 249,
 | 
			
		||||
		5, 65, 0, 0, 249, 250, 5, 83, 0, 0, 250, 251, 5, 69, 0, 0, 251, 252, 5,
 | 
			
		||||
		95, 0, 0, 252, 253, 5, 77, 0, 0, 253, 254, 5, 65, 0, 0, 254, 255, 5, 84,
 | 
			
		||||
		0, 0, 255, 256, 5, 67, 0, 0, 256, 258, 5, 72, 0, 0, 257, 233, 1, 0, 0,
 | 
			
		||||
		0, 257, 245, 1, 0, 0, 0, 258, 34, 1, 0, 0, 0, 259, 260, 5, 114, 0, 0, 260,
 | 
			
		||||
		261, 5, 97, 0, 0, 261, 262, 5, 110, 0, 0, 262, 263, 5, 100, 0, 0, 263,
 | 
			
		||||
		264, 5, 111, 0, 0, 264, 265, 5, 109, 0, 0, 265, 266, 5, 95, 0, 0, 266,
 | 
			
		||||
		267, 5, 115, 0, 0, 267, 268, 5, 97, 0, 0, 268, 269, 5, 109, 0, 0, 269,
 | 
			
		||||
		270, 5, 112, 0, 0, 270, 271, 5, 108, 0, 0, 271, 286, 5, 101, 0, 0, 272,
 | 
			
		||||
		273, 5, 82, 0, 0, 273, 274, 5, 65, 0, 0, 274, 275, 5, 78, 0, 0, 275, 276,
 | 
			
		||||
		5, 68, 0, 0, 276, 277, 5, 79, 0, 0, 277, 278, 5, 77, 0, 0, 278, 279, 5,
 | 
			
		||||
		95, 0, 0, 279, 280, 5, 83, 0, 0, 280, 281, 5, 65, 0, 0, 281, 282, 5, 77,
 | 
			
		||||
		0, 0, 282, 283, 5, 80, 0, 0, 283, 284, 5, 76, 0, 0, 284, 286, 5, 69, 0,
 | 
			
		||||
		0, 285, 259, 1, 0, 0, 0, 285, 272, 1, 0, 0, 0, 286, 36, 1, 0, 0, 0, 287,
 | 
			
		||||
		288, 5, 43, 0, 0, 288, 38, 1, 0, 0, 0, 289, 290, 5, 45, 0, 0, 290, 40,
 | 
			
		||||
		1, 0, 0, 0, 291, 292, 5, 42, 0, 0, 292, 42, 1, 0, 0, 0, 293, 294, 5, 47,
 | 
			
		||||
		0, 0, 294, 44, 1, 0, 0, 0, 295, 296, 5, 37, 0, 0, 296, 46, 1, 0, 0, 0,
 | 
			
		||||
		297, 298, 5, 42, 0, 0, 298, 299, 5, 42, 0, 0, 299, 48, 1, 0, 0, 0, 300,
 | 
			
		||||
		301, 5, 60, 0, 0, 301, 302, 5, 60, 0, 0, 302, 50, 1, 0, 0, 0, 303, 304,
 | 
			
		||||
		5, 62, 0, 0, 304, 305, 5, 62, 0, 0, 305, 52, 1, 0, 0, 0, 306, 307, 5, 38,
 | 
			
		||||
		0, 0, 307, 54, 1, 0, 0, 0, 308, 309, 5, 124, 0, 0, 309, 56, 1, 0, 0, 0,
 | 
			
		||||
		310, 311, 5, 94, 0, 0, 311, 58, 1, 0, 0, 0, 312, 313, 5, 38, 0, 0, 313,
 | 
			
		||||
		321, 5, 38, 0, 0, 314, 315, 5, 97, 0, 0, 315, 316, 5, 110, 0, 0, 316, 321,
 | 
			
		||||
		5, 100, 0, 0, 317, 318, 5, 65, 0, 0, 318, 319, 5, 78, 0, 0, 319, 321, 5,
 | 
			
		||||
		68, 0, 0, 320, 312, 1, 0, 0, 0, 320, 314, 1, 0, 0, 0, 320, 317, 1, 0, 0,
 | 
			
		||||
		0, 321, 60, 1, 0, 0, 0, 322, 323, 5, 124, 0, 0, 323, 329, 5, 124, 0, 0,
 | 
			
		||||
		324, 325, 5, 111, 0, 0, 325, 329, 5, 114, 0, 0, 326, 327, 5, 79, 0, 0,
 | 
			
		||||
		327, 329, 5, 82, 0, 0, 328, 322, 1, 0, 0, 0, 328, 324, 1, 0, 0, 0, 328,
 | 
			
		||||
		326, 1, 0, 0, 0, 329, 62, 1, 0, 0, 0, 330, 331, 5, 105, 0, 0, 331, 332,
 | 
			
		||||
		5, 115, 0, 0, 332, 333, 5, 32, 0, 0, 333, 334, 5, 110, 0, 0, 334, 335,
 | 
			
		||||
		5, 117, 0, 0, 335, 336, 5, 108, 0, 0, 336, 345, 5, 108, 0, 0, 337, 338,
 | 
			
		||||
		5, 73, 0, 0, 338, 339, 5, 83, 0, 0, 339, 340, 5, 32, 0, 0, 340, 341, 5,
 | 
			
		||||
		78, 0, 0, 341, 342, 5, 85, 0, 0, 342, 343, 5, 76, 0, 0, 343, 345, 5, 76,
 | 
			
		||||
		0, 0, 344, 330, 1, 0, 0, 0, 344, 337, 1, 0, 0, 0, 345, 64, 1, 0, 0, 0,
 | 
			
		||||
		346, 347, 5, 105, 0, 0, 347, 348, 5, 115, 0, 0, 348, 349, 5, 32, 0, 0,
 | 
			
		||||
		349, 350, 5, 110, 0, 0, 350, 351, 5, 111, 0, 0, 351, 352, 5, 116, 0, 0,
 | 
			
		||||
		352, 353, 5, 32, 0, 0, 353, 354, 5, 110, 0, 0, 354, 355, 5, 117, 0, 0,
 | 
			
		||||
		355, 356, 5, 108, 0, 0, 356, 369, 5, 108, 0, 0, 357, 358, 5, 73, 0, 0,
 | 
			
		||||
		358, 359, 5, 83, 0, 0, 359, 360, 5, 32, 0, 0, 360, 361, 5, 78, 0, 0, 361,
 | 
			
		||||
		362, 5, 79, 0, 0, 362, 363, 5, 84, 0, 0, 363, 364, 5, 32, 0, 0, 364, 365,
 | 
			
		||||
		5, 78, 0, 0, 365, 366, 5, 85, 0, 0, 366, 367, 5, 76, 0, 0, 367, 369, 5,
 | 
			
		||||
		76, 0, 0, 368, 346, 1, 0, 0, 0, 368, 357, 1, 0, 0, 0, 369, 66, 1, 0, 0,
 | 
			
		||||
		0, 370, 371, 5, 126, 0, 0, 371, 68, 1, 0, 0, 0, 372, 380, 5, 33, 0, 0,
 | 
			
		||||
		373, 374, 5, 110, 0, 0, 374, 375, 5, 111, 0, 0, 375, 380, 5, 116, 0, 0,
 | 
			
		||||
		376, 377, 5, 78, 0, 0, 377, 378, 5, 79, 0, 0, 378, 380, 5, 84, 0, 0, 379,
 | 
			
		||||
		372, 1, 0, 0, 0, 379, 373, 1, 0, 0, 0, 379, 376, 1, 0, 0, 0, 380, 70, 1,
 | 
			
		||||
		0, 0, 0, 381, 382, 5, 105, 0, 0, 382, 386, 5, 110, 0, 0, 383, 384, 5, 73,
 | 
			
		||||
		0, 0, 384, 386, 5, 78, 0, 0, 385, 381, 1, 0, 0, 0, 385, 383, 1, 0, 0, 0,
 | 
			
		||||
		386, 72, 1, 0, 0, 0, 387, 392, 5, 91, 0, 0, 388, 391, 3, 153, 76, 0, 389,
 | 
			
		||||
		391, 3, 155, 77, 0, 390, 388, 1, 0, 0, 0, 390, 389, 1, 0, 0, 0, 391, 394,
 | 
			
		||||
		1, 0, 0, 0, 392, 390, 1, 0, 0, 0, 392, 393, 1, 0, 0, 0, 393, 395, 1, 0,
 | 
			
		||||
		0, 0, 394, 392, 1, 0, 0, 0, 395, 396, 5, 93, 0, 0, 396, 74, 1, 0, 0, 0,
 | 
			
		||||
		397, 398, 5, 106, 0, 0, 398, 399, 5, 115, 0, 0, 399, 400, 5, 111, 0, 0,
 | 
			
		||||
		400, 401, 5, 110, 0, 0, 401, 402, 5, 95, 0, 0, 402, 403, 5, 99, 0, 0, 403,
 | 
			
		||||
		404, 5, 111, 0, 0, 404, 405, 5, 110, 0, 0, 405, 406, 5, 116, 0, 0, 406,
 | 
			
		||||
		407, 5, 97, 0, 0, 407, 408, 5, 105, 0, 0, 408, 409, 5, 110, 0, 0, 409,
 | 
			
		||||
		424, 5, 115, 0, 0, 410, 411, 5, 74, 0, 0, 411, 412, 5, 83, 0, 0, 412, 413,
 | 
			
		||||
		5, 79, 0, 0, 413, 414, 5, 78, 0, 0, 414, 415, 5, 95, 0, 0, 415, 416, 5,
 | 
			
		||||
		67, 0, 0, 416, 417, 5, 79, 0, 0, 417, 418, 5, 78, 0, 0, 418, 419, 5, 84,
 | 
			
		||||
		0, 0, 419, 420, 5, 65, 0, 0, 420, 421, 5, 73, 0, 0, 421, 422, 5, 78, 0,
 | 
			
		||||
		0, 422, 424, 5, 83, 0, 0, 423, 397, 1, 0, 0, 0, 423, 410, 1, 0, 0, 0, 424,
 | 
			
		||||
		76, 1, 0, 0, 0, 425, 426, 5, 106, 0, 0, 426, 427, 5, 115, 0, 0, 427, 428,
 | 
			
		||||
		5, 111, 0, 0, 428, 429, 5, 110, 0, 0, 429, 430, 5, 95, 0, 0, 430, 431,
 | 
			
		||||
		5, 99, 0, 0, 431, 432, 5, 111, 0, 0, 432, 433, 5, 110, 0, 0, 433, 434,
 | 
			
		||||
		5, 116, 0, 0, 434, 435, 5, 97, 0, 0, 435, 436, 5, 105, 0, 0, 436, 437,
 | 
			
		||||
		5, 110, 0, 0, 437, 438, 5, 115, 0, 0, 438, 439, 5, 95, 0, 0, 439, 440,
 | 
			
		||||
		5, 97, 0, 0, 440, 441, 5, 108, 0, 0, 441, 460, 5, 108, 0, 0, 442, 443,
 | 
			
		||||
		5, 74, 0, 0, 443, 444, 5, 83, 0, 0, 444, 445, 5, 79, 0, 0, 445, 446, 5,
 | 
			
		||||
		78, 0, 0, 446, 447, 5, 95, 0, 0, 447, 448, 5, 67, 0, 0, 448, 449, 5, 79,
 | 
			
		||||
		0, 0, 449, 450, 5, 78, 0, 0, 450, 451, 5, 84, 0, 0, 451, 452, 5, 65, 0,
 | 
			
		||||
		0, 452, 453, 5, 73, 0, 0, 453, 454, 5, 78, 0, 0, 454, 455, 5, 83, 0, 0,
 | 
			
		||||
		455, 456, 5, 95, 0, 0, 456, 457, 5, 65, 0, 0, 457, 458, 5, 76, 0, 0, 458,
 | 
			
		||||
		460, 5, 76, 0, 0, 459, 425, 1, 0, 0, 0, 459, 442, 1, 0, 0, 0, 460, 78,
 | 
			
		||||
		1, 0, 0, 0, 461, 462, 5, 106, 0, 0, 462, 463, 5, 115, 0, 0, 463, 464, 5,
 | 
			
		||||
		111, 0, 0, 464, 465, 5, 110, 0, 0, 465, 466, 5, 95, 0, 0, 466, 467, 5,
 | 
			
		||||
		99, 0, 0, 467, 468, 5, 111, 0, 0, 468, 469, 5, 110, 0, 0, 469, 470, 5,
 | 
			
		||||
		116, 0, 0, 470, 471, 5, 97, 0, 0, 471, 472, 5, 105, 0, 0, 472, 473, 5,
 | 
			
		||||
		110, 0, 0, 473, 474, 5, 115, 0, 0, 474, 475, 5, 95, 0, 0, 475, 476, 5,
 | 
			
		||||
		97, 0, 0, 476, 477, 5, 110, 0, 0, 477, 496, 5, 121, 0, 0, 478, 479, 5,
 | 
			
		||||
		74, 0, 0, 479, 480, 5, 83, 0, 0, 480, 481, 5, 79, 0, 0, 481, 482, 5, 78,
 | 
			
		||||
		0, 0, 482, 483, 5, 95, 0, 0, 483, 484, 5, 67, 0, 0, 484, 485, 5, 79, 0,
 | 
			
		||||
		0, 485, 486, 5, 78, 0, 0, 486, 487, 5, 84, 0, 0, 487, 488, 5, 65, 0, 0,
 | 
			
		||||
		488, 489, 5, 73, 0, 0, 489, 490, 5, 78, 0, 0, 490, 491, 5, 83, 0, 0, 491,
 | 
			
		||||
		492, 5, 95, 0, 0, 492, 493, 5, 65, 0, 0, 493, 494, 5, 78, 0, 0, 494, 496,
 | 
			
		||||
		5, 89, 0, 0, 495, 461, 1, 0, 0, 0, 495, 478, 1, 0, 0, 0, 496, 80, 1, 0,
 | 
			
		||||
		0, 0, 497, 498, 5, 97, 0, 0, 498, 499, 5, 114, 0, 0, 499, 500, 5, 114,
 | 
			
		||||
		0, 0, 500, 501, 5, 97, 0, 0, 501, 502, 5, 121, 0, 0, 502, 503, 5, 95, 0,
 | 
			
		||||
		0, 503, 504, 5, 99, 0, 0, 504, 505, 5, 111, 0, 0, 505, 506, 5, 110, 0,
 | 
			
		||||
		0, 506, 507, 5, 116, 0, 0, 507, 508, 5, 97, 0, 0, 508, 509, 5, 105, 0,
 | 
			
		||||
		0, 509, 510, 5, 110, 0, 0, 510, 526, 5, 115, 0, 0, 511, 512, 5, 65, 0,
 | 
			
		||||
		0, 512, 513, 5, 82, 0, 0, 513, 514, 5, 82, 0, 0, 514, 515, 5, 65, 0, 0,
 | 
			
		||||
		515, 516, 5, 89, 0, 0, 516, 517, 5, 95, 0, 0, 517, 518, 5, 67, 0, 0, 518,
 | 
			
		||||
		519, 5, 79, 0, 0, 519, 520, 5, 78, 0, 0, 520, 521, 5, 84, 0, 0, 521, 522,
 | 
			
		||||
		5, 65, 0, 0, 522, 523, 5, 73, 0, 0, 523, 524, 5, 78, 0, 0, 524, 526, 5,
 | 
			
		||||
		83, 0, 0, 525, 497, 1, 0, 0, 0, 525, 511, 1, 0, 0, 0, 526, 82, 1, 0, 0,
 | 
			
		||||
		0, 527, 528, 5, 97, 0, 0, 528, 529, 5, 114, 0, 0, 529, 530, 5, 114, 0,
 | 
			
		||||
		0, 530, 531, 5, 97, 0, 0, 531, 532, 5, 121, 0, 0, 532, 533, 5, 95, 0, 0,
 | 
			
		||||
		533, 534, 5, 99, 0, 0, 534, 535, 5, 111, 0, 0, 535, 536, 5, 110, 0, 0,
 | 
			
		||||
		536, 537, 5, 116, 0, 0, 537, 538, 5, 97, 0, 0, 538, 539, 5, 105, 0, 0,
 | 
			
		||||
		539, 540, 5, 110, 0, 0, 540, 541, 5, 115, 0, 0, 541, 542, 5, 95, 0, 0,
 | 
			
		||||
		542, 543, 5, 97, 0, 0, 543, 544, 5, 108, 0, 0, 544, 564, 5, 108, 0, 0,
 | 
			
		||||
		545, 546, 5, 65, 0, 0, 546, 547, 5, 82, 0, 0, 547, 548, 5, 82, 0, 0, 548,
 | 
			
		||||
		549, 5, 65, 0, 0, 549, 550, 5, 89, 0, 0, 550, 551, 5, 95, 0, 0, 551, 552,
 | 
			
		||||
		5, 67, 0, 0, 552, 553, 5, 79, 0, 0, 553, 554, 5, 78, 0, 0, 554, 555, 5,
 | 
			
		||||
		84, 0, 0, 555, 556, 5, 65, 0, 0, 556, 557, 5, 73, 0, 0, 557, 558, 5, 78,
 | 
			
		||||
		0, 0, 558, 559, 5, 83, 0, 0, 559, 560, 5, 95, 0, 0, 560, 561, 5, 65, 0,
 | 
			
		||||
		0, 561, 562, 5, 76, 0, 0, 562, 564, 5, 76, 0, 0, 563, 527, 1, 0, 0, 0,
 | 
			
		||||
		563, 545, 1, 0, 0, 0, 564, 84, 1, 0, 0, 0, 565, 566, 5, 97, 0, 0, 566,
 | 
			
		||||
		567, 5, 114, 0, 0, 567, 568, 5, 114, 0, 0, 568, 569, 5, 97, 0, 0, 569,
 | 
			
		||||
		570, 5, 121, 0, 0, 570, 571, 5, 95, 0, 0, 571, 572, 5, 99, 0, 0, 572, 573,
 | 
			
		||||
		5, 111, 0, 0, 573, 574, 5, 110, 0, 0, 574, 575, 5, 116, 0, 0, 575, 576,
 | 
			
		||||
		5, 97, 0, 0, 576, 577, 5, 105, 0, 0, 577, 578, 5, 110, 0, 0, 578, 579,
 | 
			
		||||
		5, 115, 0, 0, 579, 580, 5, 95, 0, 0, 580, 581, 5, 97, 0, 0, 581, 582, 5,
 | 
			
		||||
		110, 0, 0, 582, 602, 5, 121, 0, 0, 583, 584, 5, 65, 0, 0, 584, 585, 5,
 | 
			
		||||
		82, 0, 0, 585, 586, 5, 82, 0, 0, 586, 587, 5, 65, 0, 0, 587, 588, 5, 89,
 | 
			
		||||
		0, 0, 588, 589, 5, 95, 0, 0, 589, 590, 5, 67, 0, 0, 590, 591, 5, 79, 0,
 | 
			
		||||
		0, 591, 592, 5, 78, 0, 0, 592, 593, 5, 84, 0, 0, 593, 594, 5, 65, 0, 0,
 | 
			
		||||
		594, 595, 5, 73, 0, 0, 595, 596, 5, 78, 0, 0, 596, 597, 5, 83, 0, 0, 597,
 | 
			
		||||
		598, 5, 95, 0, 0, 598, 599, 5, 65, 0, 0, 599, 600, 5, 78, 0, 0, 600, 602,
 | 
			
		||||
		5, 89, 0, 0, 601, 565, 1, 0, 0, 0, 601, 583, 1, 0, 0, 0, 602, 86, 1, 0,
 | 
			
		||||
		0, 0, 603, 604, 5, 97, 0, 0, 604, 605, 5, 114, 0, 0, 605, 606, 5, 114,
 | 
			
		||||
		0, 0, 606, 607, 5, 97, 0, 0, 607, 608, 5, 121, 0, 0, 608, 609, 5, 95, 0,
 | 
			
		||||
		0, 609, 610, 5, 108, 0, 0, 610, 611, 5, 101, 0, 0, 611, 612, 5, 110, 0,
 | 
			
		||||
		0, 612, 613, 5, 103, 0, 0, 613, 614, 5, 116, 0, 0, 614, 628, 5, 104, 0,
 | 
			
		||||
		0, 615, 616, 5, 65, 0, 0, 616, 617, 5, 82, 0, 0, 617, 618, 5, 82, 0, 0,
 | 
			
		||||
		618, 619, 5, 65, 0, 0, 619, 620, 5, 89, 0, 0, 620, 621, 5, 95, 0, 0, 621,
 | 
			
		||||
		622, 5, 76, 0, 0, 622, 623, 5, 69, 0, 0, 623, 624, 5, 78, 0, 0, 624, 625,
 | 
			
		||||
		5, 71, 0, 0, 625, 626, 5, 84, 0, 0, 626, 628, 5, 72, 0, 0, 627, 603, 1,
 | 
			
		||||
		0, 0, 0, 627, 615, 1, 0, 0, 0, 628, 88, 1, 0, 0, 0, 629, 630, 5, 116, 0,
 | 
			
		||||
		0, 630, 631, 5, 114, 0, 0, 631, 632, 5, 117, 0, 0, 632, 657, 5, 101, 0,
 | 
			
		||||
		0, 633, 634, 5, 84, 0, 0, 634, 635, 5, 114, 0, 0, 635, 636, 5, 117, 0,
 | 
			
		||||
		0, 636, 657, 5, 101, 0, 0, 637, 638, 5, 84, 0, 0, 638, 639, 5, 82, 0, 0,
 | 
			
		||||
		639, 640, 5, 85, 0, 0, 640, 657, 5, 69, 0, 0, 641, 642, 5, 102, 0, 0, 642,
 | 
			
		||||
		643, 5, 97, 0, 0, 643, 644, 5, 108, 0, 0, 644, 645, 5, 115, 0, 0, 645,
 | 
			
		||||
		657, 5, 101, 0, 0, 646, 647, 5, 70, 0, 0, 647, 648, 5, 97, 0, 0, 648, 649,
 | 
			
		||||
		5, 108, 0, 0, 649, 650, 5, 115, 0, 0, 650, 657, 5, 101, 0, 0, 651, 652,
 | 
			
		||||
		5, 70, 0, 0, 652, 653, 5, 65, 0, 0, 653, 654, 5, 76, 0, 0, 654, 655, 5,
 | 
			
		||||
		83, 0, 0, 655, 657, 5, 69, 0, 0, 656, 629, 1, 0, 0, 0, 656, 633, 1, 0,
 | 
			
		||||
		0, 0, 656, 637, 1, 0, 0, 0, 656, 641, 1, 0, 0, 0, 656, 646, 1, 0, 0, 0,
 | 
			
		||||
		656, 651, 1, 0, 0, 0, 657, 90, 1, 0, 0, 0, 658, 663, 3, 119, 59, 0, 659,
 | 
			
		||||
		663, 3, 121, 60, 0, 660, 663, 3, 123, 61, 0, 661, 663, 3, 117, 58, 0, 662,
 | 
			
		||||
		658, 1, 0, 0, 0, 662, 659, 1, 0, 0, 0, 662, 660, 1, 0, 0, 0, 662, 661,
 | 
			
		||||
		1, 0, 0, 0, 663, 92, 1, 0, 0, 0, 664, 667, 3, 135, 67, 0, 665, 667, 3,
 | 
			
		||||
		137, 68, 0, 666, 664, 1, 0, 0, 0, 666, 665, 1, 0, 0, 0, 667, 94, 1, 0,
 | 
			
		||||
		0, 0, 668, 673, 3, 113, 56, 0, 669, 672, 3, 113, 56, 0, 670, 672, 3, 115,
 | 
			
		||||
		57, 0, 671, 669, 1, 0, 0, 0, 671, 670, 1, 0, 0, 0, 672, 675, 1, 0, 0, 0,
 | 
			
		||||
		673, 671, 1, 0, 0, 0, 673, 674, 1, 0, 0, 0, 674, 96, 1, 0, 0, 0, 675, 673,
 | 
			
		||||
		1, 0, 0, 0, 676, 677, 5, 36, 0, 0, 677, 678, 5, 109, 0, 0, 678, 679, 5,
 | 
			
		||||
		101, 0, 0, 679, 680, 5, 116, 0, 0, 680, 681, 5, 97, 0, 0, 681, 98, 1, 0,
 | 
			
		||||
		0, 0, 682, 684, 3, 103, 51, 0, 683, 682, 1, 0, 0, 0, 683, 684, 1, 0, 0,
 | 
			
		||||
		0, 684, 695, 1, 0, 0, 0, 685, 687, 5, 34, 0, 0, 686, 688, 3, 105, 52, 0,
 | 
			
		||||
		687, 686, 1, 0, 0, 0, 687, 688, 1, 0, 0, 0, 688, 689, 1, 0, 0, 0, 689,
 | 
			
		||||
		696, 5, 34, 0, 0, 690, 692, 5, 39, 0, 0, 691, 693, 3, 107, 53, 0, 692,
 | 
			
		||||
		691, 1, 0, 0, 0, 692, 693, 1, 0, 0, 0, 693, 694, 1, 0, 0, 0, 694, 696,
 | 
			
		||||
		5, 39, 0, 0, 695, 685, 1, 0, 0, 0, 695, 690, 1, 0, 0, 0, 696, 100, 1, 0,
 | 
			
		||||
		0, 0, 697, 700, 3, 95, 47, 0, 698, 700, 3, 97, 48, 0, 699, 697, 1, 0, 0,
 | 
			
		||||
		0, 699, 698, 1, 0, 0, 0, 700, 708, 1, 0, 0, 0, 701, 704, 5, 91, 0, 0, 702,
 | 
			
		||||
		705, 3, 99, 49, 0, 703, 705, 3, 119, 59, 0, 704, 702, 1, 0, 0, 0, 704,
 | 
			
		||||
		703, 1, 0, 0, 0, 705, 706, 1, 0, 0, 0, 706, 707, 5, 93, 0, 0, 707, 709,
 | 
			
		||||
		1, 0, 0, 0, 708, 701, 1, 0, 0, 0, 709, 710, 1, 0, 0, 0, 710, 708, 1, 0,
 | 
			
		||||
		0, 0, 710, 711, 1, 0, 0, 0, 711, 102, 1, 0, 0, 0, 712, 713, 5, 117, 0,
 | 
			
		||||
		0, 713, 716, 5, 56, 0, 0, 714, 716, 7, 0, 0, 0, 715, 712, 1, 0, 0, 0, 715,
 | 
			
		||||
		714, 1, 0, 0, 0, 716, 104, 1, 0, 0, 0, 717, 719, 3, 109, 54, 0, 718, 717,
 | 
			
		||||
		1, 0, 0, 0, 719, 720, 1, 0, 0, 0, 720, 718, 1, 0, 0, 0, 720, 721, 1, 0,
 | 
			
		||||
		0, 0, 721, 106, 1, 0, 0, 0, 722, 724, 3, 111, 55, 0, 723, 722, 1, 0, 0,
 | 
			
		||||
		0, 724, 725, 1, 0, 0, 0, 725, 723, 1, 0, 0, 0, 725, 726, 1, 0, 0, 0, 726,
 | 
			
		||||
		108, 1, 0, 0, 0, 727, 735, 8, 1, 0, 0, 728, 735, 3, 151, 75, 0, 729, 730,
 | 
			
		||||
		5, 92, 0, 0, 730, 735, 5, 10, 0, 0, 731, 732, 5, 92, 0, 0, 732, 733, 5,
 | 
			
		||||
		13, 0, 0, 733, 735, 5, 10, 0, 0, 734, 727, 1, 0, 0, 0, 734, 728, 1, 0,
 | 
			
		||||
		0, 0, 734, 729, 1, 0, 0, 0, 734, 731, 1, 0, 0, 0, 735, 110, 1, 0, 0, 0,
 | 
			
		||||
		736, 744, 8, 2, 0, 0, 737, 744, 3, 151, 75, 0, 738, 739, 5, 92, 0, 0, 739,
 | 
			
		||||
		744, 5, 10, 0, 0, 740, 741, 5, 92, 0, 0, 741, 742, 5, 13, 0, 0, 742, 744,
 | 
			
		||||
		5, 10, 0, 0, 743, 736, 1, 0, 0, 0, 743, 737, 1, 0, 0, 0, 743, 738, 1, 0,
 | 
			
		||||
		0, 0, 743, 740, 1, 0, 0, 0, 744, 112, 1, 0, 0, 0, 745, 746, 7, 3, 0, 0,
 | 
			
		||||
		746, 114, 1, 0, 0, 0, 747, 748, 7, 4, 0, 0, 748, 116, 1, 0, 0, 0, 749,
 | 
			
		||||
		750, 5, 48, 0, 0, 750, 752, 7, 5, 0, 0, 751, 753, 7, 6, 0, 0, 752, 751,
 | 
			
		||||
		1, 0, 0, 0, 753, 754, 1, 0, 0, 0, 754, 752, 1, 0, 0, 0, 754, 755, 1, 0,
 | 
			
		||||
		0, 0, 755, 118, 1, 0, 0, 0, 756, 760, 3, 125, 62, 0, 757, 759, 3, 115,
 | 
			
		||||
		57, 0, 758, 757, 1, 0, 0, 0, 759, 762, 1, 0, 0, 0, 760, 758, 1, 0, 0, 0,
 | 
			
		||||
		760, 761, 1, 0, 0, 0, 761, 765, 1, 0, 0, 0, 762, 760, 1, 0, 0, 0, 763,
 | 
			
		||||
		765, 5, 48, 0, 0, 764, 756, 1, 0, 0, 0, 764, 763, 1, 0, 0, 0, 765, 120,
 | 
			
		||||
		1, 0, 0, 0, 766, 770, 5, 48, 0, 0, 767, 769, 3, 127, 63, 0, 768, 767, 1,
 | 
			
		||||
		0, 0, 0, 769, 772, 1, 0, 0, 0, 770, 768, 1, 0, 0, 0, 770, 771, 1, 0, 0,
 | 
			
		||||
		0, 771, 122, 1, 0, 0, 0, 772, 770, 1, 0, 0, 0, 773, 774, 5, 48, 0, 0, 774,
 | 
			
		||||
		775, 7, 7, 0, 0, 775, 776, 3, 147, 73, 0, 776, 124, 1, 0, 0, 0, 777, 778,
 | 
			
		||||
		7, 8, 0, 0, 778, 126, 1, 0, 0, 0, 779, 780, 7, 9, 0, 0, 780, 128, 1, 0,
 | 
			
		||||
		0, 0, 781, 782, 7, 10, 0, 0, 782, 130, 1, 0, 0, 0, 783, 784, 3, 129, 64,
 | 
			
		||||
		0, 784, 785, 3, 129, 64, 0, 785, 786, 3, 129, 64, 0, 786, 787, 3, 129,
 | 
			
		||||
		64, 0, 787, 132, 1, 0, 0, 0, 788, 789, 5, 92, 0, 0, 789, 790, 5, 117, 0,
 | 
			
		||||
		0, 790, 791, 1, 0, 0, 0, 791, 799, 3, 131, 65, 0, 792, 793, 5, 92, 0, 0,
 | 
			
		||||
		793, 794, 5, 85, 0, 0, 794, 795, 1, 0, 0, 0, 795, 796, 3, 131, 65, 0, 796,
 | 
			
		||||
		797, 3, 131, 65, 0, 797, 799, 1, 0, 0, 0, 798, 788, 1, 0, 0, 0, 798, 792,
 | 
			
		||||
		1, 0, 0, 0, 799, 134, 1, 0, 0, 0, 800, 802, 3, 139, 69, 0, 801, 803, 3,
 | 
			
		||||
		141, 70, 0, 802, 801, 1, 0, 0, 0, 802, 803, 1, 0, 0, 0, 803, 808, 1, 0,
 | 
			
		||||
		0, 0, 804, 805, 3, 143, 71, 0, 805, 806, 3, 141, 70, 0, 806, 808, 1, 0,
 | 
			
		||||
		0, 0, 807, 800, 1, 0, 0, 0, 807, 804, 1, 0, 0, 0, 808, 136, 1, 0, 0, 0,
 | 
			
		||||
		809, 810, 5, 48, 0, 0, 810, 813, 7, 7, 0, 0, 811, 814, 3, 145, 72, 0, 812,
 | 
			
		||||
		814, 3, 147, 73, 0, 813, 811, 1, 0, 0, 0, 813, 812, 1, 0, 0, 0, 814, 815,
 | 
			
		||||
		1, 0, 0, 0, 815, 816, 3, 149, 74, 0, 816, 138, 1, 0, 0, 0, 817, 819, 3,
 | 
			
		||||
		143, 71, 0, 818, 817, 1, 0, 0, 0, 818, 819, 1, 0, 0, 0, 819, 820, 1, 0,
 | 
			
		||||
		0, 0, 820, 821, 5, 46, 0, 0, 821, 826, 3, 143, 71, 0, 822, 823, 3, 143,
 | 
			
		||||
		71, 0, 823, 824, 5, 46, 0, 0, 824, 826, 1, 0, 0, 0, 825, 818, 1, 0, 0,
 | 
			
		||||
		0, 825, 822, 1, 0, 0, 0, 826, 140, 1, 0, 0, 0, 827, 829, 7, 11, 0, 0, 828,
 | 
			
		||||
		830, 7, 12, 0, 0, 829, 828, 1, 0, 0, 0, 829, 830, 1, 0, 0, 0, 830, 831,
 | 
			
		||||
		1, 0, 0, 0, 831, 832, 3, 143, 71, 0, 832, 142, 1, 0, 0, 0, 833, 835, 3,
 | 
			
		||||
		115, 57, 0, 834, 833, 1, 0, 0, 0, 835, 836, 1, 0, 0, 0, 836, 834, 1, 0,
 | 
			
		||||
		0, 0, 836, 837, 1, 0, 0, 0, 837, 144, 1, 0, 0, 0, 838, 840, 3, 147, 73,
 | 
			
		||||
		0, 839, 838, 1, 0, 0, 0, 839, 840, 1, 0, 0, 0, 840, 841, 1, 0, 0, 0, 841,
 | 
			
		||||
		842, 5, 46, 0, 0, 842, 847, 3, 147, 73, 0, 843, 844, 3, 147, 73, 0, 844,
 | 
			
		||||
		845, 5, 46, 0, 0, 845, 847, 1, 0, 0, 0, 846, 839, 1, 0, 0, 0, 846, 843,
 | 
			
		||||
		1, 0, 0, 0, 847, 146, 1, 0, 0, 0, 848, 850, 3, 129, 64, 0, 849, 848, 1,
 | 
			
		||||
		0, 0, 0, 850, 851, 1, 0, 0, 0, 851, 849, 1, 0, 0, 0, 851, 852, 1, 0, 0,
 | 
			
		||||
		0, 852, 148, 1, 0, 0, 0, 853, 855, 7, 13, 0, 0, 854, 856, 7, 12, 0, 0,
 | 
			
		||||
		855, 854, 1, 0, 0, 0, 855, 856, 1, 0, 0, 0, 856, 857, 1, 0, 0, 0, 857,
 | 
			
		||||
		858, 3, 143, 71, 0, 858, 150, 1, 0, 0, 0, 859, 860, 5, 92, 0, 0, 860, 875,
 | 
			
		||||
		7, 14, 0, 0, 861, 862, 5, 92, 0, 0, 862, 864, 3, 127, 63, 0, 863, 865,
 | 
			
		||||
		3, 127, 63, 0, 864, 863, 1, 0, 0, 0, 864, 865, 1, 0, 0, 0, 865, 867, 1,
 | 
			
		||||
		0, 0, 0, 866, 868, 3, 127, 63, 0, 867, 866, 1, 0, 0, 0, 867, 868, 1, 0,
 | 
			
		||||
		0, 0, 868, 875, 1, 0, 0, 0, 869, 870, 5, 92, 0, 0, 870, 871, 5, 120, 0,
 | 
			
		||||
		0, 871, 872, 1, 0, 0, 0, 872, 875, 3, 147, 73, 0, 873, 875, 3, 133, 66,
 | 
			
		||||
		0, 874, 859, 1, 0, 0, 0, 874, 861, 1, 0, 0, 0, 874, 869, 1, 0, 0, 0, 874,
 | 
			
		||||
		873, 1, 0, 0, 0, 875, 152, 1, 0, 0, 0, 876, 878, 7, 15, 0, 0, 877, 876,
 | 
			
		||||
		1, 0, 0, 0, 878, 879, 1, 0, 0, 0, 879, 877, 1, 0, 0, 0, 879, 880, 1, 0,
 | 
			
		||||
		0, 0, 880, 881, 1, 0, 0, 0, 881, 882, 6, 76, 0, 0, 882, 154, 1, 0, 0, 0,
 | 
			
		||||
		883, 885, 5, 13, 0, 0, 884, 886, 5, 10, 0, 0, 885, 884, 1, 0, 0, 0, 885,
 | 
			
		||||
		886, 1, 0, 0, 0, 886, 889, 1, 0, 0, 0, 887, 889, 5, 10, 0, 0, 888, 883,
 | 
			
		||||
		1, 0, 0, 0, 888, 887, 1, 0, 0, 0, 889, 890, 1, 0, 0, 0, 890, 891, 6, 77,
 | 
			
		||||
		0, 0, 891, 156, 1, 0, 0, 0, 60, 0, 195, 209, 231, 257, 285, 320, 328, 344,
 | 
			
		||||
		368, 379, 385, 390, 392, 423, 459, 495, 525, 563, 601, 627, 656, 662, 666,
 | 
			
		||||
		671, 673, 683, 687, 692, 695, 699, 704, 710, 715, 720, 725, 734, 743, 754,
 | 
			
		||||
		760, 764, 770, 798, 802, 807, 813, 818, 825, 829, 836, 839, 846, 851, 855,
 | 
			
		||||
		864, 867, 874, 879, 885, 888, 1, 6, 0, 0,
 | 
			
		||||
		0, 141, 0, 143, 0, 145, 0, 147, 0, 149, 0, 151, 0, 153, 0, 155, 0, 157,
 | 
			
		||||
		54, 159, 55, 1, 0, 16, 3, 0, 76, 76, 85, 85, 117, 117, 4, 0, 10, 10, 13,
 | 
			
		||||
		13, 34, 34, 92, 92, 4, 0, 10, 10, 13, 13, 39, 39, 92, 92, 3, 0, 65, 90,
 | 
			
		||||
		95, 95, 97, 122, 1, 0, 48, 57, 2, 0, 66, 66, 98, 98, 1, 0, 48, 49, 2, 0,
 | 
			
		||||
		88, 88, 120, 120, 1, 0, 49, 57, 1, 0, 48, 55, 3, 0, 48, 57, 65, 70, 97,
 | 
			
		||||
		102, 2, 0, 69, 69, 101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 80, 80, 112, 112,
 | 
			
		||||
		10, 0, 34, 34, 39, 39, 63, 63, 92, 92, 97, 98, 102, 102, 110, 110, 114,
 | 
			
		||||
		114, 116, 116, 118, 118, 2, 0, 9, 9, 32, 32, 972, 0, 1, 1, 0, 0, 0, 0,
 | 
			
		||||
		3, 1, 0, 0, 0, 0, 5, 1, 0, 0, 0, 0, 7, 1, 0, 0, 0, 0, 9, 1, 0, 0, 0, 0,
 | 
			
		||||
		11, 1, 0, 0, 0, 0, 13, 1, 0, 0, 0, 0, 15, 1, 0, 0, 0, 0, 17, 1, 0, 0, 0,
 | 
			
		||||
		0, 19, 1, 0, 0, 0, 0, 21, 1, 0, 0, 0, 0, 23, 1, 0, 0, 0, 0, 25, 1, 0, 0,
 | 
			
		||||
		0, 0, 27, 1, 0, 0, 0, 0, 29, 1, 0, 0, 0, 0, 31, 1, 0, 0, 0, 0, 33, 1, 0,
 | 
			
		||||
		0, 0, 0, 35, 1, 0, 0, 0, 0, 37, 1, 0, 0, 0, 0, 39, 1, 0, 0, 0, 0, 41, 1,
 | 
			
		||||
		0, 0, 0, 0, 43, 1, 0, 0, 0, 0, 45, 1, 0, 0, 0, 0, 47, 1, 0, 0, 0, 0, 49,
 | 
			
		||||
		1, 0, 0, 0, 0, 51, 1, 0, 0, 0, 0, 53, 1, 0, 0, 0, 0, 55, 1, 0, 0, 0, 0,
 | 
			
		||||
		57, 1, 0, 0, 0, 0, 59, 1, 0, 0, 0, 0, 61, 1, 0, 0, 0, 0, 63, 1, 0, 0, 0,
 | 
			
		||||
		0, 65, 1, 0, 0, 0, 0, 67, 1, 0, 0, 0, 0, 69, 1, 0, 0, 0, 0, 71, 1, 0, 0,
 | 
			
		||||
		0, 0, 73, 1, 0, 0, 0, 0, 75, 1, 0, 0, 0, 0, 77, 1, 0, 0, 0, 0, 79, 1, 0,
 | 
			
		||||
		0, 0, 0, 81, 1, 0, 0, 0, 0, 83, 1, 0, 0, 0, 0, 85, 1, 0, 0, 0, 0, 87, 1,
 | 
			
		||||
		0, 0, 0, 0, 89, 1, 0, 0, 0, 0, 91, 1, 0, 0, 0, 0, 93, 1, 0, 0, 0, 0, 95,
 | 
			
		||||
		1, 0, 0, 0, 0, 97, 1, 0, 0, 0, 0, 99, 1, 0, 0, 0, 0, 101, 1, 0, 0, 0, 0,
 | 
			
		||||
		103, 1, 0, 0, 0, 0, 105, 1, 0, 0, 0, 0, 157, 1, 0, 0, 0, 0, 159, 1, 0,
 | 
			
		||||
		0, 0, 1, 161, 1, 0, 0, 0, 3, 163, 1, 0, 0, 0, 5, 165, 1, 0, 0, 0, 7, 167,
 | 
			
		||||
		1, 0, 0, 0, 9, 169, 1, 0, 0, 0, 11, 171, 1, 0, 0, 0, 13, 173, 1, 0, 0,
 | 
			
		||||
		0, 15, 175, 1, 0, 0, 0, 17, 177, 1, 0, 0, 0, 19, 180, 1, 0, 0, 0, 21, 182,
 | 
			
		||||
		1, 0, 0, 0, 23, 185, 1, 0, 0, 0, 25, 188, 1, 0, 0, 0, 27, 199, 1, 0, 0,
 | 
			
		||||
		0, 29, 213, 1, 0, 0, 0, 31, 235, 1, 0, 0, 0, 33, 261, 1, 0, 0, 0, 35, 289,
 | 
			
		||||
		1, 0, 0, 0, 37, 307, 1, 0, 0, 0, 39, 315, 1, 0, 0, 0, 41, 317, 1, 0, 0,
 | 
			
		||||
		0, 43, 319, 1, 0, 0, 0, 45, 321, 1, 0, 0, 0, 47, 323, 1, 0, 0, 0, 49, 325,
 | 
			
		||||
		1, 0, 0, 0, 51, 327, 1, 0, 0, 0, 53, 330, 1, 0, 0, 0, 55, 333, 1, 0, 0,
 | 
			
		||||
		0, 57, 336, 1, 0, 0, 0, 59, 338, 1, 0, 0, 0, 61, 340, 1, 0, 0, 0, 63, 350,
 | 
			
		||||
		1, 0, 0, 0, 65, 358, 1, 0, 0, 0, 67, 374, 1, 0, 0, 0, 69, 398, 1, 0, 0,
 | 
			
		||||
		0, 71, 400, 1, 0, 0, 0, 73, 409, 1, 0, 0, 0, 75, 415, 1, 0, 0, 0, 77, 417,
 | 
			
		||||
		1, 0, 0, 0, 79, 453, 1, 0, 0, 0, 81, 489, 1, 0, 0, 0, 83, 525, 1, 0, 0,
 | 
			
		||||
		0, 85, 555, 1, 0, 0, 0, 87, 593, 1, 0, 0, 0, 89, 631, 1, 0, 0, 0, 91, 657,
 | 
			
		||||
		1, 0, 0, 0, 93, 686, 1, 0, 0, 0, 95, 692, 1, 0, 0, 0, 97, 696, 1, 0, 0,
 | 
			
		||||
		0, 99, 698, 1, 0, 0, 0, 101, 706, 1, 0, 0, 0, 103, 713, 1, 0, 0, 0, 105,
 | 
			
		||||
		729, 1, 0, 0, 0, 107, 745, 1, 0, 0, 0, 109, 748, 1, 0, 0, 0, 111, 753,
 | 
			
		||||
		1, 0, 0, 0, 113, 764, 1, 0, 0, 0, 115, 773, 1, 0, 0, 0, 117, 775, 1, 0,
 | 
			
		||||
		0, 0, 119, 777, 1, 0, 0, 0, 121, 779, 1, 0, 0, 0, 123, 794, 1, 0, 0, 0,
 | 
			
		||||
		125, 796, 1, 0, 0, 0, 127, 803, 1, 0, 0, 0, 129, 807, 1, 0, 0, 0, 131,
 | 
			
		||||
		809, 1, 0, 0, 0, 133, 811, 1, 0, 0, 0, 135, 813, 1, 0, 0, 0, 137, 828,
 | 
			
		||||
		1, 0, 0, 0, 139, 837, 1, 0, 0, 0, 141, 839, 1, 0, 0, 0, 143, 855, 1, 0,
 | 
			
		||||
		0, 0, 145, 857, 1, 0, 0, 0, 147, 864, 1, 0, 0, 0, 149, 876, 1, 0, 0, 0,
 | 
			
		||||
		151, 879, 1, 0, 0, 0, 153, 883, 1, 0, 0, 0, 155, 904, 1, 0, 0, 0, 157,
 | 
			
		||||
		907, 1, 0, 0, 0, 159, 918, 1, 0, 0, 0, 161, 162, 5, 40, 0, 0, 162, 2, 1,
 | 
			
		||||
		0, 0, 0, 163, 164, 5, 41, 0, 0, 164, 4, 1, 0, 0, 0, 165, 166, 5, 91, 0,
 | 
			
		||||
		0, 166, 6, 1, 0, 0, 0, 167, 168, 5, 44, 0, 0, 168, 8, 1, 0, 0, 0, 169,
 | 
			
		||||
		170, 5, 93, 0, 0, 170, 10, 1, 0, 0, 0, 171, 172, 5, 123, 0, 0, 172, 12,
 | 
			
		||||
		1, 0, 0, 0, 173, 174, 5, 125, 0, 0, 174, 14, 1, 0, 0, 0, 175, 176, 5, 60,
 | 
			
		||||
		0, 0, 176, 16, 1, 0, 0, 0, 177, 178, 5, 60, 0, 0, 178, 179, 5, 61, 0, 0,
 | 
			
		||||
		179, 18, 1, 0, 0, 0, 180, 181, 5, 62, 0, 0, 181, 20, 1, 0, 0, 0, 182, 183,
 | 
			
		||||
		5, 62, 0, 0, 183, 184, 5, 61, 0, 0, 184, 22, 1, 0, 0, 0, 185, 186, 5, 61,
 | 
			
		||||
		0, 0, 186, 187, 5, 61, 0, 0, 187, 24, 1, 0, 0, 0, 188, 189, 5, 33, 0, 0,
 | 
			
		||||
		189, 190, 5, 61, 0, 0, 190, 26, 1, 0, 0, 0, 191, 192, 5, 108, 0, 0, 192,
 | 
			
		||||
		193, 5, 105, 0, 0, 193, 194, 5, 107, 0, 0, 194, 200, 5, 101, 0, 0, 195,
 | 
			
		||||
		196, 5, 76, 0, 0, 196, 197, 5, 73, 0, 0, 197, 198, 5, 75, 0, 0, 198, 200,
 | 
			
		||||
		5, 69, 0, 0, 199, 191, 1, 0, 0, 0, 199, 195, 1, 0, 0, 0, 200, 28, 1, 0,
 | 
			
		||||
		0, 0, 201, 202, 5, 101, 0, 0, 202, 203, 5, 120, 0, 0, 203, 204, 5, 105,
 | 
			
		||||
		0, 0, 204, 205, 5, 115, 0, 0, 205, 206, 5, 116, 0, 0, 206, 214, 5, 115,
 | 
			
		||||
		0, 0, 207, 208, 5, 69, 0, 0, 208, 209, 5, 88, 0, 0, 209, 210, 5, 73, 0,
 | 
			
		||||
		0, 210, 211, 5, 83, 0, 0, 211, 212, 5, 84, 0, 0, 212, 214, 5, 83, 0, 0,
 | 
			
		||||
		213, 201, 1, 0, 0, 0, 213, 207, 1, 0, 0, 0, 214, 30, 1, 0, 0, 0, 215, 216,
 | 
			
		||||
		5, 116, 0, 0, 216, 217, 5, 101, 0, 0, 217, 218, 5, 120, 0, 0, 218, 219,
 | 
			
		||||
		5, 116, 0, 0, 219, 220, 5, 95, 0, 0, 220, 221, 5, 109, 0, 0, 221, 222,
 | 
			
		||||
		5, 97, 0, 0, 222, 223, 5, 116, 0, 0, 223, 224, 5, 99, 0, 0, 224, 236, 5,
 | 
			
		||||
		104, 0, 0, 225, 226, 5, 84, 0, 0, 226, 227, 5, 69, 0, 0, 227, 228, 5, 88,
 | 
			
		||||
		0, 0, 228, 229, 5, 84, 0, 0, 229, 230, 5, 95, 0, 0, 230, 231, 5, 77, 0,
 | 
			
		||||
		0, 231, 232, 5, 65, 0, 0, 232, 233, 5, 84, 0, 0, 233, 234, 5, 67, 0, 0,
 | 
			
		||||
		234, 236, 5, 72, 0, 0, 235, 215, 1, 0, 0, 0, 235, 225, 1, 0, 0, 0, 236,
 | 
			
		||||
		32, 1, 0, 0, 0, 237, 238, 5, 112, 0, 0, 238, 239, 5, 104, 0, 0, 239, 240,
 | 
			
		||||
		5, 114, 0, 0, 240, 241, 5, 97, 0, 0, 241, 242, 5, 115, 0, 0, 242, 243,
 | 
			
		||||
		5, 101, 0, 0, 243, 244, 5, 95, 0, 0, 244, 245, 5, 109, 0, 0, 245, 246,
 | 
			
		||||
		5, 97, 0, 0, 246, 247, 5, 116, 0, 0, 247, 248, 5, 99, 0, 0, 248, 262, 5,
 | 
			
		||||
		104, 0, 0, 249, 250, 5, 80, 0, 0, 250, 251, 5, 72, 0, 0, 251, 252, 5, 82,
 | 
			
		||||
		0, 0, 252, 253, 5, 65, 0, 0, 253, 254, 5, 83, 0, 0, 254, 255, 5, 69, 0,
 | 
			
		||||
		0, 255, 256, 5, 95, 0, 0, 256, 257, 5, 77, 0, 0, 257, 258, 5, 65, 0, 0,
 | 
			
		||||
		258, 259, 5, 84, 0, 0, 259, 260, 5, 67, 0, 0, 260, 262, 5, 72, 0, 0, 261,
 | 
			
		||||
		237, 1, 0, 0, 0, 261, 249, 1, 0, 0, 0, 262, 34, 1, 0, 0, 0, 263, 264, 5,
 | 
			
		||||
		114, 0, 0, 264, 265, 5, 97, 0, 0, 265, 266, 5, 110, 0, 0, 266, 267, 5,
 | 
			
		||||
		100, 0, 0, 267, 268, 5, 111, 0, 0, 268, 269, 5, 109, 0, 0, 269, 270, 5,
 | 
			
		||||
		95, 0, 0, 270, 271, 5, 115, 0, 0, 271, 272, 5, 97, 0, 0, 272, 273, 5, 109,
 | 
			
		||||
		0, 0, 273, 274, 5, 112, 0, 0, 274, 275, 5, 108, 0, 0, 275, 290, 5, 101,
 | 
			
		||||
		0, 0, 276, 277, 5, 82, 0, 0, 277, 278, 5, 65, 0, 0, 278, 279, 5, 78, 0,
 | 
			
		||||
		0, 279, 280, 5, 68, 0, 0, 280, 281, 5, 79, 0, 0, 281, 282, 5, 77, 0, 0,
 | 
			
		||||
		282, 283, 5, 95, 0, 0, 283, 284, 5, 83, 0, 0, 284, 285, 5, 65, 0, 0, 285,
 | 
			
		||||
		286, 5, 77, 0, 0, 286, 287, 5, 80, 0, 0, 287, 288, 5, 76, 0, 0, 288, 290,
 | 
			
		||||
		5, 69, 0, 0, 289, 263, 1, 0, 0, 0, 289, 276, 1, 0, 0, 0, 290, 36, 1, 0,
 | 
			
		||||
		0, 0, 291, 292, 5, 105, 0, 0, 292, 293, 5, 110, 0, 0, 293, 294, 5, 116,
 | 
			
		||||
		0, 0, 294, 295, 5, 101, 0, 0, 295, 296, 5, 114, 0, 0, 296, 297, 5, 118,
 | 
			
		||||
		0, 0, 297, 298, 5, 97, 0, 0, 298, 308, 5, 108, 0, 0, 299, 300, 5, 73, 0,
 | 
			
		||||
		0, 300, 301, 5, 78, 0, 0, 301, 302, 5, 84, 0, 0, 302, 303, 5, 69, 0, 0,
 | 
			
		||||
		303, 304, 5, 82, 0, 0, 304, 305, 5, 86, 0, 0, 305, 306, 5, 65, 0, 0, 306,
 | 
			
		||||
		308, 5, 76, 0, 0, 307, 291, 1, 0, 0, 0, 307, 299, 1, 0, 0, 0, 308, 38,
 | 
			
		||||
		1, 0, 0, 0, 309, 310, 5, 105, 0, 0, 310, 311, 5, 115, 0, 0, 311, 316, 5,
 | 
			
		||||
		111, 0, 0, 312, 313, 5, 73, 0, 0, 313, 314, 5, 83, 0, 0, 314, 316, 5, 79,
 | 
			
		||||
		0, 0, 315, 309, 1, 0, 0, 0, 315, 312, 1, 0, 0, 0, 316, 40, 1, 0, 0, 0,
 | 
			
		||||
		317, 318, 5, 43, 0, 0, 318, 42, 1, 0, 0, 0, 319, 320, 5, 45, 0, 0, 320,
 | 
			
		||||
		44, 1, 0, 0, 0, 321, 322, 5, 42, 0, 0, 322, 46, 1, 0, 0, 0, 323, 324, 5,
 | 
			
		||||
		47, 0, 0, 324, 48, 1, 0, 0, 0, 325, 326, 5, 37, 0, 0, 326, 50, 1, 0, 0,
 | 
			
		||||
		0, 327, 328, 5, 42, 0, 0, 328, 329, 5, 42, 0, 0, 329, 52, 1, 0, 0, 0, 330,
 | 
			
		||||
		331, 5, 60, 0, 0, 331, 332, 5, 60, 0, 0, 332, 54, 1, 0, 0, 0, 333, 334,
 | 
			
		||||
		5, 62, 0, 0, 334, 335, 5, 62, 0, 0, 335, 56, 1, 0, 0, 0, 336, 337, 5, 38,
 | 
			
		||||
		0, 0, 337, 58, 1, 0, 0, 0, 338, 339, 5, 124, 0, 0, 339, 60, 1, 0, 0, 0,
 | 
			
		||||
		340, 341, 5, 94, 0, 0, 341, 62, 1, 0, 0, 0, 342, 343, 5, 38, 0, 0, 343,
 | 
			
		||||
		351, 5, 38, 0, 0, 344, 345, 5, 97, 0, 0, 345, 346, 5, 110, 0, 0, 346, 351,
 | 
			
		||||
		5, 100, 0, 0, 347, 348, 5, 65, 0, 0, 348, 349, 5, 78, 0, 0, 349, 351, 5,
 | 
			
		||||
		68, 0, 0, 350, 342, 1, 0, 0, 0, 350, 344, 1, 0, 0, 0, 350, 347, 1, 0, 0,
 | 
			
		||||
		0, 351, 64, 1, 0, 0, 0, 352, 353, 5, 124, 0, 0, 353, 359, 5, 124, 0, 0,
 | 
			
		||||
		354, 355, 5, 111, 0, 0, 355, 359, 5, 114, 0, 0, 356, 357, 5, 79, 0, 0,
 | 
			
		||||
		357, 359, 5, 82, 0, 0, 358, 352, 1, 0, 0, 0, 358, 354, 1, 0, 0, 0, 358,
 | 
			
		||||
		356, 1, 0, 0, 0, 359, 66, 1, 0, 0, 0, 360, 361, 5, 105, 0, 0, 361, 362,
 | 
			
		||||
		5, 115, 0, 0, 362, 363, 5, 32, 0, 0, 363, 364, 5, 110, 0, 0, 364, 365,
 | 
			
		||||
		5, 117, 0, 0, 365, 366, 5, 108, 0, 0, 366, 375, 5, 108, 0, 0, 367, 368,
 | 
			
		||||
		5, 73, 0, 0, 368, 369, 5, 83, 0, 0, 369, 370, 5, 32, 0, 0, 370, 371, 5,
 | 
			
		||||
		78, 0, 0, 371, 372, 5, 85, 0, 0, 372, 373, 5, 76, 0, 0, 373, 375, 5, 76,
 | 
			
		||||
		0, 0, 374, 360, 1, 0, 0, 0, 374, 367, 1, 0, 0, 0, 375, 68, 1, 0, 0, 0,
 | 
			
		||||
		376, 377, 5, 105, 0, 0, 377, 378, 5, 115, 0, 0, 378, 379, 5, 32, 0, 0,
 | 
			
		||||
		379, 380, 5, 110, 0, 0, 380, 381, 5, 111, 0, 0, 381, 382, 5, 116, 0, 0,
 | 
			
		||||
		382, 383, 5, 32, 0, 0, 383, 384, 5, 110, 0, 0, 384, 385, 5, 117, 0, 0,
 | 
			
		||||
		385, 386, 5, 108, 0, 0, 386, 399, 5, 108, 0, 0, 387, 388, 5, 73, 0, 0,
 | 
			
		||||
		388, 389, 5, 83, 0, 0, 389, 390, 5, 32, 0, 0, 390, 391, 5, 78, 0, 0, 391,
 | 
			
		||||
		392, 5, 79, 0, 0, 392, 393, 5, 84, 0, 0, 393, 394, 5, 32, 0, 0, 394, 395,
 | 
			
		||||
		5, 78, 0, 0, 395, 396, 5, 85, 0, 0, 396, 397, 5, 76, 0, 0, 397, 399, 5,
 | 
			
		||||
		76, 0, 0, 398, 376, 1, 0, 0, 0, 398, 387, 1, 0, 0, 0, 399, 70, 1, 0, 0,
 | 
			
		||||
		0, 400, 401, 5, 126, 0, 0, 401, 72, 1, 0, 0, 0, 402, 410, 5, 33, 0, 0,
 | 
			
		||||
		403, 404, 5, 110, 0, 0, 404, 405, 5, 111, 0, 0, 405, 410, 5, 116, 0, 0,
 | 
			
		||||
		406, 407, 5, 78, 0, 0, 407, 408, 5, 79, 0, 0, 408, 410, 5, 84, 0, 0, 409,
 | 
			
		||||
		402, 1, 0, 0, 0, 409, 403, 1, 0, 0, 0, 409, 406, 1, 0, 0, 0, 410, 74, 1,
 | 
			
		||||
		0, 0, 0, 411, 412, 5, 105, 0, 0, 412, 416, 5, 110, 0, 0, 413, 414, 5, 73,
 | 
			
		||||
		0, 0, 414, 416, 5, 78, 0, 0, 415, 411, 1, 0, 0, 0, 415, 413, 1, 0, 0, 0,
 | 
			
		||||
		416, 76, 1, 0, 0, 0, 417, 422, 5, 91, 0, 0, 418, 421, 3, 157, 78, 0, 419,
 | 
			
		||||
		421, 3, 159, 79, 0, 420, 418, 1, 0, 0, 0, 420, 419, 1, 0, 0, 0, 421, 424,
 | 
			
		||||
		1, 0, 0, 0, 422, 420, 1, 0, 0, 0, 422, 423, 1, 0, 0, 0, 423, 425, 1, 0,
 | 
			
		||||
		0, 0, 424, 422, 1, 0, 0, 0, 425, 426, 5, 93, 0, 0, 426, 78, 1, 0, 0, 0,
 | 
			
		||||
		427, 428, 5, 106, 0, 0, 428, 429, 5, 115, 0, 0, 429, 430, 5, 111, 0, 0,
 | 
			
		||||
		430, 431, 5, 110, 0, 0, 431, 432, 5, 95, 0, 0, 432, 433, 5, 99, 0, 0, 433,
 | 
			
		||||
		434, 5, 111, 0, 0, 434, 435, 5, 110, 0, 0, 435, 436, 5, 116, 0, 0, 436,
 | 
			
		||||
		437, 5, 97, 0, 0, 437, 438, 5, 105, 0, 0, 438, 439, 5, 110, 0, 0, 439,
 | 
			
		||||
		454, 5, 115, 0, 0, 440, 441, 5, 74, 0, 0, 441, 442, 5, 83, 0, 0, 442, 443,
 | 
			
		||||
		5, 79, 0, 0, 443, 444, 5, 78, 0, 0, 444, 445, 5, 95, 0, 0, 445, 446, 5,
 | 
			
		||||
		67, 0, 0, 446, 447, 5, 79, 0, 0, 447, 448, 5, 78, 0, 0, 448, 449, 5, 84,
 | 
			
		||||
		0, 0, 449, 450, 5, 65, 0, 0, 450, 451, 5, 73, 0, 0, 451, 452, 5, 78, 0,
 | 
			
		||||
		0, 452, 454, 5, 83, 0, 0, 453, 427, 1, 0, 0, 0, 453, 440, 1, 0, 0, 0, 454,
 | 
			
		||||
		80, 1, 0, 0, 0, 455, 456, 5, 106, 0, 0, 456, 457, 5, 115, 0, 0, 457, 458,
 | 
			
		||||
		5, 111, 0, 0, 458, 459, 5, 110, 0, 0, 459, 460, 5, 95, 0, 0, 460, 461,
 | 
			
		||||
		5, 99, 0, 0, 461, 462, 5, 111, 0, 0, 462, 463, 5, 110, 0, 0, 463, 464,
 | 
			
		||||
		5, 116, 0, 0, 464, 465, 5, 97, 0, 0, 465, 466, 5, 105, 0, 0, 466, 467,
 | 
			
		||||
		5, 110, 0, 0, 467, 468, 5, 115, 0, 0, 468, 469, 5, 95, 0, 0, 469, 470,
 | 
			
		||||
		5, 97, 0, 0, 470, 471, 5, 108, 0, 0, 471, 490, 5, 108, 0, 0, 472, 473,
 | 
			
		||||
		5, 74, 0, 0, 473, 474, 5, 83, 0, 0, 474, 475, 5, 79, 0, 0, 475, 476, 5,
 | 
			
		||||
		78, 0, 0, 476, 477, 5, 95, 0, 0, 477, 478, 5, 67, 0, 0, 478, 479, 5, 79,
 | 
			
		||||
		0, 0, 479, 480, 5, 78, 0, 0, 480, 481, 5, 84, 0, 0, 481, 482, 5, 65, 0,
 | 
			
		||||
		0, 482, 483, 5, 73, 0, 0, 483, 484, 5, 78, 0, 0, 484, 485, 5, 83, 0, 0,
 | 
			
		||||
		485, 486, 5, 95, 0, 0, 486, 487, 5, 65, 0, 0, 487, 488, 5, 76, 0, 0, 488,
 | 
			
		||||
		490, 5, 76, 0, 0, 489, 455, 1, 0, 0, 0, 489, 472, 1, 0, 0, 0, 490, 82,
 | 
			
		||||
		1, 0, 0, 0, 491, 492, 5, 106, 0, 0, 492, 493, 5, 115, 0, 0, 493, 494, 5,
 | 
			
		||||
		111, 0, 0, 494, 495, 5, 110, 0, 0, 495, 496, 5, 95, 0, 0, 496, 497, 5,
 | 
			
		||||
		99, 0, 0, 497, 498, 5, 111, 0, 0, 498, 499, 5, 110, 0, 0, 499, 500, 5,
 | 
			
		||||
		116, 0, 0, 500, 501, 5, 97, 0, 0, 501, 502, 5, 105, 0, 0, 502, 503, 5,
 | 
			
		||||
		110, 0, 0, 503, 504, 5, 115, 0, 0, 504, 505, 5, 95, 0, 0, 505, 506, 5,
 | 
			
		||||
		97, 0, 0, 506, 507, 5, 110, 0, 0, 507, 526, 5, 121, 0, 0, 508, 509, 5,
 | 
			
		||||
		74, 0, 0, 509, 510, 5, 83, 0, 0, 510, 511, 5, 79, 0, 0, 511, 512, 5, 78,
 | 
			
		||||
		0, 0, 512, 513, 5, 95, 0, 0, 513, 514, 5, 67, 0, 0, 514, 515, 5, 79, 0,
 | 
			
		||||
		0, 515, 516, 5, 78, 0, 0, 516, 517, 5, 84, 0, 0, 517, 518, 5, 65, 0, 0,
 | 
			
		||||
		518, 519, 5, 73, 0, 0, 519, 520, 5, 78, 0, 0, 520, 521, 5, 83, 0, 0, 521,
 | 
			
		||||
		522, 5, 95, 0, 0, 522, 523, 5, 65, 0, 0, 523, 524, 5, 78, 0, 0, 524, 526,
 | 
			
		||||
		5, 89, 0, 0, 525, 491, 1, 0, 0, 0, 525, 508, 1, 0, 0, 0, 526, 84, 1, 0,
 | 
			
		||||
		0, 0, 527, 528, 5, 97, 0, 0, 528, 529, 5, 114, 0, 0, 529, 530, 5, 114,
 | 
			
		||||
		0, 0, 530, 531, 5, 97, 0, 0, 531, 532, 5, 121, 0, 0, 532, 533, 5, 95, 0,
 | 
			
		||||
		0, 533, 534, 5, 99, 0, 0, 534, 535, 5, 111, 0, 0, 535, 536, 5, 110, 0,
 | 
			
		||||
		0, 536, 537, 5, 116, 0, 0, 537, 538, 5, 97, 0, 0, 538, 539, 5, 105, 0,
 | 
			
		||||
		0, 539, 540, 5, 110, 0, 0, 540, 556, 5, 115, 0, 0, 541, 542, 5, 65, 0,
 | 
			
		||||
		0, 542, 543, 5, 82, 0, 0, 543, 544, 5, 82, 0, 0, 544, 545, 5, 65, 0, 0,
 | 
			
		||||
		545, 546, 5, 89, 0, 0, 546, 547, 5, 95, 0, 0, 547, 548, 5, 67, 0, 0, 548,
 | 
			
		||||
		549, 5, 79, 0, 0, 549, 550, 5, 78, 0, 0, 550, 551, 5, 84, 0, 0, 551, 552,
 | 
			
		||||
		5, 65, 0, 0, 552, 553, 5, 73, 0, 0, 553, 554, 5, 78, 0, 0, 554, 556, 5,
 | 
			
		||||
		83, 0, 0, 555, 527, 1, 0, 0, 0, 555, 541, 1, 0, 0, 0, 556, 86, 1, 0, 0,
 | 
			
		||||
		0, 557, 558, 5, 97, 0, 0, 558, 559, 5, 114, 0, 0, 559, 560, 5, 114, 0,
 | 
			
		||||
		0, 560, 561, 5, 97, 0, 0, 561, 562, 5, 121, 0, 0, 562, 563, 5, 95, 0, 0,
 | 
			
		||||
		563, 564, 5, 99, 0, 0, 564, 565, 5, 111, 0, 0, 565, 566, 5, 110, 0, 0,
 | 
			
		||||
		566, 567, 5, 116, 0, 0, 567, 568, 5, 97, 0, 0, 568, 569, 5, 105, 0, 0,
 | 
			
		||||
		569, 570, 5, 110, 0, 0, 570, 571, 5, 115, 0, 0, 571, 572, 5, 95, 0, 0,
 | 
			
		||||
		572, 573, 5, 97, 0, 0, 573, 574, 5, 108, 0, 0, 574, 594, 5, 108, 0, 0,
 | 
			
		||||
		575, 576, 5, 65, 0, 0, 576, 577, 5, 82, 0, 0, 577, 578, 5, 82, 0, 0, 578,
 | 
			
		||||
		579, 5, 65, 0, 0, 579, 580, 5, 89, 0, 0, 580, 581, 5, 95, 0, 0, 581, 582,
 | 
			
		||||
		5, 67, 0, 0, 582, 583, 5, 79, 0, 0, 583, 584, 5, 78, 0, 0, 584, 585, 5,
 | 
			
		||||
		84, 0, 0, 585, 586, 5, 65, 0, 0, 586, 587, 5, 73, 0, 0, 587, 588, 5, 78,
 | 
			
		||||
		0, 0, 588, 589, 5, 83, 0, 0, 589, 590, 5, 95, 0, 0, 590, 591, 5, 65, 0,
 | 
			
		||||
		0, 591, 592, 5, 76, 0, 0, 592, 594, 5, 76, 0, 0, 593, 557, 1, 0, 0, 0,
 | 
			
		||||
		593, 575, 1, 0, 0, 0, 594, 88, 1, 0, 0, 0, 595, 596, 5, 97, 0, 0, 596,
 | 
			
		||||
		597, 5, 114, 0, 0, 597, 598, 5, 114, 0, 0, 598, 599, 5, 97, 0, 0, 599,
 | 
			
		||||
		600, 5, 121, 0, 0, 600, 601, 5, 95, 0, 0, 601, 602, 5, 99, 0, 0, 602, 603,
 | 
			
		||||
		5, 111, 0, 0, 603, 604, 5, 110, 0, 0, 604, 605, 5, 116, 0, 0, 605, 606,
 | 
			
		||||
		5, 97, 0, 0, 606, 607, 5, 105, 0, 0, 607, 608, 5, 110, 0, 0, 608, 609,
 | 
			
		||||
		5, 115, 0, 0, 609, 610, 5, 95, 0, 0, 610, 611, 5, 97, 0, 0, 611, 612, 5,
 | 
			
		||||
		110, 0, 0, 612, 632, 5, 121, 0, 0, 613, 614, 5, 65, 0, 0, 614, 615, 5,
 | 
			
		||||
		82, 0, 0, 615, 616, 5, 82, 0, 0, 616, 617, 5, 65, 0, 0, 617, 618, 5, 89,
 | 
			
		||||
		0, 0, 618, 619, 5, 95, 0, 0, 619, 620, 5, 67, 0, 0, 620, 621, 5, 79, 0,
 | 
			
		||||
		0, 621, 622, 5, 78, 0, 0, 622, 623, 5, 84, 0, 0, 623, 624, 5, 65, 0, 0,
 | 
			
		||||
		624, 625, 5, 73, 0, 0, 625, 626, 5, 78, 0, 0, 626, 627, 5, 83, 0, 0, 627,
 | 
			
		||||
		628, 5, 95, 0, 0, 628, 629, 5, 65, 0, 0, 629, 630, 5, 78, 0, 0, 630, 632,
 | 
			
		||||
		5, 89, 0, 0, 631, 595, 1, 0, 0, 0, 631, 613, 1, 0, 0, 0, 632, 90, 1, 0,
 | 
			
		||||
		0, 0, 633, 634, 5, 97, 0, 0, 634, 635, 5, 114, 0, 0, 635, 636, 5, 114,
 | 
			
		||||
		0, 0, 636, 637, 5, 97, 0, 0, 637, 638, 5, 121, 0, 0, 638, 639, 5, 95, 0,
 | 
			
		||||
		0, 639, 640, 5, 108, 0, 0, 640, 641, 5, 101, 0, 0, 641, 642, 5, 110, 0,
 | 
			
		||||
		0, 642, 643, 5, 103, 0, 0, 643, 644, 5, 116, 0, 0, 644, 658, 5, 104, 0,
 | 
			
		||||
		0, 645, 646, 5, 65, 0, 0, 646, 647, 5, 82, 0, 0, 647, 648, 5, 82, 0, 0,
 | 
			
		||||
		648, 649, 5, 65, 0, 0, 649, 650, 5, 89, 0, 0, 650, 651, 5, 95, 0, 0, 651,
 | 
			
		||||
		652, 5, 76, 0, 0, 652, 653, 5, 69, 0, 0, 653, 654, 5, 78, 0, 0, 654, 655,
 | 
			
		||||
		5, 71, 0, 0, 655, 656, 5, 84, 0, 0, 656, 658, 5, 72, 0, 0, 657, 633, 1,
 | 
			
		||||
		0, 0, 0, 657, 645, 1, 0, 0, 0, 658, 92, 1, 0, 0, 0, 659, 660, 5, 116, 0,
 | 
			
		||||
		0, 660, 661, 5, 114, 0, 0, 661, 662, 5, 117, 0, 0, 662, 687, 5, 101, 0,
 | 
			
		||||
		0, 663, 664, 5, 84, 0, 0, 664, 665, 5, 114, 0, 0, 665, 666, 5, 117, 0,
 | 
			
		||||
		0, 666, 687, 5, 101, 0, 0, 667, 668, 5, 84, 0, 0, 668, 669, 5, 82, 0, 0,
 | 
			
		||||
		669, 670, 5, 85, 0, 0, 670, 687, 5, 69, 0, 0, 671, 672, 5, 102, 0, 0, 672,
 | 
			
		||||
		673, 5, 97, 0, 0, 673, 674, 5, 108, 0, 0, 674, 675, 5, 115, 0, 0, 675,
 | 
			
		||||
		687, 5, 101, 0, 0, 676, 677, 5, 70, 0, 0, 677, 678, 5, 97, 0, 0, 678, 679,
 | 
			
		||||
		5, 108, 0, 0, 679, 680, 5, 115, 0, 0, 680, 687, 5, 101, 0, 0, 681, 682,
 | 
			
		||||
		5, 70, 0, 0, 682, 683, 5, 65, 0, 0, 683, 684, 5, 76, 0, 0, 684, 685, 5,
 | 
			
		||||
		83, 0, 0, 685, 687, 5, 69, 0, 0, 686, 659, 1, 0, 0, 0, 686, 663, 1, 0,
 | 
			
		||||
		0, 0, 686, 667, 1, 0, 0, 0, 686, 671, 1, 0, 0, 0, 686, 676, 1, 0, 0, 0,
 | 
			
		||||
		686, 681, 1, 0, 0, 0, 687, 94, 1, 0, 0, 0, 688, 693, 3, 123, 61, 0, 689,
 | 
			
		||||
		693, 3, 125, 62, 0, 690, 693, 3, 127, 63, 0, 691, 693, 3, 121, 60, 0, 692,
 | 
			
		||||
		688, 1, 0, 0, 0, 692, 689, 1, 0, 0, 0, 692, 690, 1, 0, 0, 0, 692, 691,
 | 
			
		||||
		1, 0, 0, 0, 693, 96, 1, 0, 0, 0, 694, 697, 3, 139, 69, 0, 695, 697, 3,
 | 
			
		||||
		141, 70, 0, 696, 694, 1, 0, 0, 0, 696, 695, 1, 0, 0, 0, 697, 98, 1, 0,
 | 
			
		||||
		0, 0, 698, 703, 3, 117, 58, 0, 699, 702, 3, 117, 58, 0, 700, 702, 3, 119,
 | 
			
		||||
		59, 0, 701, 699, 1, 0, 0, 0, 701, 700, 1, 0, 0, 0, 702, 705, 1, 0, 0, 0,
 | 
			
		||||
		703, 701, 1, 0, 0, 0, 703, 704, 1, 0, 0, 0, 704, 100, 1, 0, 0, 0, 705,
 | 
			
		||||
		703, 1, 0, 0, 0, 706, 707, 5, 36, 0, 0, 707, 708, 5, 109, 0, 0, 708, 709,
 | 
			
		||||
		5, 101, 0, 0, 709, 710, 5, 116, 0, 0, 710, 711, 5, 97, 0, 0, 711, 102,
 | 
			
		||||
		1, 0, 0, 0, 712, 714, 3, 107, 53, 0, 713, 712, 1, 0, 0, 0, 713, 714, 1,
 | 
			
		||||
		0, 0, 0, 714, 725, 1, 0, 0, 0, 715, 717, 5, 34, 0, 0, 716, 718, 3, 109,
 | 
			
		||||
		54, 0, 717, 716, 1, 0, 0, 0, 717, 718, 1, 0, 0, 0, 718, 719, 1, 0, 0, 0,
 | 
			
		||||
		719, 726, 5, 34, 0, 0, 720, 722, 5, 39, 0, 0, 721, 723, 3, 111, 55, 0,
 | 
			
		||||
		722, 721, 1, 0, 0, 0, 722, 723, 1, 0, 0, 0, 723, 724, 1, 0, 0, 0, 724,
 | 
			
		||||
		726, 5, 39, 0, 0, 725, 715, 1, 0, 0, 0, 725, 720, 1, 0, 0, 0, 726, 104,
 | 
			
		||||
		1, 0, 0, 0, 727, 730, 3, 99, 49, 0, 728, 730, 3, 101, 50, 0, 729, 727,
 | 
			
		||||
		1, 0, 0, 0, 729, 728, 1, 0, 0, 0, 730, 738, 1, 0, 0, 0, 731, 734, 5, 91,
 | 
			
		||||
		0, 0, 732, 735, 3, 103, 51, 0, 733, 735, 3, 123, 61, 0, 734, 732, 1, 0,
 | 
			
		||||
		0, 0, 734, 733, 1, 0, 0, 0, 735, 736, 1, 0, 0, 0, 736, 737, 5, 93, 0, 0,
 | 
			
		||||
		737, 739, 1, 0, 0, 0, 738, 731, 1, 0, 0, 0, 739, 740, 1, 0, 0, 0, 740,
 | 
			
		||||
		738, 1, 0, 0, 0, 740, 741, 1, 0, 0, 0, 741, 106, 1, 0, 0, 0, 742, 743,
 | 
			
		||||
		5, 117, 0, 0, 743, 746, 5, 56, 0, 0, 744, 746, 7, 0, 0, 0, 745, 742, 1,
 | 
			
		||||
		0, 0, 0, 745, 744, 1, 0, 0, 0, 746, 108, 1, 0, 0, 0, 747, 749, 3, 113,
 | 
			
		||||
		56, 0, 748, 747, 1, 0, 0, 0, 749, 750, 1, 0, 0, 0, 750, 748, 1, 0, 0, 0,
 | 
			
		||||
		750, 751, 1, 0, 0, 0, 751, 110, 1, 0, 0, 0, 752, 754, 3, 115, 57, 0, 753,
 | 
			
		||||
		752, 1, 0, 0, 0, 754, 755, 1, 0, 0, 0, 755, 753, 1, 0, 0, 0, 755, 756,
 | 
			
		||||
		1, 0, 0, 0, 756, 112, 1, 0, 0, 0, 757, 765, 8, 1, 0, 0, 758, 765, 3, 155,
 | 
			
		||||
		77, 0, 759, 760, 5, 92, 0, 0, 760, 765, 5, 10, 0, 0, 761, 762, 5, 92, 0,
 | 
			
		||||
		0, 762, 763, 5, 13, 0, 0, 763, 765, 5, 10, 0, 0, 764, 757, 1, 0, 0, 0,
 | 
			
		||||
		764, 758, 1, 0, 0, 0, 764, 759, 1, 0, 0, 0, 764, 761, 1, 0, 0, 0, 765,
 | 
			
		||||
		114, 1, 0, 0, 0, 766, 774, 8, 2, 0, 0, 767, 774, 3, 155, 77, 0, 768, 769,
 | 
			
		||||
		5, 92, 0, 0, 769, 774, 5, 10, 0, 0, 770, 771, 5, 92, 0, 0, 771, 772, 5,
 | 
			
		||||
		13, 0, 0, 772, 774, 5, 10, 0, 0, 773, 766, 1, 0, 0, 0, 773, 767, 1, 0,
 | 
			
		||||
		0, 0, 773, 768, 1, 0, 0, 0, 773, 770, 1, 0, 0, 0, 774, 116, 1, 0, 0, 0,
 | 
			
		||||
		775, 776, 7, 3, 0, 0, 776, 118, 1, 0, 0, 0, 777, 778, 7, 4, 0, 0, 778,
 | 
			
		||||
		120, 1, 0, 0, 0, 779, 780, 5, 48, 0, 0, 780, 782, 7, 5, 0, 0, 781, 783,
 | 
			
		||||
		7, 6, 0, 0, 782, 781, 1, 0, 0, 0, 783, 784, 1, 0, 0, 0, 784, 782, 1, 0,
 | 
			
		||||
		0, 0, 784, 785, 1, 0, 0, 0, 785, 122, 1, 0, 0, 0, 786, 790, 3, 129, 64,
 | 
			
		||||
		0, 787, 789, 3, 119, 59, 0, 788, 787, 1, 0, 0, 0, 789, 792, 1, 0, 0, 0,
 | 
			
		||||
		790, 788, 1, 0, 0, 0, 790, 791, 1, 0, 0, 0, 791, 795, 1, 0, 0, 0, 792,
 | 
			
		||||
		790, 1, 0, 0, 0, 793, 795, 5, 48, 0, 0, 794, 786, 1, 0, 0, 0, 794, 793,
 | 
			
		||||
		1, 0, 0, 0, 795, 124, 1, 0, 0, 0, 796, 800, 5, 48, 0, 0, 797, 799, 3, 131,
 | 
			
		||||
		65, 0, 798, 797, 1, 0, 0, 0, 799, 802, 1, 0, 0, 0, 800, 798, 1, 0, 0, 0,
 | 
			
		||||
		800, 801, 1, 0, 0, 0, 801, 126, 1, 0, 0, 0, 802, 800, 1, 0, 0, 0, 803,
 | 
			
		||||
		804, 5, 48, 0, 0, 804, 805, 7, 7, 0, 0, 805, 806, 3, 151, 75, 0, 806, 128,
 | 
			
		||||
		1, 0, 0, 0, 807, 808, 7, 8, 0, 0, 808, 130, 1, 0, 0, 0, 809, 810, 7, 9,
 | 
			
		||||
		0, 0, 810, 132, 1, 0, 0, 0, 811, 812, 7, 10, 0, 0, 812, 134, 1, 0, 0, 0,
 | 
			
		||||
		813, 814, 3, 133, 66, 0, 814, 815, 3, 133, 66, 0, 815, 816, 3, 133, 66,
 | 
			
		||||
		0, 816, 817, 3, 133, 66, 0, 817, 136, 1, 0, 0, 0, 818, 819, 5, 92, 0, 0,
 | 
			
		||||
		819, 820, 5, 117, 0, 0, 820, 821, 1, 0, 0, 0, 821, 829, 3, 135, 67, 0,
 | 
			
		||||
		822, 823, 5, 92, 0, 0, 823, 824, 5, 85, 0, 0, 824, 825, 1, 0, 0, 0, 825,
 | 
			
		||||
		826, 3, 135, 67, 0, 826, 827, 3, 135, 67, 0, 827, 829, 1, 0, 0, 0, 828,
 | 
			
		||||
		818, 1, 0, 0, 0, 828, 822, 1, 0, 0, 0, 829, 138, 1, 0, 0, 0, 830, 832,
 | 
			
		||||
		3, 143, 71, 0, 831, 833, 3, 145, 72, 0, 832, 831, 1, 0, 0, 0, 832, 833,
 | 
			
		||||
		1, 0, 0, 0, 833, 838, 1, 0, 0, 0, 834, 835, 3, 147, 73, 0, 835, 836, 3,
 | 
			
		||||
		145, 72, 0, 836, 838, 1, 0, 0, 0, 837, 830, 1, 0, 0, 0, 837, 834, 1, 0,
 | 
			
		||||
		0, 0, 838, 140, 1, 0, 0, 0, 839, 840, 5, 48, 0, 0, 840, 843, 7, 7, 0, 0,
 | 
			
		||||
		841, 844, 3, 149, 74, 0, 842, 844, 3, 151, 75, 0, 843, 841, 1, 0, 0, 0,
 | 
			
		||||
		843, 842, 1, 0, 0, 0, 844, 845, 1, 0, 0, 0, 845, 846, 3, 153, 76, 0, 846,
 | 
			
		||||
		142, 1, 0, 0, 0, 847, 849, 3, 147, 73, 0, 848, 847, 1, 0, 0, 0, 848, 849,
 | 
			
		||||
		1, 0, 0, 0, 849, 850, 1, 0, 0, 0, 850, 851, 5, 46, 0, 0, 851, 856, 3, 147,
 | 
			
		||||
		73, 0, 852, 853, 3, 147, 73, 0, 853, 854, 5, 46, 0, 0, 854, 856, 1, 0,
 | 
			
		||||
		0, 0, 855, 848, 1, 0, 0, 0, 855, 852, 1, 0, 0, 0, 856, 144, 1, 0, 0, 0,
 | 
			
		||||
		857, 859, 7, 11, 0, 0, 858, 860, 7, 12, 0, 0, 859, 858, 1, 0, 0, 0, 859,
 | 
			
		||||
		860, 1, 0, 0, 0, 860, 861, 1, 0, 0, 0, 861, 862, 3, 147, 73, 0, 862, 146,
 | 
			
		||||
		1, 0, 0, 0, 863, 865, 3, 119, 59, 0, 864, 863, 1, 0, 0, 0, 865, 866, 1,
 | 
			
		||||
		0, 0, 0, 866, 864, 1, 0, 0, 0, 866, 867, 1, 0, 0, 0, 867, 148, 1, 0, 0,
 | 
			
		||||
		0, 868, 870, 3, 151, 75, 0, 869, 868, 1, 0, 0, 0, 869, 870, 1, 0, 0, 0,
 | 
			
		||||
		870, 871, 1, 0, 0, 0, 871, 872, 5, 46, 0, 0, 872, 877, 3, 151, 75, 0, 873,
 | 
			
		||||
		874, 3, 151, 75, 0, 874, 875, 5, 46, 0, 0, 875, 877, 1, 0, 0, 0, 876, 869,
 | 
			
		||||
		1, 0, 0, 0, 876, 873, 1, 0, 0, 0, 877, 150, 1, 0, 0, 0, 878, 880, 3, 133,
 | 
			
		||||
		66, 0, 879, 878, 1, 0, 0, 0, 880, 881, 1, 0, 0, 0, 881, 879, 1, 0, 0, 0,
 | 
			
		||||
		881, 882, 1, 0, 0, 0, 882, 152, 1, 0, 0, 0, 883, 885, 7, 13, 0, 0, 884,
 | 
			
		||||
		886, 7, 12, 0, 0, 885, 884, 1, 0, 0, 0, 885, 886, 1, 0, 0, 0, 886, 887,
 | 
			
		||||
		1, 0, 0, 0, 887, 888, 3, 147, 73, 0, 888, 154, 1, 0, 0, 0, 889, 890, 5,
 | 
			
		||||
		92, 0, 0, 890, 905, 7, 14, 0, 0, 891, 892, 5, 92, 0, 0, 892, 894, 3, 131,
 | 
			
		||||
		65, 0, 893, 895, 3, 131, 65, 0, 894, 893, 1, 0, 0, 0, 894, 895, 1, 0, 0,
 | 
			
		||||
		0, 895, 897, 1, 0, 0, 0, 896, 898, 3, 131, 65, 0, 897, 896, 1, 0, 0, 0,
 | 
			
		||||
		897, 898, 1, 0, 0, 0, 898, 905, 1, 0, 0, 0, 899, 900, 5, 92, 0, 0, 900,
 | 
			
		||||
		901, 5, 120, 0, 0, 901, 902, 1, 0, 0, 0, 902, 905, 3, 151, 75, 0, 903,
 | 
			
		||||
		905, 3, 137, 68, 0, 904, 889, 1, 0, 0, 0, 904, 891, 1, 0, 0, 0, 904, 899,
 | 
			
		||||
		1, 0, 0, 0, 904, 903, 1, 0, 0, 0, 905, 156, 1, 0, 0, 0, 906, 908, 7, 15,
 | 
			
		||||
		0, 0, 907, 906, 1, 0, 0, 0, 908, 909, 1, 0, 0, 0, 909, 907, 1, 0, 0, 0,
 | 
			
		||||
		909, 910, 1, 0, 0, 0, 910, 911, 1, 0, 0, 0, 911, 912, 6, 78, 0, 0, 912,
 | 
			
		||||
		158, 1, 0, 0, 0, 913, 915, 5, 13, 0, 0, 914, 916, 5, 10, 0, 0, 915, 914,
 | 
			
		||||
		1, 0, 0, 0, 915, 916, 1, 0, 0, 0, 916, 919, 1, 0, 0, 0, 917, 919, 5, 10,
 | 
			
		||||
		0, 0, 918, 913, 1, 0, 0, 0, 918, 917, 1, 0, 0, 0, 919, 920, 1, 0, 0, 0,
 | 
			
		||||
		920, 921, 6, 79, 0, 0, 921, 160, 1, 0, 0, 0, 62, 0, 199, 213, 235, 261,
 | 
			
		||||
		289, 307, 315, 350, 358, 374, 398, 409, 415, 420, 422, 453, 489, 525, 555,
 | 
			
		||||
		593, 631, 657, 686, 692, 696, 701, 703, 713, 717, 722, 725, 729, 734, 740,
 | 
			
		||||
		745, 750, 755, 764, 773, 784, 790, 794, 800, 828, 832, 837, 843, 848, 855,
 | 
			
		||||
		859, 866, 869, 876, 881, 885, 894, 897, 904, 909, 915, 918, 1, 6, 0, 0,
 | 
			
		||||
	}
 | 
			
		||||
	deserializer := antlr.NewATNDeserializer(nil)
 | 
			
		||||
	staticData.atn = deserializer.Deserialize(staticData.serializedATN)
 | 
			
		||||
| 
						 | 
				
			
			@ -537,39 +552,41 @@ const (
 | 
			
		|||
	PlanLexerTEXTMATCH        = 16
 | 
			
		||||
	PlanLexerPHRASEMATCH      = 17
 | 
			
		||||
	PlanLexerRANDOMSAMPLE     = 18
 | 
			
		||||
	PlanLexerADD              = 19
 | 
			
		||||
	PlanLexerSUB              = 20
 | 
			
		||||
	PlanLexerMUL              = 21
 | 
			
		||||
	PlanLexerDIV              = 22
 | 
			
		||||
	PlanLexerMOD              = 23
 | 
			
		||||
	PlanLexerPOW              = 24
 | 
			
		||||
	PlanLexerSHL              = 25
 | 
			
		||||
	PlanLexerSHR              = 26
 | 
			
		||||
	PlanLexerBAND             = 27
 | 
			
		||||
	PlanLexerBOR              = 28
 | 
			
		||||
	PlanLexerBXOR             = 29
 | 
			
		||||
	PlanLexerAND              = 30
 | 
			
		||||
	PlanLexerOR               = 31
 | 
			
		||||
	PlanLexerISNULL           = 32
 | 
			
		||||
	PlanLexerISNOTNULL        = 33
 | 
			
		||||
	PlanLexerBNOT             = 34
 | 
			
		||||
	PlanLexerNOT              = 35
 | 
			
		||||
	PlanLexerIN               = 36
 | 
			
		||||
	PlanLexerEmptyArray       = 37
 | 
			
		||||
	PlanLexerJSONContains     = 38
 | 
			
		||||
	PlanLexerJSONContainsAll  = 39
 | 
			
		||||
	PlanLexerJSONContainsAny  = 40
 | 
			
		||||
	PlanLexerArrayContains    = 41
 | 
			
		||||
	PlanLexerArrayContainsAll = 42
 | 
			
		||||
	PlanLexerArrayContainsAny = 43
 | 
			
		||||
	PlanLexerArrayLength      = 44
 | 
			
		||||
	PlanLexerBooleanConstant  = 45
 | 
			
		||||
	PlanLexerIntegerConstant  = 46
 | 
			
		||||
	PlanLexerFloatingConstant = 47
 | 
			
		||||
	PlanLexerIdentifier       = 48
 | 
			
		||||
	PlanLexerMeta             = 49
 | 
			
		||||
	PlanLexerStringLiteral    = 50
 | 
			
		||||
	PlanLexerJSONIdentifier   = 51
 | 
			
		||||
	PlanLexerWhitespace       = 52
 | 
			
		||||
	PlanLexerNewline          = 53
 | 
			
		||||
	PlanLexerINTERVAL         = 19
 | 
			
		||||
	PlanLexerISO              = 20
 | 
			
		||||
	PlanLexerADD              = 21
 | 
			
		||||
	PlanLexerSUB              = 22
 | 
			
		||||
	PlanLexerMUL              = 23
 | 
			
		||||
	PlanLexerDIV              = 24
 | 
			
		||||
	PlanLexerMOD              = 25
 | 
			
		||||
	PlanLexerPOW              = 26
 | 
			
		||||
	PlanLexerSHL              = 27
 | 
			
		||||
	PlanLexerSHR              = 28
 | 
			
		||||
	PlanLexerBAND             = 29
 | 
			
		||||
	PlanLexerBOR              = 30
 | 
			
		||||
	PlanLexerBXOR             = 31
 | 
			
		||||
	PlanLexerAND              = 32
 | 
			
		||||
	PlanLexerOR               = 33
 | 
			
		||||
	PlanLexerISNULL           = 34
 | 
			
		||||
	PlanLexerISNOTNULL        = 35
 | 
			
		||||
	PlanLexerBNOT             = 36
 | 
			
		||||
	PlanLexerNOT              = 37
 | 
			
		||||
	PlanLexerIN               = 38
 | 
			
		||||
	PlanLexerEmptyArray       = 39
 | 
			
		||||
	PlanLexerJSONContains     = 40
 | 
			
		||||
	PlanLexerJSONContainsAll  = 41
 | 
			
		||||
	PlanLexerJSONContainsAny  = 42
 | 
			
		||||
	PlanLexerArrayContains    = 43
 | 
			
		||||
	PlanLexerArrayContainsAll = 44
 | 
			
		||||
	PlanLexerArrayContainsAny = 45
 | 
			
		||||
	PlanLexerArrayLength      = 46
 | 
			
		||||
	PlanLexerBooleanConstant  = 47
 | 
			
		||||
	PlanLexerIntegerConstant  = 48
 | 
			
		||||
	PlanLexerFloatingConstant = 49
 | 
			
		||||
	PlanLexerIdentifier       = 50
 | 
			
		||||
	PlanLexerMeta             = 51
 | 
			
		||||
	PlanLexerStringLiteral    = 52
 | 
			
		||||
	PlanLexerJSONIdentifier   = 53
 | 
			
		||||
	PlanLexerWhitespace       = 54
 | 
			
		||||
	PlanLexerNewline          = 55
 | 
			
		||||
)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							| 
						 | 
				
			
			@ -94,6 +94,9 @@ type PlanVisitor interface {
 | 
			
		|||
	// Visit a parse tree produced by PlanParser#Unary.
 | 
			
		||||
	VisitUnary(ctx *UnaryContext) interface{}
 | 
			
		||||
 | 
			
		||||
	// Visit a parse tree produced by PlanParser#TimestamptzCompare.
 | 
			
		||||
	VisitTimestamptzCompare(ctx *TimestamptzCompareContext) interface{}
 | 
			
		||||
 | 
			
		||||
	// Visit a parse tree produced by PlanParser#Integer.
 | 
			
		||||
	VisitInteger(ctx *IntegerContext) interface{}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -15,13 +15,18 @@ import (
 | 
			
		|||
	"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
type ParserVisitorArgs struct {
 | 
			
		||||
	TimezonePreference []string
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type ParserVisitor struct {
 | 
			
		||||
	parser.BasePlanVisitor
 | 
			
		||||
	schema *typeutil.SchemaHelper
 | 
			
		||||
	args   *ParserVisitorArgs
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func NewParserVisitor(schema *typeutil.SchemaHelper) *ParserVisitor {
 | 
			
		||||
	return &ParserVisitor{schema: schema}
 | 
			
		||||
func NewParserVisitor(schema *typeutil.SchemaHelper, args *ParserVisitorArgs) *ParserVisitor {
 | 
			
		||||
	return &ParserVisitor{schema: schema, args: args}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// VisitParens unpack the parentheses.
 | 
			
		||||
| 
						 | 
				
			
			@ -1613,3 +1618,60 @@ func (v *ParserVisitor) VisitTemplateVariable(ctx *parser.TemplateVariableContex
 | 
			
		|||
		},
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (v *ParserVisitor) VisitTimestamptzCompare(ctx *parser.TimestamptzCompareContext) interface{} {
 | 
			
		||||
	colExpr, err := v.translateIdentifier(ctx.Identifier().GetText())
 | 
			
		||||
	identifier := ctx.Identifier().Accept(v)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return fmt.Errorf("can not translate identifier: %s", identifier)
 | 
			
		||||
	}
 | 
			
		||||
	if colExpr.dataType != schemapb.DataType_Timestamptz {
 | 
			
		||||
		return fmt.Errorf("field '%s' is not a timestamptz datatype", identifier)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	arithOp := planpb.ArithOpType_Unknown
 | 
			
		||||
	interval := &planpb.Interval{}
 | 
			
		||||
	if ctx.GetOp1() != nil {
 | 
			
		||||
		arithOp = arithExprMap[ctx.GetOp1().GetTokenType()]
 | 
			
		||||
		rawIntervalStr := ctx.GetInterval_string().GetText()
 | 
			
		||||
		unquotedIntervalStr, err := convertEscapeSingle(rawIntervalStr)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return fmt.Errorf("can not convert interval string: %s", rawIntervalStr)
 | 
			
		||||
		}
 | 
			
		||||
		interval, err = parseISODuration(unquotedIntervalStr)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	rawCompareStr := ctx.GetCompare_string().GetText()
 | 
			
		||||
	unquotedCompareStr, err := convertEscapeSingle(rawCompareStr)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return fmt.Errorf("can not convert compare string: %s", rawCompareStr)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	compareOp := cmpOpMap[ctx.GetOp2().GetTokenType()]
 | 
			
		||||
 | 
			
		||||
	timestamptzInt64, err := parseISOWithTimezone(unquotedCompareStr, v.args.TimezonePreference)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	newExpr := &planpb.Expr{
 | 
			
		||||
		Expr: &planpb.Expr_TimestamptzArithCompareExpr{
 | 
			
		||||
			TimestamptzArithCompareExpr: &planpb.TimestamptzArithCompareExpr{
 | 
			
		||||
				TimestamptzColumn: toColumnInfo(colExpr),
 | 
			
		||||
				ArithOp:           arithOp,
 | 
			
		||||
				Interval:          interval,
 | 
			
		||||
				CompareOp:         compareOp,
 | 
			
		||||
				CompareValue: &planpb.GenericValue{
 | 
			
		||||
					Val: &planpb.GenericValue_Int64Val{Int64Val: timestamptzInt64},
 | 
			
		||||
				},
 | 
			
		||||
			},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return &ExprWithType{
 | 
			
		||||
		expr:     newExpr,
 | 
			
		||||
		dataType: schemapb.DataType_Bool,
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -98,7 +98,7 @@ func handleInternal(exprStr string) (ast planparserv2.IExprContext, err error) {
 | 
			
		|||
	return
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func handleExpr(schema *typeutil.SchemaHelper, exprStr string) (result interface{}) {
 | 
			
		||||
func handleExprInternal(schema *typeutil.SchemaHelper, exprStr string, visitorArgs *ParserVisitorArgs) (result interface{}) {
 | 
			
		||||
	defer func() {
 | 
			
		||||
		if r := recover(); r != nil {
 | 
			
		||||
			result = fmt.Errorf("unsupported expression: %s", exprStr)
 | 
			
		||||
| 
						 | 
				
			
			@ -113,12 +113,16 @@ func handleExpr(schema *typeutil.SchemaHelper, exprStr string) (result interface
 | 
			
		|||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	visitor := NewParserVisitor(schema)
 | 
			
		||||
	visitor := NewParserVisitor(schema, visitorArgs)
 | 
			
		||||
	return ast.Accept(visitor)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func ParseExpr(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues map[string]*schemapb.TemplateValue) (*planpb.Expr, error) {
 | 
			
		||||
	ret := handleExpr(schema, exprStr)
 | 
			
		||||
func handleExpr(schema *typeutil.SchemaHelper, exprStr string) (result interface{}) {
 | 
			
		||||
	return handleExprInternal(schema, exprStr, &ParserVisitorArgs{})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func parseExprInner(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues map[string]*schemapb.TemplateValue, visitorArgs *ParserVisitorArgs) (*planpb.Expr, error) {
 | 
			
		||||
	ret := handleExprInternal(schema, exprStr, visitorArgs)
 | 
			
		||||
 | 
			
		||||
	if err := getError(ret); err != nil {
 | 
			
		||||
		return nil, fmt.Errorf("cannot parse expression: %s, error: %s", exprStr, err)
 | 
			
		||||
| 
						 | 
				
			
			@ -144,8 +148,12 @@ func ParseExpr(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues
 | 
			
		|||
	return predicate.expr, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func ParseIdentifier(schema *typeutil.SchemaHelper, identifier string, checkFunc func(*planpb.Expr) error) error {
 | 
			
		||||
	ret := handleExpr(schema, identifier)
 | 
			
		||||
func ParseExpr(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues map[string]*schemapb.TemplateValue) (*planpb.Expr, error) {
 | 
			
		||||
	return parseExprInner(schema, exprStr, exprTemplateValues, &ParserVisitorArgs{})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func parseIdentifierInner(schema *typeutil.SchemaHelper, identifier string, checkFunc func(*planpb.Expr) error, visitorArgs *ParserVisitorArgs) error {
 | 
			
		||||
	ret := handleExprInternal(schema, identifier, visitorArgs)
 | 
			
		||||
 | 
			
		||||
	if err := getError(ret); err != nil {
 | 
			
		||||
		return fmt.Errorf("cannot parse identifier: %s, error: %s", identifier, err)
 | 
			
		||||
| 
						 | 
				
			
			@ -162,8 +170,13 @@ func ParseIdentifier(schema *typeutil.SchemaHelper, identifier string, checkFunc
 | 
			
		|||
	return checkFunc(predicate.expr)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func CreateRetrievePlan(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues map[string]*schemapb.TemplateValue) (*planpb.PlanNode, error) {
 | 
			
		||||
	expr, err := ParseExpr(schema, exprStr, exprTemplateValues)
 | 
			
		||||
func ParseIdentifier(schema *typeutil.SchemaHelper, identifier string, checkFunc func(*planpb.Expr) error) error {
 | 
			
		||||
	visitorArgs := &ParserVisitorArgs{}
 | 
			
		||||
	return parseIdentifierInner(schema, identifier, checkFunc, visitorArgs)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func CreateRetrievePlanArgs(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues map[string]*schemapb.TemplateValue, visitorArgs *ParserVisitorArgs) (*planpb.PlanNode, error) {
 | 
			
		||||
	expr, err := parseExprInner(schema, exprStr, exprTemplateValues, visitorArgs)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return nil, err
 | 
			
		||||
	}
 | 
			
		||||
| 
						 | 
				
			
			@ -183,12 +196,17 @@ func CreateRetrievePlan(schema *typeutil.SchemaHelper, exprStr string, exprTempl
 | 
			
		|||
	return planNode, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func CreateSearchPlan(schema *typeutil.SchemaHelper, exprStr string, vectorFieldName string, queryInfo *planpb.QueryInfo, exprTemplateValues map[string]*schemapb.TemplateValue, functionScorer *schemapb.FunctionScore) (*planpb.PlanNode, error) {
 | 
			
		||||
func CreateRetrievePlan(schema *typeutil.SchemaHelper, exprStr string, exprTemplateValues map[string]*schemapb.TemplateValue) (*planpb.PlanNode, error) {
 | 
			
		||||
	visitorArgs := &ParserVisitorArgs{}
 | 
			
		||||
	return CreateRetrievePlanArgs(schema, exprStr, exprTemplateValues, visitorArgs)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func CreateSearchPlanArgs(schema *typeutil.SchemaHelper, exprStr string, vectorFieldName string, queryInfo *planpb.QueryInfo, exprTemplateValues map[string]*schemapb.TemplateValue, functionScorer *schemapb.FunctionScore, visitorArgs *ParserVisitorArgs) (*planpb.PlanNode, error) {
 | 
			
		||||
	parse := func() (*planpb.Expr, error) {
 | 
			
		||||
		if len(exprStr) <= 0 {
 | 
			
		||||
			return nil, nil
 | 
			
		||||
		}
 | 
			
		||||
		return ParseExpr(schema, exprStr, exprTemplateValues)
 | 
			
		||||
		return parseExprInner(schema, exprStr, exprTemplateValues, visitorArgs)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	expr, err := parse()
 | 
			
		||||
| 
						 | 
				
			
			@ -317,6 +335,11 @@ func CreateSearchScorers(schema *typeutil.SchemaHelper, functionScore *schemapb.
 | 
			
		|||
	return scorers, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func CreateSearchPlan(schema *typeutil.SchemaHelper, exprStr string, vectorFieldName string, queryInfo *planpb.QueryInfo, exprTemplateValues map[string]*schemapb.TemplateValue, functionScorer *schemapb.FunctionScore) (*planpb.PlanNode, error) {
 | 
			
		||||
	visitorArgs := &ParserVisitorArgs{}
 | 
			
		||||
	return CreateSearchPlanArgs(schema, exprStr, vectorFieldName, queryInfo, exprTemplateValues, functionScorer, visitorArgs)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func CreateRequeryPlan(pkField *schemapb.FieldSchema, ids *schemapb.IDs) *planpb.PlanNode {
 | 
			
		||||
	var values []*planpb.GenericValue
 | 
			
		||||
	switch ids.GetIdField().(type) {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -5,6 +5,7 @@ import (
 | 
			
		|||
	"regexp"
 | 
			
		||||
	"strconv"
 | 
			
		||||
	"strings"
 | 
			
		||||
	"time"
 | 
			
		||||
	"unicode"
 | 
			
		||||
 | 
			
		||||
	"github.com/cockroachdb/errors"
 | 
			
		||||
| 
						 | 
				
			
			@ -787,3 +788,75 @@ func decodeUnicode(input string) string {
 | 
			
		|||
		return string(rune(code))
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func parseISODuration(durationStr string) (*planpb.Interval, error) {
 | 
			
		||||
	iso8601DurationRegex := regexp.MustCompile(
 | 
			
		||||
		`^P` + // P at the start
 | 
			
		||||
			`(?:(\d+)Y)?` + // Years (optional)
 | 
			
		||||
			`(?:(\d+)M)?` + // Months (optional)
 | 
			
		||||
			`(?:(\d+)D)?` + // Days (optional)
 | 
			
		||||
			`(?:T` + // T separator (optional, but required for time parts)
 | 
			
		||||
			`(?:(\d+)H)?` + // Hours (optional)
 | 
			
		||||
			`(?:(\d+)M)?` + // Minutes (optional)
 | 
			
		||||
			`(?:(\d+)S)?` + // Seconds (optional)
 | 
			
		||||
			`)?$`,
 | 
			
		||||
	)
 | 
			
		||||
	matches := iso8601DurationRegex.FindStringSubmatch(durationStr)
 | 
			
		||||
	if matches == nil {
 | 
			
		||||
		return nil, fmt.Errorf("invalid ISO 8601 duration: %s", durationStr)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	interval := &planpb.Interval{}
 | 
			
		||||
	targets := []struct {
 | 
			
		||||
		fieldPtr *int64
 | 
			
		||||
		unitName string
 | 
			
		||||
	}{
 | 
			
		||||
		{fieldPtr: &interval.Years, unitName: "year"},
 | 
			
		||||
		{fieldPtr: &interval.Months, unitName: "month"},
 | 
			
		||||
		{fieldPtr: &interval.Days, unitName: "day"},
 | 
			
		||||
		{fieldPtr: &interval.Hours, unitName: "hour"},
 | 
			
		||||
		{fieldPtr: &interval.Minutes, unitName: "minute"},
 | 
			
		||||
		{fieldPtr: &interval.Seconds, unitName: "second"},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for i, target := range targets {
 | 
			
		||||
		matchIndex := i + 1
 | 
			
		||||
		if matches[matchIndex] != "" {
 | 
			
		||||
			value, err := strconv.ParseInt(matches[matchIndex], 10, 64)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				return nil, fmt.Errorf("invalid %s value '%s' in duration: %w", target.unitName, matches[matchIndex], err)
 | 
			
		||||
			}
 | 
			
		||||
			*target.fieldPtr = value
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return interval, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func parseISOWithTimezone(isoString string, preferredZones []string) (int64, error) {
 | 
			
		||||
	timeZoneOffsetRegex := regexp.MustCompile(`([+-]\d{2}:\d{2}|Z)$`)
 | 
			
		||||
	// layout for timestamp string without timezone
 | 
			
		||||
	const layoutForNaiveTime = "2025-01-02T15:04:05.999999999"
 | 
			
		||||
	if timeZoneOffsetRegex.MatchString(isoString) { // has timezone
 | 
			
		||||
		t, err := time.Parse(time.RFC3339Nano, isoString)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return 0, fmt.Errorf("failed to parse timezone-aware string '%s': %w", isoString, err)
 | 
			
		||||
		}
 | 
			
		||||
		return t.UnixMicro(), nil
 | 
			
		||||
	}
 | 
			
		||||
	for _, zoneName := range preferredZones {
 | 
			
		||||
		loc, err := time.LoadLocation(zoneName)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
		t, err := time.ParseInLocation(layoutForNaiveTime, isoString, loc)
 | 
			
		||||
		if err == nil {
 | 
			
		||||
			return t.UnixMicro(), nil
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	t, err := time.ParseInLocation(layoutForNaiveTime, isoString, time.UTC)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return 0, fmt.Errorf("failed to parse naive time string '%s' even with UTC fallback: %w", isoString, err)
 | 
			
		||||
	}
 | 
			
		||||
	return t.UnixMicro(), nil
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1,6 +1,7 @@
 | 
			
		|||
package planparserv2
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"reflect"
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
| 
						 | 
				
			
			@ -590,3 +591,125 @@ func Test_handleCompare(t *testing.T) {
 | 
			
		|||
		assert.Equal(t, "var1", result.GetUnaryRangeExpr().GetTemplateVariableName())
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestParseISO8601Duration(t *testing.T) {
 | 
			
		||||
	testCases := []struct {
 | 
			
		||||
		name      string
 | 
			
		||||
		input     string
 | 
			
		||||
		expected  *planpb.Interval
 | 
			
		||||
		expectErr bool
 | 
			
		||||
	}{
 | 
			
		||||
		{
 | 
			
		||||
			name:  "Full duration",
 | 
			
		||||
			input: "P1Y2M3DT4H5M6S",
 | 
			
		||||
			expected: &planpb.Interval{
 | 
			
		||||
				Years:   1,
 | 
			
		||||
				Months:  2,
 | 
			
		||||
				Days:    3,
 | 
			
		||||
				Hours:   4,
 | 
			
		||||
				Minutes: 5,
 | 
			
		||||
				Seconds: 6,
 | 
			
		||||
			},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:  "Date part only",
 | 
			
		||||
			input: "P3Y6M4D",
 | 
			
		||||
			expected: &planpb.Interval{
 | 
			
		||||
				Years:  3,
 | 
			
		||||
				Months: 6,
 | 
			
		||||
				Days:   4,
 | 
			
		||||
			},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:  "Time part only",
 | 
			
		||||
			input: "PT10H30M15S",
 | 
			
		||||
			expected: &planpb.Interval{
 | 
			
		||||
				Hours:   10,
 | 
			
		||||
				Minutes: 30,
 | 
			
		||||
				Seconds: 15,
 | 
			
		||||
			},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:  "handle 0",
 | 
			
		||||
			input: "P0D",
 | 
			
		||||
			expected: &planpb.Interval{
 | 
			
		||||
				Days: 0,
 | 
			
		||||
			},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Ambiguous M for Month",
 | 
			
		||||
			input:     "P2M",
 | 
			
		||||
			expected:  &planpb.Interval{Months: 2},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Ambiguous M for Minute",
 | 
			
		||||
			input:     "PT2M",
 | 
			
		||||
			expected:  &planpb.Interval{Minutes: 2},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Mixed date and time with missing parts",
 | 
			
		||||
			input:     "P1DT12H",
 | 
			
		||||
			expected:  &planpb.Interval{Days: 1, Hours: 12},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Only P (valid empty duration)",
 | 
			
		||||
			input:     "P",
 | 
			
		||||
			expected:  &planpb.Interval{},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Only PT (valid empty time part)",
 | 
			
		||||
			input:     "PT",
 | 
			
		||||
			expected:  &planpb.Interval{},
 | 
			
		||||
			expectErr: false,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Invalid format - no P prefix",
 | 
			
		||||
			input:     "1Y2M",
 | 
			
		||||
			expected:  nil,
 | 
			
		||||
			expectErr: true,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Invalid format - unknown character",
 | 
			
		||||
			input:     "P1Y2X",
 | 
			
		||||
			expected:  nil,
 | 
			
		||||
			expectErr: true,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Invalid format - time part without T",
 | 
			
		||||
			input:     "P1H",
 | 
			
		||||
			expected:  nil,
 | 
			
		||||
			expectErr: true,
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			name:      "Invalid format - empty string",
 | 
			
		||||
			input:     "",
 | 
			
		||||
			expected:  nil,
 | 
			
		||||
			expectErr: true,
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for _, tc := range testCases {
 | 
			
		||||
		t.Run(tc.name, func(t *testing.T) {
 | 
			
		||||
			actual, err := parseISODuration(tc.input)
 | 
			
		||||
			if tc.expectErr {
 | 
			
		||||
				if err == nil {
 | 
			
		||||
					t.Errorf("expected an error but got none")
 | 
			
		||||
				}
 | 
			
		||||
				return
 | 
			
		||||
			}
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				t.Fatalf("did not expect an error but got: %v", err)
 | 
			
		||||
			}
 | 
			
		||||
			if !reflect.DeepEqual(actual, tc.expected) {
 | 
			
		||||
				t.Errorf("result mismatch:\nexpected: %+v\nactual:   %+v", tc.expected, actual)
 | 
			
		||||
			}
 | 
			
		||||
		})
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -633,6 +633,24 @@ func parseRankParams(rankParamsPair []*commonpb.KeyValuePair, schema *schemapb.C
 | 
			
		|||
	}, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func parseTimezone(params []*commonpb.KeyValuePair) string {
 | 
			
		||||
	timezone, err := funcutil.GetAttrByKeyFromRepeatedKV(TimezoneKey, params)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return ""
 | 
			
		||||
	}
 | 
			
		||||
	return timezone
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func parseTimeFields(params []*commonpb.KeyValuePair) []string {
 | 
			
		||||
	timeFields, err := funcutil.GetAttrByKeyFromRepeatedKV(TimefieldsKey, params)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return nil
 | 
			
		||||
	}
 | 
			
		||||
	return strings.FieldsFunc(timeFields, func(r rune) bool {
 | 
			
		||||
		return r == ',' || r == ' '
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func getGroupScorerStr(params []*commonpb.KeyValuePair) string {
 | 
			
		||||
	groupScorerStr, err := funcutil.GetAttrByKeyFromRepeatedKV(RankGroupScorer, params)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -73,6 +73,9 @@ const (
 | 
			
		|||
	LimitKey             = "limit"
 | 
			
		||||
	// offsets for embedding list search
 | 
			
		||||
	LimsKey = "lims"
 | 
			
		||||
	// key for timestamptz translation
 | 
			
		||||
	TimezoneKey   = "timezone"
 | 
			
		||||
	TimefieldsKey = "time_fields"
 | 
			
		||||
 | 
			
		||||
	SearchIterV2Key        = "search_iter_v2"
 | 
			
		||||
	SearchIterBatchSizeKey = "search_iter_batch_size"
 | 
			
		||||
| 
						 | 
				
			
			@ -449,10 +452,6 @@ func (t *createCollectionTask) PreExecute(ctx context.Context) error {
 | 
			
		|||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	// prevent user creating collection with timestamptz field for now (not implemented)
 | 
			
		||||
	if hasTimestamptzField(t.schema) {
 | 
			
		||||
		return merr.WrapErrParameterInvalidMsg("timestamptz field is still in development")
 | 
			
		||||
	}
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -1177,6 +1176,11 @@ func (t *alterCollectionTask) PreExecute(ctx context.Context) error {
 | 
			
		|||
				return merr.WrapErrCollectionLoaded(t.CollectionName, "can not alter mmap properties if collection loaded")
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
		// Check the validation of timezone
 | 
			
		||||
		err := checkTimezone(t.Properties...)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
	} else if len(t.GetDeleteKeys()) > 0 {
 | 
			
		||||
		key := hasPropInDeletekeys(t.DeleteKeys)
 | 
			
		||||
		if key != "" {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -265,6 +265,13 @@ func (t *alterDatabaseTask) OnEnqueue() error {
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
func (t *alterDatabaseTask) PreExecute(ctx context.Context) error {
 | 
			
		||||
	if len(t.GetProperties()) > 0 {
 | 
			
		||||
		// Check the validation of timezone
 | 
			
		||||
		err := checkTimezone(t.Properties...)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	_, ok := common.GetReplicateID(t.Properties)
 | 
			
		||||
	if ok {
 | 
			
		||||
		return merr.WrapErrParameterInvalidMsg("can't set the replicate id property in alter database request")
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -291,8 +291,17 @@ func (dr *deleteRunner) Init(ctx context.Context) error {
 | 
			
		|||
		return ErrWithLog(log, "Failed to get collection schema", err)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	colInfo, err := globalMetaCache.GetCollectionInfo(ctx, dr.req.GetDbName(), collName, dr.collectionID)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return ErrWithLog(log, "Failed to get collection info", err)
 | 
			
		||||
	}
 | 
			
		||||
	_, dbTimezone := getDbTimezone(db)
 | 
			
		||||
	_, colTimezone := getColTimezone(colInfo)
 | 
			
		||||
	timezonePreference := []string{colTimezone, dbTimezone}
 | 
			
		||||
	visitorArgs := &planparserv2.ParserVisitorArgs{TimezonePreference: timezonePreference}
 | 
			
		||||
 | 
			
		||||
	start := time.Now()
 | 
			
		||||
	dr.plan, err = planparserv2.CreateRetrievePlan(dr.schema.schemaHelper, dr.req.GetExpr(), dr.req.GetExprTemplateValues())
 | 
			
		||||
	dr.plan, err = planparserv2.CreateRetrievePlanArgs(dr.schema.schemaHelper, dr.req.GetExpr(), dr.req.GetExprTemplateValues(), visitorArgs)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		metrics.ProxyParseExpressionLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "delete", metrics.FailLabel).Observe(float64(time.Since(start).Milliseconds()))
 | 
			
		||||
		return merr.WrapErrAsInputError(merr.WrapErrParameterInvalidMsg("failed to create delete plan: %v", err))
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -254,6 +254,13 @@ func (it *insertTask) PreExecute(ctx context.Context) error {
 | 
			
		|||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// trans timestamptz data
 | 
			
		||||
	_, colTimezone := getColTimezone(colInfo)
 | 
			
		||||
	err = timestamptzIsoStr2Utc(it.insertMsg.GetFieldsData(), colTimezone, "")
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	partitionKeyMode, err := isPartitionKeyMode(ctx, it.insertMsg.GetDbName(), collectionName)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Warn("check partition key mode failed", zap.String("collectionName", collectionName), zap.Error(err))
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -81,11 +81,13 @@ type queryTask struct {
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
type queryParams struct {
 | 
			
		||||
	limit        int64
 | 
			
		||||
	offset       int64
 | 
			
		||||
	reduceType   reduce.IReduceType
 | 
			
		||||
	isIterator   bool
 | 
			
		||||
	collectionID int64
 | 
			
		||||
	limit             int64
 | 
			
		||||
	offset            int64
 | 
			
		||||
	reduceType        reduce.IReduceType
 | 
			
		||||
	isIterator        bool
 | 
			
		||||
	collectionID      int64
 | 
			
		||||
	timezone          string
 | 
			
		||||
	extractTimeFields []string
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// translateToOutputFieldIDs translates output fields name to output fields id.
 | 
			
		||||
| 
						 | 
				
			
			@ -168,6 +170,8 @@ func parseQueryParams(queryParamsPair []*commonpb.KeyValuePair) (*queryParams, e
 | 
			
		|||
		isIterator        bool
 | 
			
		||||
		err               error
 | 
			
		||||
		collectionID      int64
 | 
			
		||||
		timezone          string
 | 
			
		||||
		extractTimeFields []string
 | 
			
		||||
	)
 | 
			
		||||
	reduceStopForBestStr, err := funcutil.GetAttrByKeyFromRepeatedKV(ReduceStopForBestKey, queryParamsPair)
 | 
			
		||||
	// if reduce_stop_for_best is provided
 | 
			
		||||
| 
						 | 
				
			
			@ -226,17 +230,31 @@ func parseQueryParams(queryParamsPair []*commonpb.KeyValuePair) (*queryParams, e
 | 
			
		|||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	timezoneStr, err := funcutil.GetAttrByKeyFromRepeatedKV(TimezoneKey, queryParamsPair)
 | 
			
		||||
	if err == nil {
 | 
			
		||||
		timezone = timezoneStr
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	extractTimeFieldsStr, err := funcutil.GetAttrByKeyFromRepeatedKV(TimefieldsKey, queryParamsPair)
 | 
			
		||||
	if err == nil {
 | 
			
		||||
		extractTimeFields = strings.FieldsFunc(extractTimeFieldsStr, func(r rune) bool {
 | 
			
		||||
			return r == ',' || r == ' '
 | 
			
		||||
		})
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// validate max result window.
 | 
			
		||||
	if err = validateMaxQueryResultWindow(offset, limit); err != nil {
 | 
			
		||||
		return nil, fmt.Errorf("invalid max query result window, %w", err)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return &queryParams{
 | 
			
		||||
		limit:        limit,
 | 
			
		||||
		offset:       offset,
 | 
			
		||||
		reduceType:   reduceType,
 | 
			
		||||
		isIterator:   isIterator,
 | 
			
		||||
		collectionID: collectionID,
 | 
			
		||||
		limit:             limit,
 | 
			
		||||
		offset:            offset,
 | 
			
		||||
		reduceType:        reduceType,
 | 
			
		||||
		isIterator:        isIterator,
 | 
			
		||||
		collectionID:      collectionID,
 | 
			
		||||
		timezone:          timezone,
 | 
			
		||||
		extractTimeFields: extractTimeFields,
 | 
			
		||||
	}, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -268,6 +286,10 @@ func createCntPlan(expr string, schemaHelper *typeutil.SchemaHelper, exprTemplat
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
func (t *queryTask) createPlan(ctx context.Context) error {
 | 
			
		||||
	return t.createPlanArgs(ctx, &planparserv2.ParserVisitorArgs{})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (t *queryTask) createPlanArgs(ctx context.Context, visitorArgs *planparserv2.ParserVisitorArgs) error {
 | 
			
		||||
	schema := t.schema
 | 
			
		||||
 | 
			
		||||
	cntMatch := matchCountRule(t.request.GetOutputFields())
 | 
			
		||||
| 
						 | 
				
			
			@ -281,7 +303,7 @@ func (t *queryTask) createPlan(ctx context.Context) error {
 | 
			
		|||
	var err error
 | 
			
		||||
	if t.plan == nil {
 | 
			
		||||
		start := time.Now()
 | 
			
		||||
		t.plan, err = planparserv2.CreateRetrievePlan(schema.schemaHelper, t.request.Expr, t.request.GetExprTemplateValues())
 | 
			
		||||
		t.plan, err = planparserv2.CreateRetrievePlanArgs(schema.schemaHelper, t.request.Expr, t.request.GetExprTemplateValues(), visitorArgs)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			metrics.ProxyParseExpressionLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "query", metrics.FailLabel).Observe(float64(time.Since(start).Milliseconds()))
 | 
			
		||||
			return merr.WrapErrAsInputError(merr.WrapErrParameterInvalidMsg("failed to create query plan: %v", err))
 | 
			
		||||
| 
						 | 
				
			
			@ -360,6 +382,13 @@ func (t *queryTask) PreExecute(ctx context.Context) error {
 | 
			
		|||
		return merr.WrapErrAsInputErrorWhen(err, merr.ErrCollectionNotFound, merr.ErrDatabaseNotFound)
 | 
			
		||||
	}
 | 
			
		||||
	t.CollectionID = collID
 | 
			
		||||
 | 
			
		||||
	colInfo, err := globalMetaCache.GetCollectionInfo(ctx, t.request.GetDbName(), collectionName, t.CollectionID)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Warn("Failed to get collection info.", zap.String("collectionName", collectionName),
 | 
			
		||||
			zap.Int64("collectionID", t.CollectionID), zap.Error(err))
 | 
			
		||||
		return merr.WrapErrAsInputErrorWhen(err, merr.ErrCollectionNotFound, merr.ErrDatabaseNotFound)
 | 
			
		||||
	}
 | 
			
		||||
	log.Debug("Get collection ID by name", zap.Int64("collectionID", t.CollectionID))
 | 
			
		||||
 | 
			
		||||
	t.partitionKeyMode, err = isPartitionKeyMode(ctx, t.request.GetDbName(), collectionName)
 | 
			
		||||
| 
						 | 
				
			
			@ -421,7 +450,9 @@ func (t *queryTask) PreExecute(ctx context.Context) error {
 | 
			
		|||
		t.request.Expr = IDs2Expr(pkField, t.ids)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if err := t.createPlan(ctx); err != nil {
 | 
			
		||||
	_, colTimezone := getColTimezone(colInfo)
 | 
			
		||||
	timezonePreference := []string{t.queryParams.timezone, colTimezone}
 | 
			
		||||
	if err := t.createPlanArgs(ctx, &planparserv2.ParserVisitorArgs{TimezonePreference: timezonePreference}); err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	t.plan.Node.(*planpb.PlanNode_Query).Query.Limit = t.RetrieveRequest.Limit
 | 
			
		||||
| 
						 | 
				
			
			@ -672,6 +703,37 @@ func (t *queryTask) PostExecute(ctx context.Context) error {
 | 
			
		|||
		// first page for iteration, need to set up sessionTs for iterator
 | 
			
		||||
		t.result.SessionTs = getMaxMvccTsFromChannels(t.channelsMvcc, t.BeginTs())
 | 
			
		||||
	}
 | 
			
		||||
	// Translate timestamp to ISO string
 | 
			
		||||
	collName := t.request.GetCollectionName()
 | 
			
		||||
	dbName := t.request.GetDbName()
 | 
			
		||||
	collID, err := globalMetaCache.GetCollectionID(context.Background(), dbName, collName)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Warn("fail to get collection id", zap.Error(err))
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	colInfo, err := globalMetaCache.GetCollectionInfo(ctx, dbName, collName, collID)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Warn("fail to get collection info", zap.Error(err))
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	_, colTimezone := getColTimezone(colInfo)
 | 
			
		||||
	if !t.reQuery {
 | 
			
		||||
		if len(t.queryParams.extractTimeFields) > 0 {
 | 
			
		||||
			log.Debug("extracting fields for timestamptz", zap.Strings("fields", t.queryParams.extractTimeFields))
 | 
			
		||||
			err = extractFieldsFromResults(t.result.GetFieldsData(), []string{t.queryParams.timezone, colTimezone}, t.queryParams.extractTimeFields)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				log.Warn("fail to extract fields for timestamptz", zap.Error(err))
 | 
			
		||||
				return err
 | 
			
		||||
			}
 | 
			
		||||
		} else {
 | 
			
		||||
			log.Debug("translate timestamp to ISO string", zap.String("user define timezone", t.queryParams.timezone))
 | 
			
		||||
			err = timestamptzUTC2IsoStr(t.result.GetFieldsData(), t.queryParams.timezone, colTimezone)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				log.Warn("fail to translate timestamp", zap.Error(err))
 | 
			
		||||
				return err
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	log.Debug("Query PostExecute done")
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -819,6 +819,38 @@ func (t *searchTask) PostExecute(ctx context.Context) error {
 | 
			
		|||
 | 
			
		||||
	metrics.ProxyReduceResultLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds()))
 | 
			
		||||
 | 
			
		||||
	// Translate timestamp to ISO string
 | 
			
		||||
	collName := t.request.GetCollectionName()
 | 
			
		||||
	dbName := t.request.GetDbName()
 | 
			
		||||
	collID, err := globalMetaCache.GetCollectionID(context.Background(), dbName, collName)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Warn("fail to get collection id", zap.Error(err))
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	colInfo, err := globalMetaCache.GetCollectionInfo(ctx, dbName, collName, collID)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Warn("fail to get collection info", zap.Error(err))
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	_, colTimezone := getColTimezone(colInfo)
 | 
			
		||||
	timeFields := parseTimeFields(t.request.SearchParams)
 | 
			
		||||
	timezoneUserDefined := parseTimezone(t.request.SearchParams)
 | 
			
		||||
	if timeFields != nil {
 | 
			
		||||
		log.Debug("extracting fields for timestamptz", zap.Strings("fields", timeFields))
 | 
			
		||||
		err = extractFieldsFromResults(t.result.GetResults().GetFieldsData(), []string{timezoneUserDefined, colTimezone}, timeFields)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			log.Warn("fail to extract fields for timestamptz", zap.Error(err))
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
	} else {
 | 
			
		||||
		log.Debug("translate timstamp to ISO string", zap.String("user define timezone", timezoneUserDefined))
 | 
			
		||||
		err = timestamptzUTC2IsoStr(t.result.GetResults().GetFieldsData(), timezoneUserDefined, colTimezone)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			log.Warn("fail to translate timestamp", zap.Error(err))
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	log.Debug("Search post execute done",
 | 
			
		||||
		zap.Int64("collection", t.GetCollectionID()),
 | 
			
		||||
		zap.Int64s("partitionIDs", t.GetPartitionIDs()))
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -4068,6 +4068,7 @@ func TestSearchTask_Requery(t *testing.T) {
 | 
			
		|||
	cache.EXPECT().GetCollectionInfo(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&collectionInfo{}, nil).Maybe()
 | 
			
		||||
	cache.EXPECT().GetShard(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]nodeInfo{}, nil).Maybe()
 | 
			
		||||
	cache.EXPECT().DeprecateShardCache(mock.Anything, mock.Anything).Return().Maybe()
 | 
			
		||||
	cache.EXPECT().GetDatabaseInfo(mock.Anything, mock.Anything).Return(&databaseInfo{}, nil).Maybe()
 | 
			
		||||
	globalMetaCache = cache
 | 
			
		||||
 | 
			
		||||
	t.Run("Test normal", func(t *testing.T) {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -3530,6 +3530,11 @@ func TestCreateCollectionTaskWithPartitionKey(t *testing.T) {
 | 
			
		|||
	paramtable.Init()
 | 
			
		||||
 | 
			
		||||
	defer rc.Close()
 | 
			
		||||
	cache := globalMetaCache
 | 
			
		||||
	defer func() { globalMetaCache = cache }()
 | 
			
		||||
	mockCache := NewMockCache(t)
 | 
			
		||||
	mockCache.EXPECT().GetDatabaseInfo(mock.Anything, mock.Anything).Return(&databaseInfo{}, nil).Maybe()
 | 
			
		||||
	globalMetaCache = mockCache
 | 
			
		||||
	ctx := context.Background()
 | 
			
		||||
	shardsNum := common.DefaultShardsNum
 | 
			
		||||
	prefix := "TestCreateCollectionTaskWithPartitionKey"
 | 
			
		||||
| 
						 | 
				
			
			@ -5092,6 +5097,11 @@ func constructCollectionSchemaWithStructArrayField(collectionName string, struct
 | 
			
		|||
// TestCreateCollectionTaskWithStructArrayField tests creating collections with StructArrayField
 | 
			
		||||
func TestCreateCollectionTaskWithStructArrayField(t *testing.T) {
 | 
			
		||||
	mix := NewMixCoordMock()
 | 
			
		||||
	cache := globalMetaCache
 | 
			
		||||
	defer func() { globalMetaCache = cache }()
 | 
			
		||||
	mockCache := NewMockCache(t)
 | 
			
		||||
	mockCache.EXPECT().GetDatabaseInfo(mock.Anything, mock.Anything).Return(&databaseInfo{}, nil).Maybe()
 | 
			
		||||
	globalMetaCache = mockCache
 | 
			
		||||
	ctx := context.Background()
 | 
			
		||||
	shardsNum := common.DefaultShardsNum
 | 
			
		||||
	prefix := "TestCreateCollectionTaskWithStructArrayField"
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -2726,3 +2726,240 @@ func hasTimestamptzField(schema *schemapb.CollectionSchema) bool {
 | 
			
		|||
	}
 | 
			
		||||
	return false
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func getDefaultTimezoneVal(props ...*commonpb.KeyValuePair) (bool, string) {
 | 
			
		||||
	for _, p := range props {
 | 
			
		||||
		// used in collection or database
 | 
			
		||||
		if p.GetKey() == common.DatabaseDefaultTimezone || p.GetKey() == common.CollectionDefaultTimezone {
 | 
			
		||||
			return true, p.Value
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return false, ""
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func checkTimezone(props ...*commonpb.KeyValuePair) error {
 | 
			
		||||
	hasTImezone, timezoneStr := getDefaultTimezoneVal(props...)
 | 
			
		||||
	if hasTImezone {
 | 
			
		||||
		_, err := time.LoadLocation(timezoneStr)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return merr.WrapErrParameterInvalidMsg("invalid timezone, should be a IANA timezone name: %s", err.Error())
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func getColTimezone(colInfo *collectionInfo) (bool, string) {
 | 
			
		||||
	return getDefaultTimezoneVal(colInfo.properties...)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func getDbTimezone(dbInfo *databaseInfo) (bool, string) {
 | 
			
		||||
	return getDefaultTimezoneVal(dbInfo.properties...)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func timestamptzIsoStr2Utc(columns []*schemapb.FieldData, colTimezone string, dbTimezone string) error {
 | 
			
		||||
	naiveLayouts := []string{
 | 
			
		||||
		"2006-01-02T15:04:05.999999999",
 | 
			
		||||
		"2006-01-02T15:04:05",
 | 
			
		||||
		"2006-01-02 15:04:05.999999999",
 | 
			
		||||
		"2006-01-02 15:04:05",
 | 
			
		||||
	}
 | 
			
		||||
	for _, fieldData := range columns {
 | 
			
		||||
		if fieldData.GetType() != schemapb.DataType_Timestamptz {
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		scalarField := fieldData.GetScalars()
 | 
			
		||||
		if scalarField == nil || scalarField.GetStringData() == nil {
 | 
			
		||||
			log.Warn("field data is not string data", zap.String("fieldName", fieldData.GetFieldName()))
 | 
			
		||||
			return merr.WrapErrParameterInvalidMsg("field data is not string data")
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		stringData := scalarField.GetStringData().GetData()
 | 
			
		||||
		utcTimestamps := make([]int64, len(stringData))
 | 
			
		||||
 | 
			
		||||
		for i, isoStr := range stringData {
 | 
			
		||||
			var t time.Time
 | 
			
		||||
			var err error
 | 
			
		||||
			// parse directly
 | 
			
		||||
			t, err = time.Parse(time.RFC3339Nano, isoStr)
 | 
			
		||||
			if err == nil {
 | 
			
		||||
				utcTimestamps[i] = t.UnixMicro()
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
			// no timezone, try to find timezone in collecion -> database level
 | 
			
		||||
			defaultTZ := "UTC"
 | 
			
		||||
			if colTimezone != "" {
 | 
			
		||||
				defaultTZ = colTimezone
 | 
			
		||||
			} else if dbTimezone != "" {
 | 
			
		||||
				defaultTZ = dbTimezone
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			location, err := time.LoadLocation(defaultTZ)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				log.Error("invalid timezone", zap.String("timezone", defaultTZ), zap.Error(err))
 | 
			
		||||
				return merr.WrapErrParameterInvalidMsg("got invalid default timezone: %s", defaultTZ)
 | 
			
		||||
			}
 | 
			
		||||
			var parsed bool
 | 
			
		||||
			for _, layout := range naiveLayouts {
 | 
			
		||||
				t, err = time.ParseInLocation(layout, isoStr, location)
 | 
			
		||||
				if err == nil {
 | 
			
		||||
					parsed = true
 | 
			
		||||
					break
 | 
			
		||||
				}
 | 
			
		||||
			}
 | 
			
		||||
			if !parsed {
 | 
			
		||||
				log.Warn("Can not parse timestamptz string", zap.String("timestamp_string", isoStr))
 | 
			
		||||
				return merr.WrapErrParameterInvalidMsg("got invalid timestamptz string: %s", isoStr)
 | 
			
		||||
			}
 | 
			
		||||
			utcTimestamps[i] = t.UnixMicro()
 | 
			
		||||
		}
 | 
			
		||||
		// Replace data in place
 | 
			
		||||
		fieldData.GetScalars().Data = &schemapb.ScalarField_TimestamptzData{
 | 
			
		||||
			TimestamptzData: &schemapb.TimestamptzArray{
 | 
			
		||||
				Data: utcTimestamps,
 | 
			
		||||
			},
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func timestamptzUTC2IsoStr(results []*schemapb.FieldData, userDefineTimezone string, colTimezone string) error {
 | 
			
		||||
	// Determine the target timezone based on priority: collection -> database -> UTC.
 | 
			
		||||
	defaultTZ := "UTC"
 | 
			
		||||
	if userDefineTimezone != "" {
 | 
			
		||||
		defaultTZ = userDefineTimezone
 | 
			
		||||
	} else if colTimezone != "" {
 | 
			
		||||
		defaultTZ = colTimezone
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	location, err := time.LoadLocation(defaultTZ)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		log.Error("invalid timezone", zap.String("timezone", defaultTZ), zap.Error(err))
 | 
			
		||||
		return merr.WrapErrParameterInvalidMsg("got invalid default timezone: %s", defaultTZ)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for _, fieldData := range results {
 | 
			
		||||
		if fieldData.GetType() != schemapb.DataType_Timestamptz {
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
		scalarField := fieldData.GetScalars()
 | 
			
		||||
		if scalarField == nil || scalarField.GetTimestamptzData() == nil {
 | 
			
		||||
			if longData := scalarField.GetLongData(); longData != nil && len(longData.GetData()) > 0 {
 | 
			
		||||
				log.Warn("field data is not Timestamptz data", zap.String("fieldName", fieldData.GetFieldName()))
 | 
			
		||||
				return merr.WrapErrParameterInvalidMsg("field data for '%s' is not Timestamptz data", fieldData.GetFieldName())
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		utcTimestamps := scalarField.GetTimestamptzData().GetData()
 | 
			
		||||
		isoStrings := make([]string, len(utcTimestamps))
 | 
			
		||||
 | 
			
		||||
		for i, ts := range utcTimestamps {
 | 
			
		||||
			t := time.UnixMicro(ts).UTC()
 | 
			
		||||
			localTime := t.In(location)
 | 
			
		||||
			isoStrings[i] = localTime.Format(time.RFC3339Nano)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Replace the TimestamptzData with the new StringData in place.
 | 
			
		||||
		fieldData.GetScalars().Data = &schemapb.ScalarField_StringData{
 | 
			
		||||
			StringData: &schemapb.StringArray{
 | 
			
		||||
				Data: isoStrings,
 | 
			
		||||
			},
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// extractFields is a helper function to extract specific integer fields from a time.Time object.
 | 
			
		||||
// Supported fields are: "year", "month", "day", "hour", "minute", "second", "microsecond", "nanosecond".
 | 
			
		||||
func extractFields(t time.Time, fieldList []string) ([]int64, error) {
 | 
			
		||||
	extractedValues := make([]int64, 0, len(fieldList))
 | 
			
		||||
	for _, field := range fieldList {
 | 
			
		||||
		var val int64
 | 
			
		||||
		switch strings.ToLower(field) {
 | 
			
		||||
		case common.TszYear:
 | 
			
		||||
			val = int64(t.Year())
 | 
			
		||||
		case common.TszMonth:
 | 
			
		||||
			val = int64(t.Month())
 | 
			
		||||
		case common.TszDay:
 | 
			
		||||
			val = int64(t.Day())
 | 
			
		||||
		case common.TszHour:
 | 
			
		||||
			val = int64(t.Hour())
 | 
			
		||||
		case common.TszMinute:
 | 
			
		||||
			val = int64(t.Minute())
 | 
			
		||||
		case common.TszSecond:
 | 
			
		||||
			val = int64(t.Second())
 | 
			
		||||
		case common.TszMicrosecond:
 | 
			
		||||
			val = int64(t.Nanosecond() / 1000)
 | 
			
		||||
		default:
 | 
			
		||||
			return nil, merr.WrapErrParameterInvalidMsg("unsupported field for extraction: %s, fields should be seprated by ',' or ' '", field)
 | 
			
		||||
		}
 | 
			
		||||
		extractedValues = append(extractedValues, val)
 | 
			
		||||
	}
 | 
			
		||||
	return extractedValues, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func extractFieldsFromResults(results []*schemapb.FieldData, precedenceTimezone []string, fieldList []string) error {
 | 
			
		||||
	var targetLocation *time.Location
 | 
			
		||||
 | 
			
		||||
	for _, tz := range precedenceTimezone {
 | 
			
		||||
		if tz != "" {
 | 
			
		||||
			loc, err := time.LoadLocation(tz)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				log.Error("invalid timezone provided in precedence list", zap.String("timezone", tz), zap.Error(err))
 | 
			
		||||
				return merr.WrapErrParameterInvalidMsg("got invalid timezone: %s", tz)
 | 
			
		||||
			}
 | 
			
		||||
			targetLocation = loc
 | 
			
		||||
			break // Use the first valid timezone found.
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if targetLocation == nil {
 | 
			
		||||
		targetLocation = time.UTC
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for _, fieldData := range results {
 | 
			
		||||
		if fieldData.GetType() != schemapb.DataType_Timestamptz {
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		scalarField := fieldData.GetScalars()
 | 
			
		||||
		if scalarField == nil || scalarField.GetTimestamptzData() == nil {
 | 
			
		||||
			if longData := scalarField.GetLongData(); longData != nil && len(longData.GetData()) > 0 {
 | 
			
		||||
				log.Warn("field data is not Timestamptz data, but found LongData instead", zap.String("fieldName", fieldData.GetFieldName()))
 | 
			
		||||
				return merr.WrapErrParameterInvalidMsg("field data for '%s' is not Timestamptz data", fieldData.GetFieldName())
 | 
			
		||||
			}
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		utcTimestamps := scalarField.GetTimestamptzData().GetData()
 | 
			
		||||
		extractedResults := make([]*schemapb.ScalarField, 0, len(fieldList))
 | 
			
		||||
 | 
			
		||||
		for _, ts := range utcTimestamps {
 | 
			
		||||
			t := time.UnixMicro(ts).UTC()
 | 
			
		||||
			localTime := t.In(targetLocation)
 | 
			
		||||
 | 
			
		||||
			values, err := extractFields(localTime, fieldList)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				return err
 | 
			
		||||
			}
 | 
			
		||||
			valuesScalarField := &schemapb.ScalarField_LongData{
 | 
			
		||||
				LongData: &schemapb.LongArray{
 | 
			
		||||
					Data: values,
 | 
			
		||||
				},
 | 
			
		||||
			}
 | 
			
		||||
			extractedResults = append(extractedResults, &schemapb.ScalarField{
 | 
			
		||||
				Data: valuesScalarField,
 | 
			
		||||
			})
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		fieldData.GetScalars().Data = &schemapb.ScalarField_ArrayData{
 | 
			
		||||
			ArrayData: &schemapb.ArrayArray{
 | 
			
		||||
				Data:        extractedResults,
 | 
			
		||||
				ElementType: schemapb.DataType_Int64,
 | 
			
		||||
			},
 | 
			
		||||
		}
 | 
			
		||||
		fieldData.Type = schemapb.DataType_Array
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -109,6 +109,7 @@ func (m *collectionManager) PutOrRef(collectionID int64, schema *schemapb.Collec
 | 
			
		|||
 | 
			
		||||
	log.Info("put new collection", zap.Int64("collectionID", collectionID), zap.Any("schema", schema))
 | 
			
		||||
	collection, err := NewCollection(collectionID, schema, meta, loadMeta)
 | 
			
		||||
	log.Info("new collection created", zap.Int64("collectionID", collectionID), zap.Any("schema", schema), zap.Error(err))
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -447,6 +447,18 @@ func (t *createCollectionTask) Prepare(ctx context.Context) error {
 | 
			
		|||
	}
 | 
			
		||||
	t.dbProperties = db.Properties
 | 
			
		||||
 | 
			
		||||
	// set collection timezone
 | 
			
		||||
	properties := t.Req.GetProperties()
 | 
			
		||||
	ok, _ := getDefaultTimezoneVal(properties...)
 | 
			
		||||
	if !ok {
 | 
			
		||||
		ok, defaultTz := getDefaultTimezoneVal(db.Properties...)
 | 
			
		||||
		if !ok {
 | 
			
		||||
			defaultTz = "UTC"
 | 
			
		||||
		}
 | 
			
		||||
		timezoneKV := &commonpb.KeyValuePair{Key: common.CollectionDefaultTimezone, Value: defaultTz}
 | 
			
		||||
		t.Req.Properties = append(properties, timezoneKV)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if hookutil.GetEzPropByDBProperties(t.dbProperties) != nil {
 | 
			
		||||
		t.Req.Properties = append(t.Req.Properties, hookutil.GetEzPropByDBProperties(t.dbProperties))
 | 
			
		||||
	}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1228,7 +1228,7 @@ func TestCreateCollectionTask_Prepare_WithProperty(t *testing.T) {
 | 
			
		|||
		task.Req.ShardsNum = common.DefaultShardsNum
 | 
			
		||||
		err = task.Prepare(context.Background())
 | 
			
		||||
		assert.Len(t, task.dbProperties, 1)
 | 
			
		||||
		assert.Len(t, task.Req.Properties, 0)
 | 
			
		||||
		assert.Len(t, task.Req.Properties, 1)
 | 
			
		||||
		assert.NoError(t, err)
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -565,3 +565,13 @@ func nextFieldID(coll *model.Collection) int64 {
 | 
			
		|||
	}
 | 
			
		||||
	return maxFieldID + 1
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func getDefaultTimezoneVal(props ...*commonpb.KeyValuePair) (bool, string) {
 | 
			
		||||
	for _, p := range props {
 | 
			
		||||
		// used in collection or database
 | 
			
		||||
		if p.GetKey() == common.DatabaseDefaultTimezone || p.GetKey() == common.CollectionDefaultTimezone {
 | 
			
		||||
			return true, p.Value
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return false, ""
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -517,7 +517,6 @@ func RowBasedInsertMsgToInsertData(msg *msgstream.InsertMsg, collSchema *schemap
 | 
			
		|||
			idata.Data[field.FieldID] = &DoubleFieldData{
 | 
			
		||||
				Data: readDoubleArray(blobReaders),
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
		case schemapb.DataType_Timestamptz:
 | 
			
		||||
			idata.Data[field.FieldID] = &TimestamptzFieldData{
 | 
			
		||||
				Data: readTimestamptzArray(blobReaders),
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -240,7 +240,7 @@ func convertNumpyType(typeStr string) (schemapb.DataType, error) {
 | 
			
		|||
		return schemapb.DataType_Int16, nil
 | 
			
		||||
	case "i4", "<i4", "|i4", ">i4", "int32":
 | 
			
		||||
		return schemapb.DataType_Int32, nil
 | 
			
		||||
	case "i8", "<i8", "|i8", ">i8", "int64": // todo: how to handle Timestamptz?
 | 
			
		||||
	case "i8", "<i8", "|i8", ">i8", "int64":
 | 
			
		||||
		return schemapb.DataType_Int64, nil
 | 
			
		||||
	case "f4", "<f4", "|f4", ">f4", "float32":
 | 
			
		||||
		return schemapb.DataType_Float, nil
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -235,6 +235,10 @@ const (
 | 
			
		|||
	IndexNonEncoding           = "index.nonEncoding"
 | 
			
		||||
	EnableDynamicSchemaKey     = `dynamicfield.enabled`
 | 
			
		||||
	NamespaceEnabledKey        = "namespace.enabled"
 | 
			
		||||
 | 
			
		||||
	// timezone releated
 | 
			
		||||
	DatabaseDefaultTimezone   = "database.timezone"
 | 
			
		||||
	CollectionDefaultTimezone = "collection.timezone"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
const (
 | 
			
		||||
| 
						 | 
				
			
			@ -243,6 +247,17 @@ const (
 | 
			
		|||
	ClientRequestMsecKey string = "client-request-unixmsec"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// Timestamptz field
 | 
			
		||||
const (
 | 
			
		||||
	TszYear        string = "year"
 | 
			
		||||
	TszMonth       string = "month"
 | 
			
		||||
	TszDay         string = "day"
 | 
			
		||||
	TszHour        string = "hour"
 | 
			
		||||
	TszMinute      string = "minute"
 | 
			
		||||
	TszSecond      string = "second"
 | 
			
		||||
	TszMicrosecond string = "microsecond"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func IsSystemField(fieldID int64) bool {
 | 
			
		||||
	return fieldID < StartOfUserFieldID
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -224,6 +224,24 @@ message RandomSampleExpr {
 | 
			
		|||
 | 
			
		||||
message AlwaysTrueExpr {}
 | 
			
		||||
 | 
			
		||||
message Interval {
 | 
			
		||||
  int64 years = 1;
 | 
			
		||||
  int64 months = 2;
 | 
			
		||||
  int64 days = 3;
 | 
			
		||||
  int64 hours = 4;
 | 
			
		||||
  int64 minutes = 5;
 | 
			
		||||
  int64 seconds = 6;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// New expression type for the operation: (timestamp_col + interval) OP iso_string
 | 
			
		||||
message TimestamptzArithCompareExpr {
 | 
			
		||||
  ColumnInfo timestamptz_column = 1;
 | 
			
		||||
  ArithOpType arith_op = 2; // ADD or SUB
 | 
			
		||||
  Interval interval = 3;
 | 
			
		||||
  OpType compare_op = 4;
 | 
			
		||||
  GenericValue compare_value = 5;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
message Expr {
 | 
			
		||||
  oneof expr {
 | 
			
		||||
    TermExpr term_expr = 1;
 | 
			
		||||
| 
						 | 
				
			
			@ -242,6 +260,7 @@ message Expr {
 | 
			
		|||
    CallExpr call_expr = 14;
 | 
			
		||||
    NullExpr null_expr = 15;
 | 
			
		||||
    RandomSampleExpr random_sample_expr = 16;
 | 
			
		||||
    TimestamptzArithCompareExpr timestamptz_arith_compare_expr = 17;
 | 
			
		||||
  };
 | 
			
		||||
  bool is_template = 20;
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							| 
						 | 
				
			
			@ -596,7 +596,8 @@ func IsVectorArrayType(dataType schemapb.DataType) bool {
 | 
			
		|||
func IsIntegerType(dataType schemapb.DataType) bool {
 | 
			
		||||
	switch dataType {
 | 
			
		||||
	case schemapb.DataType_Int8, schemapb.DataType_Int16,
 | 
			
		||||
		schemapb.DataType_Int32, schemapb.DataType_Int64:
 | 
			
		||||
		schemapb.DataType_Int32, schemapb.DataType_Int64,
 | 
			
		||||
		schemapb.DataType_Timestamptz:
 | 
			
		||||
		return true
 | 
			
		||||
	default:
 | 
			
		||||
		return false
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1136,7 +1136,7 @@ func TestCollectionPropertyTtl(t *testing.T) {
 | 
			
		|||
	err := mc.AlterCollectionProperties(ctx, client.NewAlterCollectionPropertiesOption(schema.CollectionName).WithProperty(common.CollectionTTLSeconds, 2))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ := mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(schema.CollectionName))
 | 
			
		||||
	require.Equal(t, map[string]string{common.CollectionTTLSeconds: "2"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{common.CollectionTTLSeconds: "2"})
 | 
			
		||||
 | 
			
		||||
	time.Sleep(5 * time.Second)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -1147,7 +1147,7 @@ func TestCollectionPropertyTtl(t *testing.T) {
 | 
			
		|||
	err = mc.DropCollectionProperties(ctx, client.NewDropCollectionPropertiesOption(schema.CollectionName, common.CollectionTTLSeconds))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ = mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(schema.CollectionName))
 | 
			
		||||
	require.Equal(t, map[string]string{}, coll.Properties)
 | 
			
		||||
	require.Equal(t, 1, len(coll.Properties))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// create collection with property -> alter property -> writing and reading
 | 
			
		||||
| 
						 | 
				
			
			@ -1161,14 +1161,14 @@ func TestCollectionWithPropertyAlterMmap(t *testing.T) {
 | 
			
		|||
		hp.TNewFieldsOption(), hp.TNewSchemaOption(), hp.TWithProperties(map[string]any{common.MmapEnabled: false}))
 | 
			
		||||
 | 
			
		||||
	coll, _ := mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(schema.CollectionName))
 | 
			
		||||
	require.Equal(t, map[string]string{common.MmapEnabled: "false"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{common.MmapEnabled: "false"})
 | 
			
		||||
	log.Info("TestCollectionPropertyMmap.DescribeCollection", zap.Any("properties", coll.Properties))
 | 
			
		||||
 | 
			
		||||
	// alter properties
 | 
			
		||||
	err := mc.AlterCollectionProperties(ctx, client.NewAlterCollectionPropertiesOption(schema.CollectionName).WithProperty(common.MmapEnabled, true))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ = mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(schema.CollectionName))
 | 
			
		||||
	require.Equal(t, map[string]string{common.MmapEnabled: "true"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{common.MmapEnabled: "true"})
 | 
			
		||||
 | 
			
		||||
	// writing and reading
 | 
			
		||||
	prepare.InsertData(ctx, t, mc, hp.NewInsertParams(schema), hp.TNewDataOption())
 | 
			
		||||
| 
						 | 
				
			
			@ -1193,7 +1193,7 @@ func TestCollectionPropertyMmap(t *testing.T) {
 | 
			
		|||
	err := mc.AlterCollectionProperties(ctx, client.NewAlterCollectionPropertiesOption(schema.CollectionName).WithProperty(common.MmapEnabled, true))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ := mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(schema.CollectionName))
 | 
			
		||||
	require.Equal(t, map[string]string{common.MmapEnabled: "true"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{common.MmapEnabled: "true"})
 | 
			
		||||
 | 
			
		||||
	// writing and reading
 | 
			
		||||
	prepare.InsertData(ctx, t, mc, hp.NewInsertParams(schema), hp.TNewDataOption())
 | 
			
		||||
| 
						 | 
				
			
			@ -1212,7 +1212,7 @@ func TestCollectionPropertyMmap(t *testing.T) {
 | 
			
		|||
	err = mc.DropCollectionProperties(ctx, client.NewDropCollectionPropertiesOption(schema.CollectionName, common.MmapEnabled))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ = mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(schema.CollectionName))
 | 
			
		||||
	require.Equal(t, map[string]string{}, coll.Properties)
 | 
			
		||||
	require.Equal(t, 1, len(coll.Properties))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestCollectionFakeProperties(t *testing.T) {
 | 
			
		||||
| 
						 | 
				
			
			@ -1225,16 +1225,16 @@ func TestCollectionFakeProperties(t *testing.T) {
 | 
			
		|||
	err := mc.CreateCollection(ctx, client.SimpleCreateCollectionOptions(collName, common.DefaultDim).WithProperty("1", "bbb"))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ := mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(collName))
 | 
			
		||||
	require.Equal(t, map[string]string{"1": "bbb"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{"1": "bbb"})
 | 
			
		||||
 | 
			
		||||
	// alter collection with fake property
 | 
			
		||||
	// alter collection with fake kjproperty
 | 
			
		||||
	err = mc.AlterCollectionProperties(ctx, client.NewAlterCollectionPropertiesOption(collName).WithProperty("2", 1))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ = mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(collName))
 | 
			
		||||
	require.Equal(t, map[string]string{"1": "bbb", "2": "1"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{"1": "bbb", "2": "1"})
 | 
			
		||||
 | 
			
		||||
	err = mc.DropCollectionProperties(ctx, client.NewDropCollectionPropertiesOption(collName, "ccc"))
 | 
			
		||||
	common.CheckErr(t, err, true)
 | 
			
		||||
	coll, _ = mc.DescribeCollection(ctx, client.NewDescribeCollectionOption(collName))
 | 
			
		||||
	require.Equal(t, map[string]string{"1": "bbb", "2": "1"}, coll.Properties)
 | 
			
		||||
	require.Subset(t, coll.Properties, map[string]string{"1": "bbb", "2": "1"})
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -145,7 +145,7 @@ class TestMilvusClientAlterCollection(TestMilvusClientV2Base):
 | 
			
		|||
        self.create_collection(client, collection_name, ct.default_dim, consistency_level="Strong")
 | 
			
		||||
        self.load_collection(client, collection_name)
 | 
			
		||||
        res1 = self.describe_collection(client, collection_name)[0]
 | 
			
		||||
        assert res1.get('properties', None) == {}
 | 
			
		||||
        assert len(res1.get('properties', {})) == 1
 | 
			
		||||
        # 1. alter collection properties after load
 | 
			
		||||
        self.load_collection(client, collection_name)
 | 
			
		||||
        error = {ct.err_code: 999,
 | 
			
		||||
| 
						 | 
				
			
			@ -170,24 +170,24 @@ class TestMilvusClientAlterCollection(TestMilvusClientV2Base):
 | 
			
		|||
        # self.drop_collection_properties(client, collection_name, property_keys=["dynamicfield.enabled"],
 | 
			
		||||
        #                                 check_task=CheckTasks.err_res, check_items=error)
 | 
			
		||||
        res3 = self.describe_collection(client, collection_name)[0]
 | 
			
		||||
        assert res3.get('properties', None) == {}
 | 
			
		||||
        assert len(res1.get('properties', {})) == 1
 | 
			
		||||
        self.drop_collection_properties(client, collection_name, property_keys=["collection.ttl.seconds"])
 | 
			
		||||
        assert res3.get('properties', None) == {}
 | 
			
		||||
        assert len(res1.get('properties', {})) == 1
 | 
			
		||||
        # 2. alter collection properties after release
 | 
			
		||||
        self.release_collection(client, collection_name)
 | 
			
		||||
        self.alter_collection_properties(client, collection_name, properties={"mmap.enabled": True})
 | 
			
		||||
        res2 = self.describe_collection(client, collection_name)[0]
 | 
			
		||||
        assert res2.get('properties', None) == {'mmap.enabled': 'True'}
 | 
			
		||||
        assert {'mmap.enabled': 'True'}.items() <= res2.get('properties', {}).items()
 | 
			
		||||
        self.alter_collection_properties(client, collection_name,
 | 
			
		||||
                                         properties={"collection.ttl.seconds": 100, "lazyload.enabled": True})
 | 
			
		||||
        res2 = self.describe_collection(client, collection_name)[0]
 | 
			
		||||
        assert res2.get('properties', None) == {'mmap.enabled': 'True',
 | 
			
		||||
                                                'collection.ttl.seconds': '100', 'lazyload.enabled': 'True'}
 | 
			
		||||
        assert {'mmap.enabled': 'True', 'collection.ttl.seconds': '100', 'lazyload.enabled': 'True'}.items()  \
 | 
			
		||||
                <= res2.get('properties', {}).items()
 | 
			
		||||
        self.drop_collection_properties(client, collection_name,
 | 
			
		||||
                                        property_keys=["mmap.enabled", "lazyload.enabled",
 | 
			
		||||
                                                       "collection.ttl.seconds"])
 | 
			
		||||
        res3 = self.describe_collection(client, collection_name)[0]
 | 
			
		||||
        assert res3.get('properties', None) == {}
 | 
			
		||||
        assert len(res1.get('properties', {})) == 1
 | 
			
		||||
 | 
			
		||||
    @pytest.mark.tags(CaseLabel.L1)
 | 
			
		||||
    def test_milvus_client_alter_enable_dynamic_collection_field(self):
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue