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))
|
||||
|
|
|
|||
|
|
@ -86,6 +86,8 @@ type queryParams struct {
|
|||
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,6 +230,18 @@ 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)
|
||||
|
|
@ -237,6 +253,8 @@ func parseQueryParams(queryParamsPair []*commonpb.KeyValuePair) (*queryParams, e
|
|||
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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -2037,6 +2037,173 @@ func (*AlwaysTrueExpr) Descriptor() ([]byte, []int) {
|
|||
return file_plan_proto_rawDescGZIP(), []int{21}
|
||||
}
|
||||
|
||||
type Interval struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Years int64 `protobuf:"varint,1,opt,name=years,proto3" json:"years,omitempty"`
|
||||
Months int64 `protobuf:"varint,2,opt,name=months,proto3" json:"months,omitempty"`
|
||||
Days int64 `protobuf:"varint,3,opt,name=days,proto3" json:"days,omitempty"`
|
||||
Hours int64 `protobuf:"varint,4,opt,name=hours,proto3" json:"hours,omitempty"`
|
||||
Minutes int64 `protobuf:"varint,5,opt,name=minutes,proto3" json:"minutes,omitempty"`
|
||||
Seconds int64 `protobuf:"varint,6,opt,name=seconds,proto3" json:"seconds,omitempty"`
|
||||
}
|
||||
|
||||
func (x *Interval) Reset() {
|
||||
*x = Interval{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[22]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *Interval) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*Interval) ProtoMessage() {}
|
||||
|
||||
func (x *Interval) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[22]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use Interval.ProtoReflect.Descriptor instead.
|
||||
func (*Interval) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{22}
|
||||
}
|
||||
|
||||
func (x *Interval) GetYears() int64 {
|
||||
if x != nil {
|
||||
return x.Years
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *Interval) GetMonths() int64 {
|
||||
if x != nil {
|
||||
return x.Months
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *Interval) GetDays() int64 {
|
||||
if x != nil {
|
||||
return x.Days
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *Interval) GetHours() int64 {
|
||||
if x != nil {
|
||||
return x.Hours
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *Interval) GetMinutes() int64 {
|
||||
if x != nil {
|
||||
return x.Minutes
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *Interval) GetSeconds() int64 {
|
||||
if x != nil {
|
||||
return x.Seconds
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
// New expression type for the operation: (timestamp_col + interval) OP iso_string
|
||||
type TimestamptzArithCompareExpr struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
TimestamptzColumn *ColumnInfo `protobuf:"bytes,1,opt,name=timestamptz_column,json=timestamptzColumn,proto3" json:"timestamptz_column,omitempty"`
|
||||
ArithOp ArithOpType `protobuf:"varint,2,opt,name=arith_op,json=arithOp,proto3,enum=milvus.proto.plan.ArithOpType" json:"arith_op,omitempty"` // ADD or SUB
|
||||
Interval *Interval `protobuf:"bytes,3,opt,name=interval,proto3" json:"interval,omitempty"`
|
||||
CompareOp OpType `protobuf:"varint,4,opt,name=compare_op,json=compareOp,proto3,enum=milvus.proto.plan.OpType" json:"compare_op,omitempty"`
|
||||
CompareValue *GenericValue `protobuf:"bytes,5,opt,name=compare_value,json=compareValue,proto3" json:"compare_value,omitempty"`
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) Reset() {
|
||||
*x = TimestamptzArithCompareExpr{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[23]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*TimestamptzArithCompareExpr) ProtoMessage() {}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[23]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use TimestamptzArithCompareExpr.ProtoReflect.Descriptor instead.
|
||||
func (*TimestamptzArithCompareExpr) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{23}
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) GetTimestamptzColumn() *ColumnInfo {
|
||||
if x != nil {
|
||||
return x.TimestamptzColumn
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) GetArithOp() ArithOpType {
|
||||
if x != nil {
|
||||
return x.ArithOp
|
||||
}
|
||||
return ArithOpType_Unknown
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) GetInterval() *Interval {
|
||||
if x != nil {
|
||||
return x.Interval
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) GetCompareOp() OpType {
|
||||
if x != nil {
|
||||
return x.CompareOp
|
||||
}
|
||||
return OpType_Invalid
|
||||
}
|
||||
|
||||
func (x *TimestamptzArithCompareExpr) GetCompareValue() *GenericValue {
|
||||
if x != nil {
|
||||
return x.CompareValue
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type Expr struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
|
|
@ -2060,6 +2227,7 @@ type Expr struct {
|
|||
// *Expr_CallExpr
|
||||
// *Expr_NullExpr
|
||||
// *Expr_RandomSampleExpr
|
||||
// *Expr_TimestamptzArithCompareExpr
|
||||
Expr isExpr_Expr `protobuf_oneof:"expr"`
|
||||
IsTemplate bool `protobuf:"varint,20,opt,name=is_template,json=isTemplate,proto3" json:"is_template,omitempty"`
|
||||
}
|
||||
|
|
@ -2067,7 +2235,7 @@ type Expr struct {
|
|||
func (x *Expr) Reset() {
|
||||
*x = Expr{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[22]
|
||||
mi := &file_plan_proto_msgTypes[24]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
|
@ -2080,7 +2248,7 @@ func (x *Expr) String() string {
|
|||
func (*Expr) ProtoMessage() {}
|
||||
|
||||
func (x *Expr) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[22]
|
||||
mi := &file_plan_proto_msgTypes[24]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
|
@ -2093,7 +2261,7 @@ func (x *Expr) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use Expr.ProtoReflect.Descriptor instead.
|
||||
func (*Expr) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{22}
|
||||
return file_plan_proto_rawDescGZIP(), []int{24}
|
||||
}
|
||||
|
||||
func (m *Expr) GetExpr() isExpr_Expr {
|
||||
|
|
@ -2215,6 +2383,13 @@ func (x *Expr) GetRandomSampleExpr() *RandomSampleExpr {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (x *Expr) GetTimestamptzArithCompareExpr() *TimestamptzArithCompareExpr {
|
||||
if x, ok := x.GetExpr().(*Expr_TimestamptzArithCompareExpr); ok {
|
||||
return x.TimestamptzArithCompareExpr
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *Expr) GetIsTemplate() bool {
|
||||
if x != nil {
|
||||
return x.IsTemplate
|
||||
|
|
@ -2290,6 +2465,10 @@ type Expr_RandomSampleExpr struct {
|
|||
RandomSampleExpr *RandomSampleExpr `protobuf:"bytes,16,opt,name=random_sample_expr,json=randomSampleExpr,proto3,oneof"`
|
||||
}
|
||||
|
||||
type Expr_TimestamptzArithCompareExpr struct {
|
||||
TimestamptzArithCompareExpr *TimestamptzArithCompareExpr `protobuf:"bytes,17,opt,name=timestamptz_arith_compare_expr,json=timestamptzArithCompareExpr,proto3,oneof"`
|
||||
}
|
||||
|
||||
func (*Expr_TermExpr) isExpr_Expr() {}
|
||||
|
||||
func (*Expr_UnaryExpr) isExpr_Expr() {}
|
||||
|
|
@ -2322,6 +2501,8 @@ func (*Expr_NullExpr) isExpr_Expr() {}
|
|||
|
||||
func (*Expr_RandomSampleExpr) isExpr_Expr() {}
|
||||
|
||||
func (*Expr_TimestamptzArithCompareExpr) isExpr_Expr() {}
|
||||
|
||||
type VectorANNS struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
|
|
@ -2337,7 +2518,7 @@ type VectorANNS struct {
|
|||
func (x *VectorANNS) Reset() {
|
||||
*x = VectorANNS{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[23]
|
||||
mi := &file_plan_proto_msgTypes[25]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
|
@ -2350,7 +2531,7 @@ func (x *VectorANNS) String() string {
|
|||
func (*VectorANNS) ProtoMessage() {}
|
||||
|
||||
func (x *VectorANNS) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[23]
|
||||
mi := &file_plan_proto_msgTypes[25]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
|
@ -2363,7 +2544,7 @@ func (x *VectorANNS) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use VectorANNS.ProtoReflect.Descriptor instead.
|
||||
func (*VectorANNS) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{23}
|
||||
return file_plan_proto_rawDescGZIP(), []int{25}
|
||||
}
|
||||
|
||||
func (x *VectorANNS) GetVectorType() VectorType {
|
||||
|
|
@ -2414,7 +2595,7 @@ type QueryPlanNode struct {
|
|||
func (x *QueryPlanNode) Reset() {
|
||||
*x = QueryPlanNode{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[24]
|
||||
mi := &file_plan_proto_msgTypes[26]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
|
@ -2427,7 +2608,7 @@ func (x *QueryPlanNode) String() string {
|
|||
func (*QueryPlanNode) ProtoMessage() {}
|
||||
|
||||
func (x *QueryPlanNode) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[24]
|
||||
mi := &file_plan_proto_msgTypes[26]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
|
@ -2440,7 +2621,7 @@ func (x *QueryPlanNode) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use QueryPlanNode.ProtoReflect.Descriptor instead.
|
||||
func (*QueryPlanNode) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{24}
|
||||
return file_plan_proto_rawDescGZIP(), []int{26}
|
||||
}
|
||||
|
||||
func (x *QueryPlanNode) GetPredicates() *Expr {
|
||||
|
|
@ -2476,7 +2657,7 @@ type ScoreFunction struct {
|
|||
func (x *ScoreFunction) Reset() {
|
||||
*x = ScoreFunction{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[25]
|
||||
mi := &file_plan_proto_msgTypes[27]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
|
@ -2489,7 +2670,7 @@ func (x *ScoreFunction) String() string {
|
|||
func (*ScoreFunction) ProtoMessage() {}
|
||||
|
||||
func (x *ScoreFunction) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[25]
|
||||
mi := &file_plan_proto_msgTypes[27]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
|
@ -2502,7 +2683,7 @@ func (x *ScoreFunction) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use ScoreFunction.ProtoReflect.Descriptor instead.
|
||||
func (*ScoreFunction) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{25}
|
||||
return file_plan_proto_rawDescGZIP(), []int{27}
|
||||
}
|
||||
|
||||
func (x *ScoreFunction) GetFilter() *Expr {
|
||||
|
|
@ -2530,7 +2711,7 @@ type PlanOption struct {
|
|||
func (x *PlanOption) Reset() {
|
||||
*x = PlanOption{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[26]
|
||||
mi := &file_plan_proto_msgTypes[28]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
|
@ -2543,7 +2724,7 @@ func (x *PlanOption) String() string {
|
|||
func (*PlanOption) ProtoMessage() {}
|
||||
|
||||
func (x *PlanOption) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[26]
|
||||
mi := &file_plan_proto_msgTypes[28]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
|
@ -2556,7 +2737,7 @@ func (x *PlanOption) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use PlanOption.ProtoReflect.Descriptor instead.
|
||||
func (*PlanOption) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{26}
|
||||
return file_plan_proto_rawDescGZIP(), []int{28}
|
||||
}
|
||||
|
||||
func (x *PlanOption) GetExprUseJsonStats() bool {
|
||||
|
|
@ -2586,7 +2767,7 @@ type PlanNode struct {
|
|||
func (x *PlanNode) Reset() {
|
||||
*x = PlanNode{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_plan_proto_msgTypes[27]
|
||||
mi := &file_plan_proto_msgTypes[29]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
|
|
@ -2599,7 +2780,7 @@ func (x *PlanNode) String() string {
|
|||
func (*PlanNode) ProtoMessage() {}
|
||||
|
||||
func (x *PlanNode) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_plan_proto_msgTypes[27]
|
||||
mi := &file_plan_proto_msgTypes[29]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
|
|
@ -2612,7 +2793,7 @@ func (x *PlanNode) ProtoReflect() protoreflect.Message {
|
|||
|
||||
// Deprecated: Use PlanNode.ProtoReflect.Descriptor instead.
|
||||
func (*PlanNode) Descriptor() ([]byte, []int) {
|
||||
return file_plan_proto_rawDescGZIP(), []int{27}
|
||||
return file_plan_proto_rawDescGZIP(), []int{29}
|
||||
}
|
||||
|
||||
func (m *PlanNode) GetNode() isPlanNode_Node {
|
||||
|
|
@ -3008,182 +3189,221 @@ var file_plan_proto_rawDesc = []byte{
|
|||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c,
|
||||
0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x09, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61,
|
||||
0x74, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75, 0x65,
|
||||
0x45, 0x78, 0x70, 0x72, 0x22, 0xce, 0x09, 0x0a, 0x04, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a,
|
||||
0x09, 0x74, 0x65, 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52,
|
||||
0x08, 0x74, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x75, 0x6e, 0x61,
|
||||
0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e,
|
||||
0x45, 0x78, 0x70, 0x72, 0x22, 0x96, 0x01, 0x0a, 0x08, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61,
|
||||
0x6c, 0x12, 0x14, 0x0a, 0x05, 0x79, 0x65, 0x61, 0x72, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
|
||||
0x52, 0x05, 0x79, 0x65, 0x61, 0x72, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x6f, 0x6e, 0x74, 0x68,
|
||||
0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6d, 0x6f, 0x6e, 0x74, 0x68, 0x73, 0x12,
|
||||
0x12, 0x0a, 0x04, 0x64, 0x61, 0x79, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64,
|
||||
0x61, 0x79, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x68, 0x6f, 0x75, 0x72, 0x73, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x05, 0x68, 0x6f, 0x75, 0x72, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x69, 0x6e,
|
||||
0x75, 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x69, 0x6e, 0x75,
|
||||
0x74, 0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x06,
|
||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xdf, 0x02,
|
||||
0x0a, 0x1b, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x74, 0x7a, 0x41, 0x72, 0x69,
|
||||
0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4c, 0x0a,
|
||||
0x12, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x74, 0x7a, 0x5f, 0x63, 0x6f, 0x6c,
|
||||
0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f,
|
||||
0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x11, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74,
|
||||
0x61, 0x6d, 0x70, 0x74, 0x7a, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x39, 0x0a, 0x08, 0x61,
|
||||
0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61,
|
||||
0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x09, 0x75,
|
||||
0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61,
|
||||
0x72, 0x79, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e,
|
||||
0x6e, 0x2e, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x61,
|
||||
0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x12, 0x37, 0x0a, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76,
|
||||
0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x49, 0x6e, 0x74,
|
||||
0x65, 0x72, 0x76, 0x61, 0x6c, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12,
|
||||
0x38, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x18, 0x04, 0x20,
|
||||
0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09,
|
||||
0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x4f, 0x70, 0x12, 0x44, 0x0a, 0x0d, 0x63, 0x6f, 0x6d,
|
||||
0x70, 0x61, 0x72, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22,
|
||||
0xc5, 0x0a, 0x0a, 0x04, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x74, 0x65, 0x72, 0x6d,
|
||||
0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e,
|
||||
0x54, 0x65, 0x72, 0x6d, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x08, 0x74, 0x65, 0x72, 0x6d,
|
||||
0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78,
|
||||
0x70, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61,
|
||||
0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x09, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x45,
|
||||
0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78,
|
||||
0x70, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e,
|
||||
0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72,
|
||||
0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x43, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65,
|
||||
0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e,
|
||||
0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0b, 0x63,
|
||||
0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x75, 0x6e,
|
||||
0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x05,
|
||||
0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61,
|
||||
0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e, 0x75, 0x6e, 0x61, 0x72, 0x79,
|
||||
0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e,
|
||||
0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x06,
|
||||
0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52,
|
||||
0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61,
|
||||
0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x74, 0x0a, 0x1f, 0x62,
|
||||
0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x6f, 0x70, 0x5f, 0x65,
|
||||
0x76, 0x61, 0x6c, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x07,
|
||||
0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41,
|
||||
0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45,
|
||||
0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x1a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69,
|
||||
0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70,
|
||||
0x72, 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, 0x74,
|
||||
0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e,
|
||||
0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72,
|
||||
0x48, 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x45,
|
||||
0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70,
|
||||
0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78,
|
||||
0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x65, 0x78, 0x70,
|
||||
0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6c, 0x75,
|
||||
0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,
|
||||
0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x65,
|
||||
0x78, 0x70, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78,
|
||||
0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x65, 0x78, 0x69, 0x73,
|
||||
0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73,
|
||||
0x5f, 0x74, 0x72, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45,
|
||||
0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x54, 0x72, 0x75,
|
||||
0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x53, 0x0a, 0x12, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x63, 0x6f,
|
||||
0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69,
|
||||
0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x10, 0x6a, 0x73, 0x6f, 0x6e, 0x43, 0x6f,
|
||||
0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x63, 0x61,
|
||||
0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e,
|
||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61,
|
||||
0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a,
|
||||
0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x12, 0x43, 0x0a, 0x0c, 0x63, 0x6f,
|
||||
0x6d, 0x70, 0x61, 0x72, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
|
||||
0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72,
|
||||
0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12,
|
||||
0x4d, 0x0a, 0x10, 0x75, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65,
|
||||
0x78, 0x70, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x55, 0x6e,
|
||||
0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e,
|
||||
0x75, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x50,
|
||||
0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65,
|
||||
0x78, 0x70, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69,
|
||||
0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52,
|
||||
0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72,
|
||||
0x12, 0x74, 0x0a, 0x1f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68,
|
||||
0x5f, 0x6f, 0x70, 0x5f, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65,
|
||||
0x78, 0x70, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69,
|
||||
0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52,
|
||||
0x61, 0x6e, 0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x1a, 0x62, 0x69, 0x6e, 0x61,
|
||||
0x72, 0x79, 0x41, 0x72, 0x69, 0x74, 0x68, 0x4f, 0x70, 0x45, 0x76, 0x61, 0x6c, 0x52, 0x61, 0x6e,
|
||||
0x67, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x50, 0x0a, 0x11, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79,
|
||||
0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41, 0x72, 0x69, 0x74,
|
||||
0x68, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x41,
|
||||
0x72, 0x69, 0x74, 0x68, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3d, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e,
|
||||
0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x09, 0x76, 0x61,
|
||||
0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d,
|
||||
0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e,
|
||||
0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x63,
|
||||
0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x12, 0x40, 0x0a, 0x0b, 0x65, 0x78, 0x69,
|
||||
0x73, 0x74, 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d,
|
||||
0x6e, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x08, 0x63, 0x61,
|
||||
0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x65,
|
||||
0x78, 0x70, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4e, 0x75,
|
||||
0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x45, 0x78,
|
||||
0x70, 0x72, 0x12, 0x53, 0x0a, 0x12, 0x72, 0x61, 0x6e, 0x64, 0x6f, 0x6d, 0x5f, 0x73, 0x61, 0x6d,
|
||||
0x70, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c,
|
||||
0x61, 0x6e, 0x2e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52,
|
||||
0x0a, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12, 0x4d, 0x0a, 0x10, 0x61,
|
||||
0x6c, 0x77, 0x61, 0x79, 0x73, 0x5f, 0x74, 0x72, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18,
|
||||
0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73,
|
||||
0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0e, 0x61, 0x6c, 0x77, 0x61,
|
||||
0x79, 0x73, 0x54, 0x72, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x53, 0x0a, 0x12, 0x6a, 0x73,
|
||||
0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x65, 0x78, 0x70, 0x72,
|
||||
0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x4a, 0x53, 0x4f, 0x4e, 0x43,
|
||||
0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x10, 0x6a,
|
||||
0x73, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x45, 0x78, 0x70, 0x72, 0x12,
|
||||
0x3a, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0e, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48,
|
||||
0x00, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x3a, 0x0a, 0x09, 0x6e,
|
||||
0x75, 0x6c, 0x6c, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b,
|
||||
0x61, 0x6e, 0x2e, 0x52, 0x61, 0x6e, 0x64, 0x6f, 0x6d, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x45,
|
||||
0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x10, 0x72, 0x61, 0x6e, 0x64, 0x6f, 0x6d, 0x53, 0x61, 0x6d,
|
||||
0x70, 0x6c, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x75, 0x0a, 0x1e, 0x74, 0x69, 0x6d, 0x65, 0x73,
|
||||
0x74, 0x61, 0x6d, 0x70, 0x74, 0x7a, 0x5f, 0x61, 0x72, 0x69, 0x74, 0x68, 0x5f, 0x63, 0x6f, 0x6d,
|
||||
0x70, 0x61, 0x72, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
||||
0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70,
|
||||
0x6c, 0x61, 0x6e, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x74, 0x7a, 0x41,
|
||||
0x72, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48,
|
||||
0x00, 0x52, 0x1b, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x74, 0x7a, 0x41, 0x72,
|
||||
0x69, 0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x1f,
|
||||
0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x14, 0x20,
|
||||
0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x42,
|
||||
0x06, 0x0a, 0x04, 0x65, 0x78, 0x70, 0x72, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x41, 0x4e, 0x4e, 0x53, 0x12, 0x3e, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72,
|
||||
0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e,
|
||||
0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f,
|
||||
0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49,
|
||||
0x64, 0x12, 0x37, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18,
|
||||
0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a,
|
||||
0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x0a, 0x71, 0x75,
|
||||
0x65, 0x72, 0x79, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c,
|
||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c,
|
||||
0x61, 0x6e, 0x2e, 0x4e, 0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x08, 0x6e,
|
||||
0x75, 0x6c, 0x6c, 0x45, 0x78, 0x70, 0x72, 0x12, 0x53, 0x0a, 0x12, 0x72, 0x61, 0x6e, 0x64, 0x6f,
|
||||
0x6d, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x10, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x52, 0x61, 0x6e, 0x64, 0x6f, 0x6d, 0x53, 0x61,
|
||||
0x6d, 0x70, 0x6c, 0x65, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x10, 0x72, 0x61, 0x6e, 0x64,
|
||||
0x6f, 0x6d, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x45, 0x78, 0x70, 0x72, 0x12, 0x1f, 0x0a, 0x0b,
|
||||
0x69, 0x73, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28,
|
||||
0x08, 0x52, 0x0a, 0x69, 0x73, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x42, 0x06, 0x0a,
|
||||
0x04, 0x65, 0x78, 0x70, 0x72, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72,
|
||||
0x41, 0x4e, 0x4e, 0x53, 0x12, 0x3e, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x74,
|
||||
0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65,
|
||||
0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72,
|
||||
0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x12,
|
||||
0x37, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, 0x72,
|
||||
0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x0a, 0x71, 0x75, 0x65, 0x72,
|
||||
0x79, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d,
|
||||
0x61, 0x6e, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x71, 0x75,
|
||||
0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6c, 0x61, 0x63, 0x65,
|
||||
0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x0e, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x54, 0x61, 0x67,
|
||||
0x22, 0x79, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64,
|
||||
0x65, 0x12, 0x37, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a,
|
||||
0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73,
|
||||
0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73,
|
||||
0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0x58, 0x0a, 0x0d, 0x53,
|
||||
0x63, 0x6f, 0x72, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x06,
|
||||
0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e,
|
||||
0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x71, 0x75, 0x65, 0x72,
|
||||
0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f,
|
||||
0x6c, 0x64, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,
|
||||
0x70, 0x6c, 0x61, 0x63, 0x65, 0x68, 0x6f, 0x6c, 0x64, 0x65, 0x72, 0x54, 0x61, 0x67, 0x22, 0x79,
|
||||
0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x12,
|
||||
0x37, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x0a, 0x70, 0x72,
|
||||
0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63,
|
||||
0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, 0x6f,
|
||||
0x75, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0x58, 0x0a, 0x0d, 0x53, 0x63, 0x6f,
|
||||
0x72, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x06, 0x66, 0x69,
|
||||
0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c,
|
||||
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45,
|
||||
0x78, 0x70, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x77,
|
||||
0x65, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x06, 0x77, 0x65, 0x69,
|
||||
0x67, 0x68, 0x74, 0x22, 0x3b, 0x0a, 0x0a, 0x50, 0x6c, 0x61, 0x6e, 0x4f, 0x70, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x12, 0x2d, 0x0a, 0x13, 0x65, 0x78, 0x70, 0x72, 0x5f, 0x75, 0x73, 0x65, 0x5f, 0x6a, 0x73,
|
||||
0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10,
|
||||
0x65, 0x78, 0x70, 0x72, 0x55, 0x73, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73,
|
||||
0x22, 0x98, 0x03, 0x0a, 0x08, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x40, 0x0a,
|
||||
0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x6e, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x4e, 0x4e,
|
||||
0x53, 0x48, 0x00, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x6e, 0x6e, 0x73, 0x12,
|
||||
0x39, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00, 0x52, 0x0a,
|
||||
0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x05, 0x71, 0x75,
|
||||
0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x51, 0x75,
|
||||
0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x00, 0x52, 0x05, 0x71,
|
||||
0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x66,
|
||||
0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e,
|
||||
0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x12, 0x25,
|
||||
0x0a, 0x0e, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73,
|
||||
0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x46,
|
||||
0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x3a, 0x0a, 0x07, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x72, 0x73,
|
||||
0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x53, 0x63, 0x6f, 0x72, 0x65,
|
||||
0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x72,
|
||||
0x73, 0x12, 0x40, 0x0a, 0x0c, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x50, 0x6c, 0x61, 0x6e,
|
||||
0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x6c, 0x61, 0x6e, 0x4f, 0x70, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x73, 0x42, 0x06, 0x0a, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x2a, 0xea, 0x01, 0x0a, 0x06,
|
||||
0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69,
|
||||
0x64, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x54, 0x68,
|
||||
0x61, 0x6e, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72, 0x45,
|
||||
0x71, 0x75, 0x61, 0x6c, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x4c, 0x65, 0x73, 0x73, 0x54, 0x68,
|
||||
0x61, 0x6e, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x65, 0x73, 0x73, 0x45, 0x71, 0x75, 0x61,
|
||||
0x6c, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x05, 0x12, 0x0c,
|
||||
0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b,
|
||||
0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x07, 0x12, 0x10, 0x0a,
|
||||
0x0c, 0x50, 0x6f, 0x73, 0x74, 0x66, 0x69, 0x78, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x08, 0x12,
|
||||
0x09, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x09, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x61,
|
||||
0x6e, 0x67, 0x65, 0x10, 0x0a, 0x12, 0x06, 0x0a, 0x02, 0x49, 0x6e, 0x10, 0x0b, 0x12, 0x09, 0x0a,
|
||||
0x05, 0x4e, 0x6f, 0x74, 0x49, 0x6e, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x65, 0x78, 0x74,
|
||||
0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0d, 0x12, 0x0f, 0x0a, 0x0b, 0x50, 0x68, 0x72, 0x61, 0x73,
|
||||
0x65, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0e, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x6e, 0x65,
|
||||
0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0f, 0x2a, 0x58, 0x0a, 0x0b, 0x41, 0x72, 0x69, 0x74,
|
||||
0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
|
||||
0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x10, 0x01, 0x12, 0x07, 0x0a,
|
||||
0x03, 0x53, 0x75, 0x62, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x75, 0x6c, 0x10, 0x03, 0x12,
|
||||
0x07, 0x0a, 0x03, 0x44, 0x69, 0x76, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x6f, 0x64, 0x10,
|
||||
0x05, 0x12, 0x0f, 0x0a, 0x0b, 0x41, 0x72, 0x72, 0x61, 0x79, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68,
|
||||
0x10, 0x06, 0x2a, 0xe1, 0x01, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70,
|
||||
0x65, 0x12, 0x10, 0x0a, 0x0c, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f,
|
||||
0x72, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56,
|
||||
0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x42, 0x46, 0x6c, 0x6f, 0x61,
|
||||
0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x03, 0x12, 0x15, 0x0a, 0x11, 0x53,
|
||||
0x70, 0x61, 0x72, 0x73, 0x65, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72,
|
||||
0x10, 0x04, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x74, 0x38, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72,
|
||||
0x10, 0x05, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x6c, 0x6f,
|
||||
0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x06, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x6d,
|
||||
0x62, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x10, 0x07, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73, 0x74, 0x42,
|
||||
0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x08, 0x12,
|
||||
0x15, 0x0a, 0x11, 0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x38, 0x56, 0x65,
|
||||
0x63, 0x74, 0x6f, 0x72, 0x10, 0x09, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
|
||||
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d,
|
||||
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x2f, 0x70, 0x6c, 0x61, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x33,
|
||||
0x2e, 0x45, 0x78, 0x70, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x16, 0x0a,
|
||||
0x06, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x06, 0x77,
|
||||
0x65, 0x69, 0x67, 0x68, 0x74, 0x22, 0x3b, 0x0a, 0x0a, 0x50, 0x6c, 0x61, 0x6e, 0x4f, 0x70, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x13, 0x65, 0x78, 0x70, 0x72, 0x5f, 0x75, 0x73, 0x65, 0x5f,
|
||||
0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08,
|
||||
0x52, 0x10, 0x65, 0x78, 0x70, 0x72, 0x55, 0x73, 0x65, 0x4a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61,
|
||||
0x74, 0x73, 0x22, 0x98, 0x03, 0x0a, 0x08, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x12,
|
||||
0x40, 0x0a, 0x0b, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x6e, 0x6e, 0x73, 0x18, 0x01,
|
||||
0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41,
|
||||
0x4e, 0x4e, 0x53, 0x48, 0x00, 0x52, 0x0a, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x41, 0x6e, 0x6e,
|
||||
0x73, 0x12, 0x39, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x48, 0x00,
|
||||
0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x05,
|
||||
0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69,
|
||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e,
|
||||
0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x6c, 0x61, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x00, 0x52,
|
||||
0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74,
|
||||
0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03,
|
||||
0x52, 0x0e, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73,
|
||||
0x12, 0x25, 0x0a, 0x0e, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, 0x5f, 0x66, 0x69, 0x65, 0x6c,
|
||||
0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x79, 0x6e, 0x61, 0x6d, 0x69,
|
||||
0x63, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x3a, 0x0a, 0x07, 0x73, 0x63, 0x6f, 0x72, 0x65,
|
||||
0x72, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x53, 0x63, 0x6f,
|
||||
0x72, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x63, 0x6f, 0x72,
|
||||
0x65, 0x72, 0x73, 0x12, 0x40, 0x0a, 0x0c, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x6f, 0x70, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x6c, 0x61, 0x6e, 0x2e, 0x50, 0x6c,
|
||||
0x61, 0x6e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x6c, 0x61, 0x6e, 0x4f, 0x70,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x06, 0x0a, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x2a, 0xea, 0x01,
|
||||
0x0a, 0x06, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61,
|
||||
0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x47, 0x72, 0x65, 0x61, 0x74, 0x65, 0x72,
|
||||
0x54, 0x68, 0x61, 0x6e, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x72, 0x65, 0x61, 0x74, 0x65,
|
||||
0x72, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x4c, 0x65, 0x73, 0x73,
|
||||
0x54, 0x68, 0x61, 0x6e, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x65, 0x73, 0x73, 0x45, 0x71,
|
||||
0x75, 0x61, 0x6c, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x05,
|
||||
0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x10, 0x06, 0x12, 0x0f,
|
||||
0x0a, 0x0b, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x07, 0x12,
|
||||
0x10, 0x0a, 0x0c, 0x50, 0x6f, 0x73, 0x74, 0x66, 0x69, 0x78, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10,
|
||||
0x08, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x09, 0x12, 0x09, 0x0a, 0x05,
|
||||
0x52, 0x61, 0x6e, 0x67, 0x65, 0x10, 0x0a, 0x12, 0x06, 0x0a, 0x02, 0x49, 0x6e, 0x10, 0x0b, 0x12,
|
||||
0x09, 0x0a, 0x05, 0x4e, 0x6f, 0x74, 0x49, 0x6e, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x65,
|
||||
0x78, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0d, 0x12, 0x0f, 0x0a, 0x0b, 0x50, 0x68, 0x72,
|
||||
0x61, 0x73, 0x65, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0e, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e,
|
||||
0x6e, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x10, 0x0f, 0x2a, 0x58, 0x0a, 0x0b, 0x41, 0x72,
|
||||
0x69, 0x74, 0x68, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b,
|
||||
0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x10, 0x01, 0x12,
|
||||
0x07, 0x0a, 0x03, 0x53, 0x75, 0x62, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x75, 0x6c, 0x10,
|
||||
0x03, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x69, 0x76, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x6f,
|
||||
0x64, 0x10, 0x05, 0x12, 0x0f, 0x0a, 0x0b, 0x41, 0x72, 0x72, 0x61, 0x79, 0x4c, 0x65, 0x6e, 0x67,
|
||||
0x74, 0x68, 0x10, 0x06, 0x2a, 0xe1, 0x01, 0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54,
|
||||
0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x56, 0x65, 0x63,
|
||||
0x74, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65,
|
||||
0x63, 0x74, 0x6f, 0x72, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31,
|
||||
0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x42, 0x46, 0x6c,
|
||||
0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x03, 0x12, 0x15, 0x0a,
|
||||
0x11, 0x53, 0x70, 0x61, 0x72, 0x73, 0x65, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x10, 0x04, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x74, 0x38, 0x56, 0x65, 0x63, 0x74,
|
||||
0x6f, 0x72, 0x10, 0x05, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73, 0x74, 0x46,
|
||||
0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x06, 0x12, 0x18, 0x0a, 0x14,
|
||||
0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65,
|
||||
0x63, 0x74, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73,
|
||||
0x74, 0x42, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10,
|
||||
0x08, 0x12, 0x15, 0x0a, 0x11, 0x45, 0x6d, 0x62, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x38,
|
||||
0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x09, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68,
|
||||
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f,
|
||||
0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x6c, 0x61, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
|
@ -3199,7 +3419,7 @@ func file_plan_proto_rawDescGZIP() []byte {
|
|||
}
|
||||
|
||||
var file_plan_proto_enumTypes = make([]protoimpl.EnumInfo, 7)
|
||||
var file_plan_proto_msgTypes = make([]protoimpl.MessageInfo, 28)
|
||||
var file_plan_proto_msgTypes = make([]protoimpl.MessageInfo, 30)
|
||||
var file_plan_proto_goTypes = []interface{}{
|
||||
(OpType)(0), // 0: milvus.proto.plan.OpType
|
||||
(ArithOpType)(0), // 1: milvus.proto.plan.ArithOpType
|
||||
|
|
@ -3230,22 +3450,24 @@ var file_plan_proto_goTypes = []interface{}{
|
|||
(*BinaryArithOpEvalRangeExpr)(nil), // 26: milvus.proto.plan.BinaryArithOpEvalRangeExpr
|
||||
(*RandomSampleExpr)(nil), // 27: milvus.proto.plan.RandomSampleExpr
|
||||
(*AlwaysTrueExpr)(nil), // 28: milvus.proto.plan.AlwaysTrueExpr
|
||||
(*Expr)(nil), // 29: milvus.proto.plan.Expr
|
||||
(*VectorANNS)(nil), // 30: milvus.proto.plan.VectorANNS
|
||||
(*QueryPlanNode)(nil), // 31: milvus.proto.plan.QueryPlanNode
|
||||
(*ScoreFunction)(nil), // 32: milvus.proto.plan.ScoreFunction
|
||||
(*PlanOption)(nil), // 33: milvus.proto.plan.PlanOption
|
||||
(*PlanNode)(nil), // 34: milvus.proto.plan.PlanNode
|
||||
(schemapb.DataType)(0), // 35: milvus.proto.schema.DataType
|
||||
(*Interval)(nil), // 29: milvus.proto.plan.Interval
|
||||
(*TimestamptzArithCompareExpr)(nil), // 30: milvus.proto.plan.TimestamptzArithCompareExpr
|
||||
(*Expr)(nil), // 31: milvus.proto.plan.Expr
|
||||
(*VectorANNS)(nil), // 32: milvus.proto.plan.VectorANNS
|
||||
(*QueryPlanNode)(nil), // 33: milvus.proto.plan.QueryPlanNode
|
||||
(*ScoreFunction)(nil), // 34: milvus.proto.plan.ScoreFunction
|
||||
(*PlanOption)(nil), // 35: milvus.proto.plan.PlanOption
|
||||
(*PlanNode)(nil), // 36: milvus.proto.plan.PlanNode
|
||||
(schemapb.DataType)(0), // 37: milvus.proto.schema.DataType
|
||||
}
|
||||
var file_plan_proto_depIdxs = []int32{
|
||||
8, // 0: milvus.proto.plan.GenericValue.array_val:type_name -> milvus.proto.plan.Array
|
||||
7, // 1: milvus.proto.plan.Array.array:type_name -> milvus.proto.plan.GenericValue
|
||||
35, // 2: milvus.proto.plan.Array.element_type:type_name -> milvus.proto.schema.DataType
|
||||
37, // 2: milvus.proto.plan.Array.element_type:type_name -> milvus.proto.schema.DataType
|
||||
9, // 3: milvus.proto.plan.QueryInfo.search_iterator_v2_info:type_name -> milvus.proto.plan.SearchIteratorV2Info
|
||||
35, // 4: milvus.proto.plan.QueryInfo.json_type:type_name -> milvus.proto.schema.DataType
|
||||
35, // 5: milvus.proto.plan.ColumnInfo.data_type:type_name -> milvus.proto.schema.DataType
|
||||
35, // 6: milvus.proto.plan.ColumnInfo.element_type:type_name -> milvus.proto.schema.DataType
|
||||
37, // 4: milvus.proto.plan.QueryInfo.json_type:type_name -> milvus.proto.schema.DataType
|
||||
37, // 5: milvus.proto.plan.ColumnInfo.data_type:type_name -> milvus.proto.schema.DataType
|
||||
37, // 6: milvus.proto.plan.ColumnInfo.element_type:type_name -> milvus.proto.schema.DataType
|
||||
11, // 7: milvus.proto.plan.ColumnExpr.info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
11, // 8: milvus.proto.plan.ExistsExpr.info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
7, // 9: milvus.proto.plan.ValueExpr.value:type_name -> milvus.proto.plan.GenericValue
|
||||
|
|
@ -3256,7 +3478,7 @@ var file_plan_proto_depIdxs = []int32{
|
|||
11, // 14: milvus.proto.plan.BinaryRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
7, // 15: milvus.proto.plan.BinaryRangeExpr.lower_value:type_name -> milvus.proto.plan.GenericValue
|
||||
7, // 16: milvus.proto.plan.BinaryRangeExpr.upper_value:type_name -> milvus.proto.plan.GenericValue
|
||||
29, // 17: milvus.proto.plan.CallExpr.function_parameters:type_name -> milvus.proto.plan.Expr
|
||||
31, // 17: milvus.proto.plan.CallExpr.function_parameters:type_name -> milvus.proto.plan.Expr
|
||||
11, // 18: milvus.proto.plan.CompareExpr.left_column_info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
11, // 19: milvus.proto.plan.CompareExpr.right_column_info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
0, // 20: milvus.proto.plan.CompareExpr.op:type_name -> milvus.proto.plan.OpType
|
||||
|
|
@ -3268,53 +3490,59 @@ var file_plan_proto_depIdxs = []int32{
|
|||
11, // 26: milvus.proto.plan.NullExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
4, // 27: milvus.proto.plan.NullExpr.op:type_name -> milvus.proto.plan.NullExpr.NullOp
|
||||
5, // 28: milvus.proto.plan.UnaryExpr.op:type_name -> milvus.proto.plan.UnaryExpr.UnaryOp
|
||||
29, // 29: milvus.proto.plan.UnaryExpr.child:type_name -> milvus.proto.plan.Expr
|
||||
31, // 29: milvus.proto.plan.UnaryExpr.child:type_name -> milvus.proto.plan.Expr
|
||||
6, // 30: milvus.proto.plan.BinaryExpr.op:type_name -> milvus.proto.plan.BinaryExpr.BinaryOp
|
||||
29, // 31: milvus.proto.plan.BinaryExpr.left:type_name -> milvus.proto.plan.Expr
|
||||
29, // 32: milvus.proto.plan.BinaryExpr.right:type_name -> milvus.proto.plan.Expr
|
||||
31, // 31: milvus.proto.plan.BinaryExpr.left:type_name -> milvus.proto.plan.Expr
|
||||
31, // 32: milvus.proto.plan.BinaryExpr.right:type_name -> milvus.proto.plan.Expr
|
||||
11, // 33: milvus.proto.plan.BinaryArithOp.column_info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
1, // 34: milvus.proto.plan.BinaryArithOp.arith_op:type_name -> milvus.proto.plan.ArithOpType
|
||||
7, // 35: milvus.proto.plan.BinaryArithOp.right_operand:type_name -> milvus.proto.plan.GenericValue
|
||||
29, // 36: milvus.proto.plan.BinaryArithExpr.left:type_name -> milvus.proto.plan.Expr
|
||||
29, // 37: milvus.proto.plan.BinaryArithExpr.right:type_name -> milvus.proto.plan.Expr
|
||||
31, // 36: milvus.proto.plan.BinaryArithExpr.left:type_name -> milvus.proto.plan.Expr
|
||||
31, // 37: milvus.proto.plan.BinaryArithExpr.right:type_name -> milvus.proto.plan.Expr
|
||||
1, // 38: milvus.proto.plan.BinaryArithExpr.op:type_name -> milvus.proto.plan.ArithOpType
|
||||
11, // 39: milvus.proto.plan.BinaryArithOpEvalRangeExpr.column_info:type_name -> milvus.proto.plan.ColumnInfo
|
||||
1, // 40: milvus.proto.plan.BinaryArithOpEvalRangeExpr.arith_op:type_name -> milvus.proto.plan.ArithOpType
|
||||
7, // 41: milvus.proto.plan.BinaryArithOpEvalRangeExpr.right_operand:type_name -> milvus.proto.plan.GenericValue
|
||||
0, // 42: milvus.proto.plan.BinaryArithOpEvalRangeExpr.op:type_name -> milvus.proto.plan.OpType
|
||||
7, // 43: milvus.proto.plan.BinaryArithOpEvalRangeExpr.value:type_name -> milvus.proto.plan.GenericValue
|
||||
29, // 44: milvus.proto.plan.RandomSampleExpr.predicate:type_name -> milvus.proto.plan.Expr
|
||||
19, // 45: milvus.proto.plan.Expr.term_expr:type_name -> milvus.proto.plan.TermExpr
|
||||
22, // 46: milvus.proto.plan.Expr.unary_expr:type_name -> milvus.proto.plan.UnaryExpr
|
||||
23, // 47: milvus.proto.plan.Expr.binary_expr:type_name -> milvus.proto.plan.BinaryExpr
|
||||
18, // 48: milvus.proto.plan.Expr.compare_expr:type_name -> milvus.proto.plan.CompareExpr
|
||||
15, // 49: milvus.proto.plan.Expr.unary_range_expr:type_name -> milvus.proto.plan.UnaryRangeExpr
|
||||
16, // 50: milvus.proto.plan.Expr.binary_range_expr:type_name -> milvus.proto.plan.BinaryRangeExpr
|
||||
26, // 51: milvus.proto.plan.Expr.binary_arith_op_eval_range_expr:type_name -> milvus.proto.plan.BinaryArithOpEvalRangeExpr
|
||||
25, // 52: milvus.proto.plan.Expr.binary_arith_expr:type_name -> milvus.proto.plan.BinaryArithExpr
|
||||
14, // 53: milvus.proto.plan.Expr.value_expr:type_name -> milvus.proto.plan.ValueExpr
|
||||
12, // 54: milvus.proto.plan.Expr.column_expr:type_name -> milvus.proto.plan.ColumnExpr
|
||||
13, // 55: milvus.proto.plan.Expr.exists_expr:type_name -> milvus.proto.plan.ExistsExpr
|
||||
28, // 56: milvus.proto.plan.Expr.always_true_expr:type_name -> milvus.proto.plan.AlwaysTrueExpr
|
||||
20, // 57: milvus.proto.plan.Expr.json_contains_expr:type_name -> milvus.proto.plan.JSONContainsExpr
|
||||
17, // 58: milvus.proto.plan.Expr.call_expr:type_name -> milvus.proto.plan.CallExpr
|
||||
21, // 59: milvus.proto.plan.Expr.null_expr:type_name -> milvus.proto.plan.NullExpr
|
||||
27, // 60: milvus.proto.plan.Expr.random_sample_expr:type_name -> milvus.proto.plan.RandomSampleExpr
|
||||
2, // 61: milvus.proto.plan.VectorANNS.vector_type:type_name -> milvus.proto.plan.VectorType
|
||||
29, // 62: milvus.proto.plan.VectorANNS.predicates:type_name -> milvus.proto.plan.Expr
|
||||
10, // 63: milvus.proto.plan.VectorANNS.query_info:type_name -> milvus.proto.plan.QueryInfo
|
||||
29, // 64: milvus.proto.plan.QueryPlanNode.predicates:type_name -> milvus.proto.plan.Expr
|
||||
29, // 65: milvus.proto.plan.ScoreFunction.filter:type_name -> milvus.proto.plan.Expr
|
||||
30, // 66: milvus.proto.plan.PlanNode.vector_anns:type_name -> milvus.proto.plan.VectorANNS
|
||||
29, // 67: milvus.proto.plan.PlanNode.predicates:type_name -> milvus.proto.plan.Expr
|
||||
31, // 68: milvus.proto.plan.PlanNode.query:type_name -> milvus.proto.plan.QueryPlanNode
|
||||
32, // 69: milvus.proto.plan.PlanNode.scorers:type_name -> milvus.proto.plan.ScoreFunction
|
||||
33, // 70: milvus.proto.plan.PlanNode.plan_options:type_name -> milvus.proto.plan.PlanOption
|
||||
71, // [71:71] is the sub-list for method output_type
|
||||
71, // [71:71] is the sub-list for method input_type
|
||||
71, // [71:71] is the sub-list for extension type_name
|
||||
71, // [71:71] is the sub-list for extension extendee
|
||||
0, // [0:71] is the sub-list for field type_name
|
||||
31, // 44: milvus.proto.plan.RandomSampleExpr.predicate:type_name -> milvus.proto.plan.Expr
|
||||
11, // 45: milvus.proto.plan.TimestamptzArithCompareExpr.timestamptz_column:type_name -> milvus.proto.plan.ColumnInfo
|
||||
1, // 46: milvus.proto.plan.TimestamptzArithCompareExpr.arith_op:type_name -> milvus.proto.plan.ArithOpType
|
||||
29, // 47: milvus.proto.plan.TimestamptzArithCompareExpr.interval:type_name -> milvus.proto.plan.Interval
|
||||
0, // 48: milvus.proto.plan.TimestamptzArithCompareExpr.compare_op:type_name -> milvus.proto.plan.OpType
|
||||
7, // 49: milvus.proto.plan.TimestamptzArithCompareExpr.compare_value:type_name -> milvus.proto.plan.GenericValue
|
||||
19, // 50: milvus.proto.plan.Expr.term_expr:type_name -> milvus.proto.plan.TermExpr
|
||||
22, // 51: milvus.proto.plan.Expr.unary_expr:type_name -> milvus.proto.plan.UnaryExpr
|
||||
23, // 52: milvus.proto.plan.Expr.binary_expr:type_name -> milvus.proto.plan.BinaryExpr
|
||||
18, // 53: milvus.proto.plan.Expr.compare_expr:type_name -> milvus.proto.plan.CompareExpr
|
||||
15, // 54: milvus.proto.plan.Expr.unary_range_expr:type_name -> milvus.proto.plan.UnaryRangeExpr
|
||||
16, // 55: milvus.proto.plan.Expr.binary_range_expr:type_name -> milvus.proto.plan.BinaryRangeExpr
|
||||
26, // 56: milvus.proto.plan.Expr.binary_arith_op_eval_range_expr:type_name -> milvus.proto.plan.BinaryArithOpEvalRangeExpr
|
||||
25, // 57: milvus.proto.plan.Expr.binary_arith_expr:type_name -> milvus.proto.plan.BinaryArithExpr
|
||||
14, // 58: milvus.proto.plan.Expr.value_expr:type_name -> milvus.proto.plan.ValueExpr
|
||||
12, // 59: milvus.proto.plan.Expr.column_expr:type_name -> milvus.proto.plan.ColumnExpr
|
||||
13, // 60: milvus.proto.plan.Expr.exists_expr:type_name -> milvus.proto.plan.ExistsExpr
|
||||
28, // 61: milvus.proto.plan.Expr.always_true_expr:type_name -> milvus.proto.plan.AlwaysTrueExpr
|
||||
20, // 62: milvus.proto.plan.Expr.json_contains_expr:type_name -> milvus.proto.plan.JSONContainsExpr
|
||||
17, // 63: milvus.proto.plan.Expr.call_expr:type_name -> milvus.proto.plan.CallExpr
|
||||
21, // 64: milvus.proto.plan.Expr.null_expr:type_name -> milvus.proto.plan.NullExpr
|
||||
27, // 65: milvus.proto.plan.Expr.random_sample_expr:type_name -> milvus.proto.plan.RandomSampleExpr
|
||||
30, // 66: milvus.proto.plan.Expr.timestamptz_arith_compare_expr:type_name -> milvus.proto.plan.TimestamptzArithCompareExpr
|
||||
2, // 67: milvus.proto.plan.VectorANNS.vector_type:type_name -> milvus.proto.plan.VectorType
|
||||
31, // 68: milvus.proto.plan.VectorANNS.predicates:type_name -> milvus.proto.plan.Expr
|
||||
10, // 69: milvus.proto.plan.VectorANNS.query_info:type_name -> milvus.proto.plan.QueryInfo
|
||||
31, // 70: milvus.proto.plan.QueryPlanNode.predicates:type_name -> milvus.proto.plan.Expr
|
||||
31, // 71: milvus.proto.plan.ScoreFunction.filter:type_name -> milvus.proto.plan.Expr
|
||||
32, // 72: milvus.proto.plan.PlanNode.vector_anns:type_name -> milvus.proto.plan.VectorANNS
|
||||
31, // 73: milvus.proto.plan.PlanNode.predicates:type_name -> milvus.proto.plan.Expr
|
||||
33, // 74: milvus.proto.plan.PlanNode.query:type_name -> milvus.proto.plan.QueryPlanNode
|
||||
34, // 75: milvus.proto.plan.PlanNode.scorers:type_name -> milvus.proto.plan.ScoreFunction
|
||||
35, // 76: milvus.proto.plan.PlanNode.plan_options:type_name -> milvus.proto.plan.PlanOption
|
||||
77, // [77:77] is the sub-list for method output_type
|
||||
77, // [77:77] is the sub-list for method input_type
|
||||
77, // [77:77] is the sub-list for extension type_name
|
||||
77, // [77:77] is the sub-list for extension extendee
|
||||
0, // [0:77] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_plan_proto_init() }
|
||||
|
|
@ -3588,7 +3816,7 @@ func file_plan_proto_init() {
|
|||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*Expr); i {
|
||||
switch v := v.(*Interval); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
|
@ -3600,7 +3828,7 @@ func file_plan_proto_init() {
|
|||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*VectorANNS); i {
|
||||
switch v := v.(*TimestamptzArithCompareExpr); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
|
@ -3612,7 +3840,7 @@ func file_plan_proto_init() {
|
|||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*QueryPlanNode); i {
|
||||
switch v := v.(*Expr); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
|
@ -3624,7 +3852,7 @@ func file_plan_proto_init() {
|
|||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*ScoreFunction); i {
|
||||
switch v := v.(*VectorANNS); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
|
@ -3636,7 +3864,7 @@ func file_plan_proto_init() {
|
|||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*PlanOption); i {
|
||||
switch v := v.(*QueryPlanNode); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
|
|
@ -3648,6 +3876,30 @@ func file_plan_proto_init() {
|
|||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*ScoreFunction); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*PlanOption); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_plan_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*PlanNode); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
|
|
@ -3669,7 +3921,7 @@ func file_plan_proto_init() {
|
|||
}
|
||||
file_plan_proto_msgTypes[2].OneofWrappers = []interface{}{}
|
||||
file_plan_proto_msgTypes[3].OneofWrappers = []interface{}{}
|
||||
file_plan_proto_msgTypes[22].OneofWrappers = []interface{}{
|
||||
file_plan_proto_msgTypes[24].OneofWrappers = []interface{}{
|
||||
(*Expr_TermExpr)(nil),
|
||||
(*Expr_UnaryExpr)(nil),
|
||||
(*Expr_BinaryExpr)(nil),
|
||||
|
|
@ -3686,8 +3938,9 @@ func file_plan_proto_init() {
|
|||
(*Expr_CallExpr)(nil),
|
||||
(*Expr_NullExpr)(nil),
|
||||
(*Expr_RandomSampleExpr)(nil),
|
||||
(*Expr_TimestamptzArithCompareExpr)(nil),
|
||||
}
|
||||
file_plan_proto_msgTypes[27].OneofWrappers = []interface{}{
|
||||
file_plan_proto_msgTypes[29].OneofWrappers = []interface{}{
|
||||
(*PlanNode_VectorAnns)(nil),
|
||||
(*PlanNode_Predicates)(nil),
|
||||
(*PlanNode_Query)(nil),
|
||||
|
|
@ -3698,7 +3951,7 @@ func file_plan_proto_init() {
|
|||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_plan_proto_rawDesc,
|
||||
NumEnums: 7,
|
||||
NumMessages: 28,
|
||||
NumMessages: 30,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
|
|
|
|||
|
|
@ -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