mirror of https://github.com/milvus-io/milvus.git
enhance: support null expr (#38772)
#31728 --------- Signed-off-by: lixinguo <xinguo.li@zilliz.com> Co-authored-by: lixinguo <xinguo.li@zilliz.com>pull/38622/head
parent
19a5c31907
commit
907fc24f85
|
@ -215,6 +215,7 @@ using GroupByValueType = std::variant<std::monostate,
|
|||
bool,
|
||||
std::string>;
|
||||
using ContainsType = proto::plan::JSONContainsExpr_JSONOp;
|
||||
using NullExprType = proto::plan::NullExpr_NullOp;
|
||||
|
||||
inline bool
|
||||
IsPrimaryKeyDataType(DataType data_type) {
|
||||
|
|
|
@ -161,6 +161,7 @@ PhyBinaryRangeFilterExpr::PreCheckOverflow(HighPrecisionType& val1,
|
|||
(input != nullptr)
|
||||
? ProcessChunksForValidByOffsets<T>(is_index_mode_, *input)
|
||||
: ProcessChunksForValid<T>(is_index_mode_);
|
||||
|
||||
auto res_vec = std::make_shared<ColumnVector>(TargetBitmap(batch_size),
|
||||
std::move(valid_res));
|
||||
return res_vec;
|
||||
|
|
|
@ -28,9 +28,11 @@
|
|||
#include "exec/expression/JsonContainsExpr.h"
|
||||
#include "exec/expression/LogicalBinaryExpr.h"
|
||||
#include "exec/expression/LogicalUnaryExpr.h"
|
||||
#include "exec/expression/NullExpr.h"
|
||||
#include "exec/expression/TermExpr.h"
|
||||
#include "exec/expression/UnaryExpr.h"
|
||||
#include "exec/expression/ValueExpr.h"
|
||||
#include "expr/ITypeExpr.h"
|
||||
|
||||
#include <memory>
|
||||
|
||||
|
@ -285,6 +287,16 @@ CompileExpression(const expr::TypedExprPtr& expr,
|
|||
context->get_segment(),
|
||||
context->get_active_count(),
|
||||
context->query_config()->get_expr_batch_size());
|
||||
} else if (auto column_expr =
|
||||
std::dynamic_pointer_cast<const milvus::expr::NullExpr>(
|
||||
expr)) {
|
||||
result = std::make_shared<PhyNullExpr>(
|
||||
compiled_inputs,
|
||||
column_expr,
|
||||
"PhyNullExpr",
|
||||
context->get_segment(),
|
||||
context->get_active_count(),
|
||||
context->query_config()->get_expr_batch_size());
|
||||
} else {
|
||||
PanicInfo(ExprInvalid, "unsupport expr: ", expr->ToString());
|
||||
}
|
||||
|
|
|
@ -0,0 +1,150 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you 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.
|
||||
|
||||
#include "NullExpr.h"
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
#include "common/Array.h"
|
||||
#include "common/Types.h"
|
||||
#include "log/Log.h"
|
||||
#include "query/Utils.h"
|
||||
namespace milvus {
|
||||
namespace exec {
|
||||
|
||||
void
|
||||
PhyNullExpr::Eval(EvalCtx& context, VectorPtr& result) {
|
||||
auto input = context.get_offset_input();
|
||||
switch (expr_->column_.data_type_) {
|
||||
case DataType::BOOL: {
|
||||
result = ExecVisitorImpl<bool>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::INT8: {
|
||||
result = ExecVisitorImpl<int8_t>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::INT16: {
|
||||
result = ExecVisitorImpl<int16_t>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::INT32: {
|
||||
result = ExecVisitorImpl<int32_t>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::INT64: {
|
||||
result = ExecVisitorImpl<int64_t>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::FLOAT: {
|
||||
result = ExecVisitorImpl<float>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::DOUBLE: {
|
||||
result = ExecVisitorImpl<double>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::VARCHAR: {
|
||||
if (segment_->type() == SegmentType::Growing &&
|
||||
!storage::MmapManager::GetInstance()
|
||||
.GetMmapConfig()
|
||||
.growing_enable_mmap) {
|
||||
result = ExecVisitorImpl<std::string>(input);
|
||||
} else {
|
||||
result = ExecVisitorImpl<std::string_view>(input);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case DataType::JSON: {
|
||||
result = ExecVisitorImpl<Json>(input);
|
||||
break;
|
||||
}
|
||||
case DataType::ARRAY: {
|
||||
result = ExecVisitorImpl<ArrayView>(input);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
PanicInfo(DataTypeInvalid,
|
||||
"unsupported data type: {}",
|
||||
expr_->column_.data_type_);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
VectorPtr
|
||||
PhyNullExpr::ExecVisitorImpl(OffsetVector* input) {
|
||||
if (auto res = PreCheckNullable(input)) {
|
||||
return res;
|
||||
}
|
||||
auto valid_res =
|
||||
(input != nullptr)
|
||||
? ProcessChunksForValidByOffsets<T>(is_index_mode_, *input)
|
||||
: ProcessChunksForValid<T>(is_index_mode_);
|
||||
TargetBitmap res = valid_res.clone();
|
||||
if (expr_->op_ == proto::plan::NullExpr_NullOp_IsNull) {
|
||||
res.flip();
|
||||
}
|
||||
auto res_vec =
|
||||
std::make_shared<ColumnVector>(std::move(res), std::move(valid_res));
|
||||
return res_vec;
|
||||
}
|
||||
|
||||
// if nullable is false, no need to process chunks
|
||||
// res is all false when is null, and is all true when is not null
|
||||
ColumnVectorPtr
|
||||
PhyNullExpr::PreCheckNullable(OffsetVector* input) {
|
||||
if (expr_->column_.nullable_) {
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
int64_t batch_size;
|
||||
if (input != nullptr) {
|
||||
batch_size = input->size();
|
||||
} else {
|
||||
batch_size = precheck_pos_ + batch_size_ >= active_count_
|
||||
? active_count_ - precheck_pos_
|
||||
: batch_size_;
|
||||
precheck_pos_ += batch_size;
|
||||
}
|
||||
if (cached_precheck_res_ != nullptr &&
|
||||
cached_precheck_res_->size() == batch_size) {
|
||||
return cached_precheck_res_;
|
||||
}
|
||||
|
||||
auto res_vec = std::make_shared<ColumnVector>(TargetBitmap(batch_size),
|
||||
TargetBitmap(batch_size));
|
||||
TargetBitmapView res(res_vec->GetRawData(), batch_size);
|
||||
TargetBitmapView valid_res(res_vec->GetValidRawData(), batch_size);
|
||||
valid_res.set();
|
||||
switch (expr_->op_) {
|
||||
case proto::plan::NullExpr_NullOp_IsNull: {
|
||||
res.reset();
|
||||
break;
|
||||
}
|
||||
case proto::plan::NullExpr_NullOp_IsNotNull: {
|
||||
res.set();
|
||||
break;
|
||||
}
|
||||
default:
|
||||
PanicInfo(ExprInvalid,
|
||||
"unsupported null expr type {}",
|
||||
proto::plan::NullExpr_NullOp_Name(expr_->op_));
|
||||
}
|
||||
cached_precheck_res_ = res_vec;
|
||||
return cached_precheck_res_;
|
||||
}
|
||||
|
||||
} //namespace exec
|
||||
} // namespace milvus
|
|
@ -0,0 +1,65 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you 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 <fmt/core.h>
|
||||
|
||||
#include "common/EasyAssert.h"
|
||||
#include "common/Types.h"
|
||||
#include "common/Vector.h"
|
||||
#include "exec/expression/Expr.h"
|
||||
#include "expr/ITypeExpr.h"
|
||||
#include "segcore/SegmentInterface.h"
|
||||
|
||||
namespace milvus {
|
||||
namespace exec {
|
||||
|
||||
class PhyNullExpr : public SegmentExpr {
|
||||
public:
|
||||
PhyNullExpr(const std::vector<std::shared_ptr<Expr>>& input,
|
||||
const std::shared_ptr<const milvus::expr::NullExpr>& expr,
|
||||
const std::string& name,
|
||||
const segcore::SegmentInternalInterface* segment,
|
||||
int64_t active_count,
|
||||
int64_t batch_size)
|
||||
: SegmentExpr(std::move(input),
|
||||
name,
|
||||
segment,
|
||||
expr->column_.field_id_,
|
||||
active_count,
|
||||
batch_size),
|
||||
expr_(expr) {
|
||||
}
|
||||
|
||||
void
|
||||
Eval(EvalCtx& context, VectorPtr& result) override;
|
||||
|
||||
private:
|
||||
ColumnVectorPtr
|
||||
PreCheckNullable(OffsetVector* input);
|
||||
|
||||
template <typename T>
|
||||
VectorPtr
|
||||
ExecVisitorImpl(OffsetVector* input);
|
||||
|
||||
private:
|
||||
std::shared_ptr<const milvus::expr::NullExpr> expr_;
|
||||
ColumnVectorPtr cached_precheck_res_{nullptr};
|
||||
int64_t precheck_pos_{0};
|
||||
};
|
||||
} //namespace exec
|
||||
} // namespace milvus
|
|
@ -905,10 +905,10 @@ PhyUnaryRangeFilterExpr::PreCheckOverflow(OffsetVector* input) {
|
|||
: batch_size_;
|
||||
overflow_check_pos_ += batch_size;
|
||||
}
|
||||
auto valid = (input != nullptr)
|
||||
? ProcessChunksForValidByOffsets<T>(
|
||||
CanUseIndex<T>(), *input)
|
||||
: ProcessChunksForValid<T>(CanUseIndex<T>());
|
||||
auto valid =
|
||||
(input != nullptr)
|
||||
? ProcessChunksForValidByOffsets<T>(is_index_mode_, *input)
|
||||
: ProcessChunksForValid<T>(is_index_mode_);
|
||||
auto res_vec = std::make_shared<ColumnVector>(
|
||||
TargetBitmap(batch_size), std::move(valid));
|
||||
TargetBitmapView res(res_vec->GetRawData(), batch_size);
|
||||
|
|
|
@ -116,22 +116,26 @@ struct ColumnInfo {
|
|||
DataType data_type_;
|
||||
DataType element_type_;
|
||||
std::vector<std::string> nested_path_;
|
||||
bool nullable_;
|
||||
|
||||
ColumnInfo(const proto::plan::ColumnInfo& column_info)
|
||||
: field_id_(column_info.field_id()),
|
||||
data_type_(static_cast<DataType>(column_info.data_type())),
|
||||
element_type_(static_cast<DataType>(column_info.element_type())),
|
||||
nested_path_(column_info.nested_path().begin(),
|
||||
column_info.nested_path().end()) {
|
||||
column_info.nested_path().end()),
|
||||
nullable_(column_info.nullable()) {
|
||||
}
|
||||
|
||||
ColumnInfo(FieldId field_id,
|
||||
DataType data_type,
|
||||
std::vector<std::string> nested_path = {})
|
||||
std::vector<std::string> nested_path = {},
|
||||
bool nullable = false)
|
||||
: field_id_(field_id),
|
||||
data_type_(data_type),
|
||||
element_type_(DataType::NONE),
|
||||
nested_path_(std::move(nested_path)) {
|
||||
nested_path_(std::move(nested_path)),
|
||||
nullable_(nullable) {
|
||||
}
|
||||
|
||||
bool
|
||||
|
@ -627,6 +631,24 @@ class BinaryArithOpEvalRangeExpr : public ITypeFilterExpr {
|
|||
const proto::plan::GenericValue value_;
|
||||
};
|
||||
|
||||
class NullExpr : public ITypeFilterExpr {
|
||||
public:
|
||||
explicit NullExpr(const ColumnInfo& column, NullExprType op)
|
||||
: ITypeFilterExpr(), column_(column), op_(op) {
|
||||
}
|
||||
|
||||
std::string
|
||||
ToString() const override {
|
||||
return fmt::format("NullExpr:[Column: {}, Operator: {} ",
|
||||
column_.ToString(),
|
||||
NullExpr_NullOp_Name(op_));
|
||||
}
|
||||
|
||||
public:
|
||||
const ColumnInfo column_;
|
||||
NullExprType op_;
|
||||
};
|
||||
|
||||
class CallExpr : public ITypeFilterExpr {
|
||||
public:
|
||||
CallExpr(const std::string fun_name,
|
||||
|
|
|
@ -315,6 +315,16 @@ ProtoParser::ParseUnaryRangeExprs(const proto::plan::UnaryRangeExpr& expr_pb) {
|
|||
expr::ColumnInfo(column_info), expr_pb.op(), expr_pb.value());
|
||||
}
|
||||
|
||||
expr::TypedExprPtr
|
||||
ProtoParser::ParseNullExprs(const proto::plan::NullExpr& expr_pb) {
|
||||
auto& column_info = expr_pb.column_info();
|
||||
auto field_id = FieldId(column_info.field_id());
|
||||
auto data_type = schema[field_id].get_data_type();
|
||||
Assert(data_type == static_cast<DataType>(column_info.data_type()));
|
||||
return std::make_shared<milvus::expr::NullExpr>(
|
||||
expr::ColumnInfo(column_info), expr_pb.op());
|
||||
}
|
||||
|
||||
expr::TypedExprPtr
|
||||
ProtoParser::ParseBinaryRangeExprs(
|
||||
const proto::plan::BinaryRangeExpr& expr_pb) {
|
||||
|
@ -521,6 +531,10 @@ ProtoParser::ParseExprs(const proto::plan::Expr& expr_pb,
|
|||
result = ParseValueExprs(expr_pb.value_expr());
|
||||
break;
|
||||
}
|
||||
case ppe::kNullExpr: {
|
||||
result = ParseNullExprs(expr_pb.null_expr());
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
std::string s;
|
||||
google::protobuf::TextFormat::PrintToString(expr_pb, &s);
|
||||
|
|
|
@ -80,6 +80,9 @@ class ProtoParser {
|
|||
expr::TypedExprPtr
|
||||
ParseExistExprs(const proto::plan::ExistsExpr& expr_pb);
|
||||
|
||||
expr::TypedExprPtr
|
||||
ParseNullExprs(const proto::plan::NullExpr& expr_pb);
|
||||
|
||||
expr::TypedExprPtr
|
||||
ParseJsonContainsExprs(const proto::plan::JSONContainsExpr& expr_pb);
|
||||
|
||||
|
|
|
@ -782,6 +782,106 @@ TEST(Expr, TestArrayEqual) {
|
|||
}
|
||||
}
|
||||
|
||||
TEST(Expr, TestArrayNullExpr) {
|
||||
std::vector<std::tuple<std::string, std::function<bool(bool)>>> testcases =
|
||||
{
|
||||
{R"(null_expr: <
|
||||
column_info: <
|
||||
field_id: 102
|
||||
data_type: Array
|
||||
element_type:Int64
|
||||
nullable: true
|
||||
>
|
||||
op:IsNull
|
||||
>)",
|
||||
[](bool v) { return !v; }},
|
||||
};
|
||||
|
||||
std::string raw_plan_tmp = R"(vector_anns: <
|
||||
field_id: 100
|
||||
predicates: <
|
||||
@@@@
|
||||
>
|
||||
query_info: <
|
||||
topk: 10
|
||||
round_decimal: 3
|
||||
metric_type: "L2"
|
||||
search_params: "{\"nprobe\": 10}"
|
||||
>
|
||||
placeholder_tag: "$0"
|
||||
>)";
|
||||
auto schema = std::make_shared<Schema>();
|
||||
auto vec_fid = schema->AddDebugField(
|
||||
"fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
|
||||
auto i64_fid = schema->AddDebugField("id", DataType::INT64);
|
||||
auto long_array_fid = schema->AddDebugField(
|
||||
"long_array", DataType::ARRAY, DataType::INT64, true);
|
||||
schema->set_primary_field_id(i64_fid);
|
||||
|
||||
auto seg = CreateGrowingSegment(schema, empty_index_meta);
|
||||
int N = 1000;
|
||||
std::vector<ScalarArray> long_array_col;
|
||||
int num_iters = 1;
|
||||
FixedVector<bool> valid_data;
|
||||
|
||||
for (int iter = 0; iter < num_iters; ++iter) {
|
||||
auto raw_data = DataGen(schema, N, iter, 0, 1, 3);
|
||||
auto new_long_array_col = raw_data.get_col<ScalarArray>(long_array_fid);
|
||||
long_array_col.insert(long_array_col.end(),
|
||||
new_long_array_col.begin(),
|
||||
new_long_array_col.end());
|
||||
auto new_valid_col = raw_data.get_col_valid(long_array_fid);
|
||||
valid_data.insert(
|
||||
valid_data.end(), new_valid_col.begin(), new_valid_col.end());
|
||||
seg->PreInsert(N);
|
||||
seg->Insert(iter * N,
|
||||
N,
|
||||
raw_data.row_ids_.data(),
|
||||
raw_data.timestamps_.data(),
|
||||
raw_data.raw_);
|
||||
}
|
||||
|
||||
auto seg_promote = dynamic_cast<SegmentGrowingImpl*>(seg.get());
|
||||
for (auto [clause, ref_func] : testcases) {
|
||||
auto loc = raw_plan_tmp.find("@@@@");
|
||||
auto raw_plan = raw_plan_tmp;
|
||||
raw_plan.replace(loc, 4, clause);
|
||||
auto plan_str = translate_text_plan_to_binary_plan(raw_plan.c_str());
|
||||
auto plan =
|
||||
CreateSearchPlanByExpr(*schema, plan_str.data(), plan_str.size());
|
||||
BitsetType final;
|
||||
final = ExecuteQueryExpr(
|
||||
plan->plan_node_->plannodes_->sources()[0]->sources()[0],
|
||||
seg_promote,
|
||||
N * num_iters,
|
||||
MAX_TIMESTAMP);
|
||||
EXPECT_EQ(final.size(), N * num_iters);
|
||||
|
||||
// specify some offsets and do scalar filtering on these offsets
|
||||
milvus::exec::OffsetVector offsets;
|
||||
offsets.reserve(N * num_iters / 2);
|
||||
for (auto i = 0; i < N * num_iters; ++i) {
|
||||
if (i % 2 == 0) {
|
||||
offsets.emplace_back(i);
|
||||
}
|
||||
}
|
||||
auto col_vec = milvus::test::gen_filter_res(
|
||||
plan->plan_node_->plannodes_->sources()[0]->sources()[0].get(),
|
||||
seg_promote,
|
||||
N * num_iters,
|
||||
MAX_TIMESTAMP,
|
||||
&offsets);
|
||||
BitsetTypeView view(col_vec->GetRawData(), col_vec->size());
|
||||
EXPECT_EQ(view.size(), N * num_iters / 2);
|
||||
for (int i = 0; i < N * num_iters; ++i) {
|
||||
auto ans = final[i];
|
||||
auto valid = valid_data[i];
|
||||
auto ref = ref_func(valid);
|
||||
ASSERT_EQ(ans, ref);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(Expr, PraseArrayContainsExpr) {
|
||||
std::vector<const char*> raw_plans{
|
||||
R"(vector_anns:<
|
||||
|
|
|
@ -5138,6 +5138,233 @@ TEST_P(ExprTest, TestBinaryArithOpEvalRangeBenchExpr) {
|
|||
}
|
||||
}
|
||||
|
||||
TEST(Expr, TestExprNull) {
|
||||
auto schema = std::make_shared<Schema>();
|
||||
auto bool_fid = schema->AddDebugField("bool", DataType::BOOL, true);
|
||||
auto bool_1_fid = schema->AddDebugField("bool1", DataType::BOOL);
|
||||
auto int8_fid = schema->AddDebugField("int8", DataType::INT8, true);
|
||||
auto int8_1_fid = schema->AddDebugField("int81", DataType::INT8);
|
||||
auto int16_fid = schema->AddDebugField("int16", DataType::INT16, true);
|
||||
auto int16_1_fid = schema->AddDebugField("int161", DataType::INT16);
|
||||
auto int32_fid = schema->AddDebugField("int32", DataType::INT32, true);
|
||||
auto int32_1_fid = schema->AddDebugField("int321", DataType::INT32);
|
||||
auto int64_fid = schema->AddDebugField("int64", DataType::INT64, true);
|
||||
auto int64_1_fid = schema->AddDebugField("int641", DataType::INT64);
|
||||
auto str1_fid = schema->AddDebugField("string1", DataType::VARCHAR);
|
||||
auto str2_fid = schema->AddDebugField("string2", DataType::VARCHAR, true);
|
||||
auto float_fid = schema->AddDebugField("float", DataType::FLOAT, true);
|
||||
auto float_1_fid = schema->AddDebugField("float1", DataType::FLOAT);
|
||||
auto double_fid = schema->AddDebugField("double", DataType::DOUBLE, true);
|
||||
auto double_1_fid = schema->AddDebugField("double1", DataType::DOUBLE);
|
||||
schema->set_primary_field_id(str1_fid);
|
||||
|
||||
std::map<DataType, FieldId> fids = {{DataType::BOOL, bool_fid},
|
||||
{DataType::INT8, int8_fid},
|
||||
{DataType::INT16, int16_fid},
|
||||
{DataType::INT32, int32_fid},
|
||||
{DataType::INT64, int64_fid},
|
||||
{DataType::VARCHAR, str2_fid},
|
||||
{DataType::FLOAT, float_fid},
|
||||
{DataType::DOUBLE, double_fid}};
|
||||
|
||||
std::map<DataType, FieldId> fids_not_nullable = {
|
||||
{DataType::BOOL, bool_1_fid},
|
||||
{DataType::INT8, int8_1_fid},
|
||||
{DataType::INT16, int16_1_fid},
|
||||
{DataType::INT32, int32_1_fid},
|
||||
{DataType::INT64, int64_1_fid},
|
||||
{DataType::VARCHAR, str1_fid},
|
||||
{DataType::FLOAT, float_1_fid},
|
||||
{DataType::DOUBLE, double_1_fid}};
|
||||
|
||||
auto seg = CreateSealedSegment(schema);
|
||||
FixedVector<bool> valid_data_bool;
|
||||
FixedVector<bool> valid_data_i8;
|
||||
FixedVector<bool> valid_data_i16;
|
||||
FixedVector<bool> valid_data_i32;
|
||||
FixedVector<bool> valid_data_i64;
|
||||
FixedVector<bool> valid_data_str;
|
||||
FixedVector<bool> valid_data_float;
|
||||
FixedVector<bool> valid_data_double;
|
||||
|
||||
int N = 1000;
|
||||
auto raw_data = DataGen(schema, N);
|
||||
valid_data_bool = raw_data.get_col_valid(bool_fid);
|
||||
valid_data_i8 = raw_data.get_col_valid(int8_fid);
|
||||
valid_data_i16 = raw_data.get_col_valid(int16_fid);
|
||||
valid_data_i32 = raw_data.get_col_valid(int32_fid);
|
||||
valid_data_i64 = raw_data.get_col_valid(int64_fid);
|
||||
valid_data_str = raw_data.get_col_valid(str2_fid);
|
||||
valid_data_float = raw_data.get_col_valid(float_fid);
|
||||
valid_data_double = raw_data.get_col_valid(double_fid);
|
||||
|
||||
FixedVector<bool> valid_data_all_true(N, true);
|
||||
|
||||
// load field data
|
||||
auto fields = schema->get_fields();
|
||||
for (auto field_data : raw_data.raw_->fields_data()) {
|
||||
int64_t field_id = field_data.field_id();
|
||||
|
||||
auto info = FieldDataInfo(field_data.field_id(), N, "/tmp/a");
|
||||
auto field_meta = fields.at(FieldId(field_id));
|
||||
info.channel->push(
|
||||
CreateFieldDataFromDataArray(N, &field_data, field_meta));
|
||||
info.channel->close();
|
||||
|
||||
seg->LoadFieldData(FieldId(field_id), info);
|
||||
}
|
||||
|
||||
auto build_nullable_expr = [&](DataType data_type,
|
||||
NullExprType op) -> expr::TypedExprPtr {
|
||||
return std::make_shared<expr::NullExpr>(
|
||||
expr::ColumnInfo(fids[data_type], data_type, {}, true), op);
|
||||
};
|
||||
|
||||
auto build_not_nullable_expr = [&](DataType data_type,
|
||||
NullExprType op) -> expr::TypedExprPtr {
|
||||
return std::make_shared<expr::NullExpr>(
|
||||
expr::ColumnInfo(
|
||||
fids_not_nullable[data_type], data_type, {}, false),
|
||||
op);
|
||||
};
|
||||
|
||||
auto test_is_null_ans = [=, &seg](expr::TypedExprPtr expr,
|
||||
FixedVector<bool> valid_data) {
|
||||
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
|
||||
BitsetType final;
|
||||
auto plan =
|
||||
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, expr);
|
||||
final = ExecuteQueryExpr(plan, seg.get(), N, MAX_TIMESTAMP);
|
||||
EXPECT_EQ(final.size(), N);
|
||||
for (int i = 0; i < N; i++) {
|
||||
EXPECT_NE(final[i], valid_data[i]);
|
||||
}
|
||||
};
|
||||
|
||||
auto test_is_not_null_ans = [=, &seg](expr::TypedExprPtr expr,
|
||||
FixedVector<bool> valid_data) {
|
||||
query::ExecPlanNodeVisitor visitor(*seg, MAX_TIMESTAMP);
|
||||
BitsetType final;
|
||||
auto plan =
|
||||
std::make_shared<plan::FilterBitsNode>(DEFAULT_PLANNODE_ID, expr);
|
||||
final = ExecuteQueryExpr(plan, seg.get(), N, MAX_TIMESTAMP);
|
||||
EXPECT_EQ(final.size(), N);
|
||||
for (int i = 0; i < N; i++) {
|
||||
EXPECT_EQ(final[i], valid_data[i]);
|
||||
}
|
||||
};
|
||||
|
||||
auto expr = build_nullable_expr(DataType::BOOL,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_bool);
|
||||
expr = build_nullable_expr(DataType::INT8,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_i8);
|
||||
expr = build_nullable_expr(DataType::INT16,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_i16);
|
||||
expr = build_nullable_expr(DataType::INT32,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_i32);
|
||||
expr = build_nullable_expr(DataType::INT64,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_i64);
|
||||
expr = build_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_float);
|
||||
expr = build_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_double);
|
||||
expr = build_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_float);
|
||||
expr = build_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_double);
|
||||
expr = build_nullable_expr(DataType::BOOL,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_bool);
|
||||
expr = build_nullable_expr(DataType::INT8,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_i8);
|
||||
expr = build_nullable_expr(DataType::INT16,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_i16);
|
||||
expr = build_nullable_expr(DataType::INT32,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_i32);
|
||||
expr = build_nullable_expr(DataType::INT64,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_i64);
|
||||
expr = build_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_float);
|
||||
expr = build_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_double);
|
||||
expr = build_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_float);
|
||||
expr = build_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_double);
|
||||
//not nullable expr
|
||||
expr = build_not_nullable_expr(DataType::BOOL,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT8,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT16,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT32,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT64,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNull);
|
||||
test_is_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::BOOL,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT8,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT16,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT32,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::INT64,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::FLOAT,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
expr = build_not_nullable_expr(DataType::DOUBLE,
|
||||
proto::plan::NullExpr_NullOp_IsNotNull);
|
||||
test_is_not_null_ans(expr, valid_data_all_true);
|
||||
}
|
||||
|
||||
TEST_P(ExprTest, TestCompareExprBenchTest) {
|
||||
auto schema = std::make_shared<Schema>();
|
||||
auto vec_fid = schema->AddDebugField("fakevec", data_type, 16, metric_type);
|
||||
|
@ -11455,6 +11682,104 @@ TEST_P(ExprTest, TestUnaryRangeWithJSONNullable) {
|
|||
}
|
||||
}
|
||||
|
||||
TEST_P(ExprTest, TestNullExprWithJSON) {
|
||||
std::vector<std::tuple<std::string, std::function<bool(bool)>>> testcases =
|
||||
{
|
||||
{R"(null_expr: <
|
||||
column_info: <
|
||||
field_id: 102
|
||||
data_type:JSON
|
||||
nullable: true
|
||||
>
|
||||
op:IsNull
|
||||
>)",
|
||||
[](bool v) { return !v; }},
|
||||
};
|
||||
|
||||
std::string raw_plan_tmp = R"(vector_anns: <
|
||||
field_id: 100
|
||||
predicates: <
|
||||
@@@@
|
||||
>
|
||||
query_info: <
|
||||
topk: 10
|
||||
round_decimal: 3
|
||||
metric_type: "L2"
|
||||
search_params: "{\"nprobe\": 10}"
|
||||
>
|
||||
placeholder_tag: "$0"
|
||||
>)";
|
||||
auto schema = std::make_shared<Schema>();
|
||||
auto vec_fid = schema->AddDebugField(
|
||||
"fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
|
||||
auto i64_fid = schema->AddDebugField("id", DataType::INT64);
|
||||
auto json_fid = schema->AddDebugField("json", DataType::JSON, true);
|
||||
schema->set_primary_field_id(i64_fid);
|
||||
|
||||
auto seg = CreateGrowingSegment(schema, empty_index_meta);
|
||||
int N = 1000;
|
||||
int num_iters = 1;
|
||||
FixedVector<bool> valid_data;
|
||||
std::vector<std::string> json_col;
|
||||
|
||||
for (int iter = 0; iter < num_iters; ++iter) {
|
||||
auto raw_data = DataGen(schema, N, iter, 0, 1, 3);
|
||||
auto new_json_col = raw_data.get_col<std::string>(json_fid);
|
||||
|
||||
json_col.insert(
|
||||
json_col.end(), new_json_col.begin(), new_json_col.end());
|
||||
auto new_valid_col = raw_data.get_col_valid(json_fid);
|
||||
valid_data.insert(
|
||||
valid_data.end(), new_valid_col.begin(), new_valid_col.end());
|
||||
seg->PreInsert(N);
|
||||
seg->Insert(iter * N,
|
||||
N,
|
||||
raw_data.row_ids_.data(),
|
||||
raw_data.timestamps_.data(),
|
||||
raw_data.raw_);
|
||||
}
|
||||
|
||||
auto seg_promote = dynamic_cast<SegmentGrowingImpl*>(seg.get());
|
||||
for (auto [clause, ref_func] : testcases) {
|
||||
auto loc = raw_plan_tmp.find("@@@@");
|
||||
auto raw_plan = raw_plan_tmp;
|
||||
raw_plan.replace(loc, 4, clause);
|
||||
auto plan_str = translate_text_plan_to_binary_plan(raw_plan.c_str());
|
||||
auto plan =
|
||||
CreateSearchPlanByExpr(*schema, plan_str.data(), plan_str.size());
|
||||
BitsetType final;
|
||||
final = ExecuteQueryExpr(
|
||||
plan->plan_node_->plannodes_->sources()[0]->sources()[0],
|
||||
seg_promote,
|
||||
N * num_iters,
|
||||
MAX_TIMESTAMP);
|
||||
EXPECT_EQ(final.size(), N * num_iters);
|
||||
|
||||
// specify some offsets and do scalar filtering on these offsets
|
||||
milvus::exec::OffsetVector offsets;
|
||||
offsets.reserve(N * num_iters / 2);
|
||||
for (auto i = 0; i < N * num_iters; ++i) {
|
||||
if (i % 2 == 0) {
|
||||
offsets.emplace_back(i);
|
||||
}
|
||||
}
|
||||
auto col_vec = milvus::test::gen_filter_res(
|
||||
plan->plan_node_->plannodes_->sources()[0]->sources()[0].get(),
|
||||
seg_promote,
|
||||
N * num_iters,
|
||||
MAX_TIMESTAMP,
|
||||
&offsets);
|
||||
BitsetTypeView view(col_vec->GetRawData(), col_vec->size());
|
||||
EXPECT_EQ(view.size(), N * num_iters / 2);
|
||||
for (int i = 0; i < N * num_iters; ++i) {
|
||||
auto ans = final[i];
|
||||
auto valid = valid_data[i];
|
||||
auto ref = ref_func(valid);
|
||||
ASSERT_EQ(ans, ref);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST_P(ExprTest, TestTermWithJSON) {
|
||||
std::vector<
|
||||
std::tuple<std::string,
|
||||
|
|
|
@ -1109,6 +1109,113 @@ TEST(StringExpr, UnaryRangeNullable) {
|
|||
}
|
||||
}
|
||||
|
||||
TEST(StringExpr, NullExpr) {
|
||||
auto schema = std::make_shared<Schema>();
|
||||
schema->AddDebugField("str", DataType::VARCHAR, true);
|
||||
schema->AddDebugField("another_str", DataType::VARCHAR);
|
||||
schema->AddDebugField(
|
||||
"fvec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
|
||||
auto pk = schema->AddDebugField("int64", DataType::INT64);
|
||||
schema->set_primary_field_id(pk);
|
||||
const auto& fvec_meta = schema->operator[](FieldName("fvec"));
|
||||
const auto& str_meta = schema->operator[](FieldName("str"));
|
||||
|
||||
auto gen_plan =
|
||||
[&, fvec_meta, str_meta](
|
||||
NullExprType op) -> std::unique_ptr<proto::plan::PlanNode> {
|
||||
auto column_info = test::GenColumnInfo(str_meta.get_id().get(),
|
||||
proto::schema::DataType::VarChar,
|
||||
false,
|
||||
false,
|
||||
proto::schema::DataType::None,
|
||||
true);
|
||||
auto null_expr = test::GenNullExpr(op);
|
||||
null_expr->set_allocated_column_info(column_info);
|
||||
|
||||
auto expr = test::GenExpr().release();
|
||||
expr->set_allocated_null_expr(null_expr);
|
||||
|
||||
proto::plan::VectorType vector_type;
|
||||
if (fvec_meta.get_data_type() == DataType::VECTOR_FLOAT) {
|
||||
vector_type = proto::plan::VectorType::FloatVector;
|
||||
} else if (fvec_meta.get_data_type() == DataType::VECTOR_BINARY) {
|
||||
vector_type = proto::plan::VectorType::BinaryVector;
|
||||
} else if (fvec_meta.get_data_type() == DataType::VECTOR_FLOAT16) {
|
||||
vector_type = proto::plan::VectorType::Float16Vector;
|
||||
}
|
||||
auto anns = GenAnns(expr, vector_type, fvec_meta.get_id().get(), "$0");
|
||||
|
||||
auto plan_node = std::make_unique<proto::plan::PlanNode>();
|
||||
plan_node->set_allocated_vector_anns(anns);
|
||||
return plan_node;
|
||||
};
|
||||
|
||||
auto seg = CreateGrowingSegment(schema, empty_index_meta);
|
||||
int N = 1000;
|
||||
std::vector<std::string> str_col;
|
||||
FixedVector<bool> valid_data;
|
||||
int num_iters = 100;
|
||||
for (int iter = 0; iter < num_iters; ++iter) {
|
||||
auto raw_data = DataGen(schema, N, iter);
|
||||
auto new_str_col = raw_data.get_col(str_meta.get_id());
|
||||
auto begin = FIELD_DATA(new_str_col, string).begin();
|
||||
auto end = FIELD_DATA(new_str_col, string).end();
|
||||
str_col.insert(str_col.end(), begin, end);
|
||||
auto new_str_valid_col = raw_data.get_col_valid(str_meta.get_id());
|
||||
valid_data.insert(valid_data.end(),
|
||||
new_str_valid_col.begin(),
|
||||
new_str_valid_col.end());
|
||||
seg->PreInsert(N);
|
||||
seg->Insert(iter * N,
|
||||
N,
|
||||
raw_data.row_ids_.data(),
|
||||
raw_data.timestamps_.data(),
|
||||
raw_data.raw_);
|
||||
}
|
||||
std::vector<NullExprType> ops{NullExprType::NullExpr_NullOp_IsNull,
|
||||
NullExprType::NullExpr_NullOp_IsNotNull};
|
||||
|
||||
auto seg_promote = dynamic_cast<SegmentGrowingImpl*>(seg.get());
|
||||
// is_null
|
||||
for (const auto op : ops) {
|
||||
auto plan_proto = gen_plan(op);
|
||||
auto plan = ProtoParser(*schema).CreatePlan(*plan_proto);
|
||||
BitsetType final;
|
||||
final = ExecuteQueryExpr(
|
||||
plan->plan_node_->plannodes_->sources()[0]->sources()[0],
|
||||
seg_promote,
|
||||
N * num_iters,
|
||||
MAX_TIMESTAMP);
|
||||
EXPECT_EQ(final.size(), N * num_iters);
|
||||
|
||||
// specify some offsets and do scalar filtering on these offsets
|
||||
milvus::exec::OffsetVector offsets;
|
||||
offsets.reserve(N * num_iters / 2);
|
||||
for (auto i = 0; i < N * num_iters; ++i) {
|
||||
if (i % 2 == 0) {
|
||||
offsets.emplace_back(i);
|
||||
}
|
||||
}
|
||||
auto col_vec = milvus::test::gen_filter_res(
|
||||
plan->plan_node_->plannodes_->sources()[0]->sources()[0].get(),
|
||||
seg_promote,
|
||||
N * num_iters,
|
||||
MAX_TIMESTAMP,
|
||||
&offsets);
|
||||
BitsetTypeView view(col_vec->GetRawData(), col_vec->size());
|
||||
EXPECT_EQ(view.size(), N * num_iters / 2);
|
||||
|
||||
for (int i = 0; i < N * num_iters; ++i) {
|
||||
auto ans = final[i];
|
||||
if (op == NullExprType::NullExpr_NullOp_IsNull) {
|
||||
ASSERT_EQ(ans, !valid_data[i]);
|
||||
} else {
|
||||
ASSERT_EQ(ans, valid_data[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(StringExpr, BinaryRange) {
|
||||
auto schema = GenTestSchema();
|
||||
const auto& fvec_meta = schema->operator[](FieldName("fvec"));
|
||||
|
|
|
@ -27,13 +27,15 @@ GenColumnInfo(
|
|||
proto::schema::DataType field_type,
|
||||
bool auto_id,
|
||||
bool is_pk,
|
||||
proto::schema::DataType element_type = proto::schema::DataType::None) {
|
||||
proto::schema::DataType element_type = proto::schema::DataType::None,
|
||||
bool nullable = false) {
|
||||
auto column_info = new proto::plan::ColumnInfo();
|
||||
column_info->set_field_id(field_id);
|
||||
column_info->set_data_type(field_type);
|
||||
column_info->set_is_autoid(auto_id);
|
||||
column_info->set_is_primary_key(is_pk);
|
||||
column_info->set_element_type(element_type);
|
||||
column_info->set_nullable(nullable);
|
||||
return column_info;
|
||||
}
|
||||
|
||||
|
@ -65,6 +67,13 @@ GenUnaryRangeExpr(proto::plan::OpType op, T& value) {
|
|||
return unary_range_expr;
|
||||
}
|
||||
|
||||
inline auto
|
||||
GenNullExpr(NullExprType op) {
|
||||
auto null_expr = new proto::plan::NullExpr();
|
||||
null_expr->set_op(op);
|
||||
return null_expr;
|
||||
}
|
||||
|
||||
inline auto
|
||||
GenExpr() {
|
||||
return std::make_unique<proto::plan::Expr>();
|
||||
|
|
|
@ -34,6 +34,8 @@ expr:
|
|||
| expr BOR expr # BitOr
|
||||
| expr AND expr # LogicalAnd
|
||||
| expr OR expr # LogicalOr
|
||||
| Identifier ISNULL # IsNull
|
||||
| Identifier ISNOTNULL # IsNotNull
|
||||
| EXISTS expr # Exists;
|
||||
|
||||
// typeName: ty = (BOOL | INT8 | INT16 | INT32 | INT64 | FLOAT | DOUBLE);
|
||||
|
@ -74,6 +76,9 @@ BXOR: '^';
|
|||
AND: '&&' | 'and';
|
||||
OR: '||' | 'or';
|
||||
|
||||
ISNULL: 'is null' | 'IS NULL';
|
||||
ISNOTNULL: 'is not null' | 'IS NOT NULL';
|
||||
|
||||
BNOT: '~';
|
||||
NOT: '!' | 'not';
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ import (
|
|||
)
|
||||
|
||||
func TestCheckIdentical(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
|
File diff suppressed because one or more lines are too long
|
@ -27,26 +27,28 @@ BOR=26
|
|||
BXOR=27
|
||||
AND=28
|
||||
OR=29
|
||||
BNOT=30
|
||||
NOT=31
|
||||
IN=32
|
||||
EmptyArray=33
|
||||
JSONContains=34
|
||||
JSONContainsAll=35
|
||||
JSONContainsAny=36
|
||||
ArrayContains=37
|
||||
ArrayContainsAll=38
|
||||
ArrayContainsAny=39
|
||||
ArrayLength=40
|
||||
BooleanConstant=41
|
||||
IntegerConstant=42
|
||||
FloatingConstant=43
|
||||
Identifier=44
|
||||
Meta=45
|
||||
StringLiteral=46
|
||||
JSONIdentifier=47
|
||||
Whitespace=48
|
||||
Newline=49
|
||||
ISNULL=30
|
||||
ISNOTNULL=31
|
||||
BNOT=32
|
||||
NOT=33
|
||||
IN=34
|
||||
EmptyArray=35
|
||||
JSONContains=36
|
||||
JSONContainsAll=37
|
||||
JSONContainsAny=38
|
||||
ArrayContains=39
|
||||
ArrayContainsAll=40
|
||||
ArrayContainsAny=41
|
||||
ArrayLength=42
|
||||
BooleanConstant=43
|
||||
IntegerConstant=44
|
||||
FloatingConstant=45
|
||||
Identifier=46
|
||||
Meta=47
|
||||
StringLiteral=48
|
||||
JSONIdentifier=49
|
||||
Whitespace=50
|
||||
Newline=51
|
||||
'('=1
|
||||
')'=2
|
||||
'['=3
|
||||
|
@ -71,5 +73,5 @@ Newline=49
|
|||
'&'=25
|
||||
'|'=26
|
||||
'^'=27
|
||||
'~'=30
|
||||
'$meta'=45
|
||||
'~'=32
|
||||
'$meta'=47
|
||||
|
|
File diff suppressed because one or more lines are too long
|
@ -27,26 +27,28 @@ BOR=26
|
|||
BXOR=27
|
||||
AND=28
|
||||
OR=29
|
||||
BNOT=30
|
||||
NOT=31
|
||||
IN=32
|
||||
EmptyArray=33
|
||||
JSONContains=34
|
||||
JSONContainsAll=35
|
||||
JSONContainsAny=36
|
||||
ArrayContains=37
|
||||
ArrayContainsAll=38
|
||||
ArrayContainsAny=39
|
||||
ArrayLength=40
|
||||
BooleanConstant=41
|
||||
IntegerConstant=42
|
||||
FloatingConstant=43
|
||||
Identifier=44
|
||||
Meta=45
|
||||
StringLiteral=46
|
||||
JSONIdentifier=47
|
||||
Whitespace=48
|
||||
Newline=49
|
||||
ISNULL=30
|
||||
ISNOTNULL=31
|
||||
BNOT=32
|
||||
NOT=33
|
||||
IN=34
|
||||
EmptyArray=35
|
||||
JSONContains=36
|
||||
JSONContainsAll=37
|
||||
JSONContainsAny=38
|
||||
ArrayContains=39
|
||||
ArrayContainsAll=40
|
||||
ArrayContainsAny=41
|
||||
ArrayLength=42
|
||||
BooleanConstant=43
|
||||
IntegerConstant=44
|
||||
FloatingConstant=45
|
||||
Identifier=46
|
||||
Meta=47
|
||||
StringLiteral=48
|
||||
JSONIdentifier=49
|
||||
Whitespace=50
|
||||
Newline=51
|
||||
'('=1
|
||||
')'=2
|
||||
'['=3
|
||||
|
@ -71,5 +73,5 @@ Newline=49
|
|||
'&'=25
|
||||
'|'=26
|
||||
'^'=27
|
||||
'~'=30
|
||||
'$meta'=45
|
||||
'~'=32
|
||||
'$meta'=47
|
||||
|
|
|
@ -31,6 +31,10 @@ func (v *BasePlanVisitor) VisitLogicalOr(ctx *LogicalOrContext) interface{} {
|
|||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BasePlanVisitor) VisitIsNotNull(ctx *IsNotNullContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BasePlanVisitor) VisitMulDivMod(ctx *MulDivModContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
@ -135,6 +139,10 @@ func (v *BasePlanVisitor) VisitBitAnd(ctx *BitAndContext) interface{} {
|
|||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BasePlanVisitor) VisitIsNull(ctx *IsNullContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BasePlanVisitor) VisitPower(ctx *PowerContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
|
|
@ -45,37 +45,38 @@ func planlexerLexerInit() {
|
|||
staticData.LiteralNames = []string{
|
||||
"", "'('", "')'", "'['", "','", "']'", "'{'", "'}'", "'<'", "'<='",
|
||||
"'>'", "'>='", "'=='", "'!='", "", "", "", "'+'", "'-'", "'*'", "'/'",
|
||||
"'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "'~'", "",
|
||||
"", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'",
|
||||
"'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "", "",
|
||||
"'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'",
|
||||
}
|
||||
staticData.SymbolicNames = []string{
|
||||
"", "", "", "", "", "", "LBRACE", "RBRACE", "LT", "LE", "GT", "GE",
|
||||
"EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "ADD", "SUB", "MUL", "DIV",
|
||||
"MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "BNOT",
|
||||
"NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll", "JSONContainsAny",
|
||||
"ArrayContains", "ArrayContainsAll", "ArrayContainsAny", "ArrayLength",
|
||||
"BooleanConstant", "IntegerConstant", "FloatingConstant", "Identifier",
|
||||
"Meta", "StringLiteral", "JSONIdentifier", "Whitespace", "Newline",
|
||||
"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", "ADD", "SUB",
|
||||
"MUL", "DIV", "MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND",
|
||||
"OR", "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",
|
||||
"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, 49, 782, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2,
|
||||
4, 0, 51, 826, 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,
|
||||
|
@ -89,346 +90,366 @@ 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, 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, 188, 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, 202, 8, 14, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15,
|
||||
73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 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,
|
||||
192, 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, 206, 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, 3, 15, 224, 8, 15, 1, 16, 1, 16, 1, 17, 1, 17, 1, 18, 1, 18,
|
||||
1, 19, 1, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1,
|
||||
23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 25, 1, 25, 1, 26, 1, 26, 1, 27, 1, 27,
|
||||
1, 27, 1, 27, 1, 27, 3, 27, 256, 8, 27, 1, 28, 1, 28, 1, 28, 1, 28, 3,
|
||||
28, 262, 8, 28, 1, 29, 1, 29, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 270, 8,
|
||||
30, 1, 31, 1, 31, 1, 31, 1, 31, 3, 31, 276, 8, 31, 1, 32, 1, 32, 1, 32,
|
||||
5, 32, 281, 8, 32, 10, 32, 12, 32, 284, 9, 32, 1, 32, 1, 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, 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, 314, 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, 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, 350,
|
||||
8, 34, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1,
|
||||
35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35,
|
||||
15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 3, 15, 228, 8, 15, 1, 16, 1, 16,
|
||||
1, 17, 1, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1,
|
||||
21, 1, 22, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 25, 1, 25,
|
||||
1, 26, 1, 26, 1, 27, 1, 27, 1, 27, 1, 27, 1, 27, 3, 27, 260, 8, 27, 1,
|
||||
28, 1, 28, 1, 28, 1, 28, 3, 28, 266, 8, 28, 1, 29, 1, 29, 1, 29, 1, 29,
|
||||
1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 1, 29, 3,
|
||||
29, 282, 8, 29, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30,
|
||||
1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1, 30, 1,
|
||||
30, 1, 30, 1, 30, 1, 30, 3, 30, 306, 8, 30, 1, 31, 1, 31, 1, 32, 1, 32,
|
||||
1, 32, 1, 32, 3, 32, 314, 8, 32, 1, 33, 1, 33, 1, 33, 1, 33, 3, 33, 320,
|
||||
8, 33, 1, 34, 1, 34, 1, 34, 5, 34, 325, 8, 34, 10, 34, 12, 34, 328, 9,
|
||||
34, 1, 34, 1, 34, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35,
|
||||
1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1,
|
||||
35, 1, 35, 1, 35, 1, 35, 3, 35, 386, 8, 35, 1, 36, 1, 36, 1, 36, 1, 36,
|
||||
35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 1, 35, 3, 35, 358, 8, 35,
|
||||
1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1,
|
||||
36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36,
|
||||
1, 36, 1, 36, 1, 36, 3, 36, 416, 8, 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, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1, 36, 1,
|
||||
36, 1, 36, 1, 36, 3, 36, 394, 8, 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, 454, 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, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 1, 37, 3, 37, 430, 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, 3, 38, 492, 8, 38,
|
||||
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, 39,
|
||||
1, 39, 1, 39, 1, 39, 3, 39, 518, 8, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1,
|
||||
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, 498, 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, 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, 547, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 553, 8,
|
||||
41, 1, 42, 1, 42, 3, 42, 557, 8, 42, 1, 43, 1, 43, 1, 43, 5, 43, 562, 8,
|
||||
43, 10, 43, 12, 43, 565, 9, 43, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44,
|
||||
1, 45, 3, 45, 574, 8, 45, 1, 45, 1, 45, 3, 45, 578, 8, 45, 1, 45, 1, 45,
|
||||
1, 45, 3, 45, 583, 8, 45, 1, 45, 3, 45, 586, 8, 45, 1, 46, 1, 46, 3, 46,
|
||||
590, 8, 46, 1, 46, 1, 46, 1, 46, 3, 46, 595, 8, 46, 1, 46, 1, 46, 4, 46,
|
||||
599, 8, 46, 11, 46, 12, 46, 600, 1, 47, 1, 47, 1, 47, 3, 47, 606, 8, 47,
|
||||
1, 48, 4, 48, 609, 8, 48, 11, 48, 12, 48, 610, 1, 49, 4, 49, 614, 8, 49,
|
||||
11, 49, 12, 49, 615, 1, 50, 1, 50, 1, 50, 1, 50, 1, 50, 1, 50, 1, 50, 3,
|
||||
50, 625, 8, 50, 1, 51, 1, 51, 1, 51, 1, 51, 1, 51, 1, 51, 1, 51, 3, 51,
|
||||
634, 8, 51, 1, 52, 1, 52, 1, 53, 1, 53, 1, 54, 1, 54, 1, 54, 4, 54, 643,
|
||||
8, 54, 11, 54, 12, 54, 644, 1, 55, 1, 55, 5, 55, 649, 8, 55, 10, 55, 12,
|
||||
55, 652, 9, 55, 1, 55, 3, 55, 655, 8, 55, 1, 56, 1, 56, 5, 56, 659, 8,
|
||||
56, 10, 56, 12, 56, 662, 9, 56, 1, 57, 1, 57, 1, 57, 1, 57, 1, 58, 1, 58,
|
||||
1, 59, 1, 59, 1, 60, 1, 60, 1, 61, 1, 61, 1, 61, 1, 61, 1, 61, 1, 62, 1,
|
||||
62, 1, 62, 1, 62, 1, 62, 1, 62, 1, 62, 1, 62, 1, 62, 1, 62, 3, 62, 689,
|
||||
8, 62, 1, 63, 1, 63, 3, 63, 693, 8, 63, 1, 63, 1, 63, 1, 63, 3, 63, 698,
|
||||
8, 63, 1, 64, 1, 64, 1, 64, 1, 64, 3, 64, 704, 8, 64, 1, 64, 1, 64, 1,
|
||||
65, 3, 65, 709, 8, 65, 1, 65, 1, 65, 1, 65, 1, 65, 1, 65, 3, 65, 716, 8,
|
||||
65, 1, 66, 1, 66, 3, 66, 720, 8, 66, 1, 66, 1, 66, 1, 67, 4, 67, 725, 8,
|
||||
67, 11, 67, 12, 67, 726, 1, 68, 3, 68, 730, 8, 68, 1, 68, 1, 68, 1, 68,
|
||||
1, 68, 1, 68, 3, 68, 737, 8, 68, 1, 69, 4, 69, 740, 8, 69, 11, 69, 12,
|
||||
69, 741, 1, 70, 1, 70, 3, 70, 746, 8, 70, 1, 70, 1, 70, 1, 71, 1, 71, 1,
|
||||
71, 1, 71, 1, 71, 3, 71, 755, 8, 71, 1, 71, 3, 71, 758, 8, 71, 1, 71, 1,
|
||||
71, 1, 71, 1, 71, 1, 71, 3, 71, 765, 8, 71, 1, 72, 4, 72, 768, 8, 72, 11,
|
||||
72, 12, 72, 769, 1, 72, 1, 72, 1, 73, 1, 73, 3, 73, 776, 8, 73, 1, 73,
|
||||
3, 73, 779, 8, 73, 1, 73, 1, 73, 0, 0, 74, 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, 0, 97, 0, 99, 0, 101, 0,
|
||||
103, 0, 105, 0, 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, 48, 147, 49, 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,
|
||||
823, 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, 145, 1, 0, 0, 0, 0, 147, 1, 0, 0, 0, 1, 149, 1,
|
||||
0, 0, 0, 3, 151, 1, 0, 0, 0, 5, 153, 1, 0, 0, 0, 7, 155, 1, 0, 0, 0, 9,
|
||||
157, 1, 0, 0, 0, 11, 159, 1, 0, 0, 0, 13, 161, 1, 0, 0, 0, 15, 163, 1,
|
||||
0, 0, 0, 17, 165, 1, 0, 0, 0, 19, 168, 1, 0, 0, 0, 21, 170, 1, 0, 0, 0,
|
||||
23, 173, 1, 0, 0, 0, 25, 176, 1, 0, 0, 0, 27, 187, 1, 0, 0, 0, 29, 201,
|
||||
1, 0, 0, 0, 31, 223, 1, 0, 0, 0, 33, 225, 1, 0, 0, 0, 35, 227, 1, 0, 0,
|
||||
0, 37, 229, 1, 0, 0, 0, 39, 231, 1, 0, 0, 0, 41, 233, 1, 0, 0, 0, 43, 235,
|
||||
1, 0, 0, 0, 45, 238, 1, 0, 0, 0, 47, 241, 1, 0, 0, 0, 49, 244, 1, 0, 0,
|
||||
0, 51, 246, 1, 0, 0, 0, 53, 248, 1, 0, 0, 0, 55, 255, 1, 0, 0, 0, 57, 261,
|
||||
1, 0, 0, 0, 59, 263, 1, 0, 0, 0, 61, 269, 1, 0, 0, 0, 63, 275, 1, 0, 0,
|
||||
0, 65, 277, 1, 0, 0, 0, 67, 313, 1, 0, 0, 0, 69, 349, 1, 0, 0, 0, 71, 385,
|
||||
1, 0, 0, 0, 73, 415, 1, 0, 0, 0, 75, 453, 1, 0, 0, 0, 77, 491, 1, 0, 0,
|
||||
0, 79, 517, 1, 0, 0, 0, 81, 546, 1, 0, 0, 0, 83, 552, 1, 0, 0, 0, 85, 556,
|
||||
1, 0, 0, 0, 87, 558, 1, 0, 0, 0, 89, 566, 1, 0, 0, 0, 91, 573, 1, 0, 0,
|
||||
0, 93, 589, 1, 0, 0, 0, 95, 605, 1, 0, 0, 0, 97, 608, 1, 0, 0, 0, 99, 613,
|
||||
1, 0, 0, 0, 101, 624, 1, 0, 0, 0, 103, 633, 1, 0, 0, 0, 105, 635, 1, 0,
|
||||
0, 0, 107, 637, 1, 0, 0, 0, 109, 639, 1, 0, 0, 0, 111, 654, 1, 0, 0, 0,
|
||||
113, 656, 1, 0, 0, 0, 115, 663, 1, 0, 0, 0, 117, 667, 1, 0, 0, 0, 119,
|
||||
669, 1, 0, 0, 0, 121, 671, 1, 0, 0, 0, 123, 673, 1, 0, 0, 0, 125, 688,
|
||||
1, 0, 0, 0, 127, 697, 1, 0, 0, 0, 129, 699, 1, 0, 0, 0, 131, 715, 1, 0,
|
||||
0, 0, 133, 717, 1, 0, 0, 0, 135, 724, 1, 0, 0, 0, 137, 736, 1, 0, 0, 0,
|
||||
139, 739, 1, 0, 0, 0, 141, 743, 1, 0, 0, 0, 143, 764, 1, 0, 0, 0, 145,
|
||||
767, 1, 0, 0, 0, 147, 778, 1, 0, 0, 0, 149, 150, 5, 40, 0, 0, 150, 2, 1,
|
||||
0, 0, 0, 151, 152, 5, 41, 0, 0, 152, 4, 1, 0, 0, 0, 153, 154, 5, 91, 0,
|
||||
0, 154, 6, 1, 0, 0, 0, 155, 156, 5, 44, 0, 0, 156, 8, 1, 0, 0, 0, 157,
|
||||
158, 5, 93, 0, 0, 158, 10, 1, 0, 0, 0, 159, 160, 5, 123, 0, 0, 160, 12,
|
||||
1, 0, 0, 0, 161, 162, 5, 125, 0, 0, 162, 14, 1, 0, 0, 0, 163, 164, 5, 60,
|
||||
0, 0, 164, 16, 1, 0, 0, 0, 165, 166, 5, 60, 0, 0, 166, 167, 5, 61, 0, 0,
|
||||
167, 18, 1, 0, 0, 0, 168, 169, 5, 62, 0, 0, 169, 20, 1, 0, 0, 0, 170, 171,
|
||||
5, 62, 0, 0, 171, 172, 5, 61, 0, 0, 172, 22, 1, 0, 0, 0, 173, 174, 5, 61,
|
||||
0, 0, 174, 175, 5, 61, 0, 0, 175, 24, 1, 0, 0, 0, 176, 177, 5, 33, 0, 0,
|
||||
177, 178, 5, 61, 0, 0, 178, 26, 1, 0, 0, 0, 179, 180, 5, 108, 0, 0, 180,
|
||||
181, 5, 105, 0, 0, 181, 182, 5, 107, 0, 0, 182, 188, 5, 101, 0, 0, 183,
|
||||
184, 5, 76, 0, 0, 184, 185, 5, 73, 0, 0, 185, 186, 5, 75, 0, 0, 186, 188,
|
||||
5, 69, 0, 0, 187, 179, 1, 0, 0, 0, 187, 183, 1, 0, 0, 0, 188, 28, 1, 0,
|
||||
0, 0, 189, 190, 5, 101, 0, 0, 190, 191, 5, 120, 0, 0, 191, 192, 5, 105,
|
||||
0, 0, 192, 193, 5, 115, 0, 0, 193, 194, 5, 116, 0, 0, 194, 202, 5, 115,
|
||||
0, 0, 195, 196, 5, 69, 0, 0, 196, 197, 5, 88, 0, 0, 197, 198, 5, 73, 0,
|
||||
0, 198, 199, 5, 83, 0, 0, 199, 200, 5, 84, 0, 0, 200, 202, 5, 83, 0, 0,
|
||||
201, 189, 1, 0, 0, 0, 201, 195, 1, 0, 0, 0, 202, 30, 1, 0, 0, 0, 203, 204,
|
||||
5, 116, 0, 0, 204, 205, 5, 101, 0, 0, 205, 206, 5, 120, 0, 0, 206, 207,
|
||||
5, 116, 0, 0, 207, 208, 5, 95, 0, 0, 208, 209, 5, 109, 0, 0, 209, 210,
|
||||
5, 97, 0, 0, 210, 211, 5, 116, 0, 0, 211, 212, 5, 99, 0, 0, 212, 224, 5,
|
||||
104, 0, 0, 213, 214, 5, 84, 0, 0, 214, 215, 5, 69, 0, 0, 215, 216, 5, 88,
|
||||
0, 0, 216, 217, 5, 84, 0, 0, 217, 218, 5, 95, 0, 0, 218, 219, 5, 77, 0,
|
||||
0, 219, 220, 5, 65, 0, 0, 220, 221, 5, 84, 0, 0, 221, 222, 5, 67, 0, 0,
|
||||
222, 224, 5, 72, 0, 0, 223, 203, 1, 0, 0, 0, 223, 213, 1, 0, 0, 0, 224,
|
||||
32, 1, 0, 0, 0, 225, 226, 5, 43, 0, 0, 226, 34, 1, 0, 0, 0, 227, 228, 5,
|
||||
45, 0, 0, 228, 36, 1, 0, 0, 0, 229, 230, 5, 42, 0, 0, 230, 38, 1, 0, 0,
|
||||
0, 231, 232, 5, 47, 0, 0, 232, 40, 1, 0, 0, 0, 233, 234, 5, 37, 0, 0, 234,
|
||||
42, 1, 0, 0, 0, 235, 236, 5, 42, 0, 0, 236, 237, 5, 42, 0, 0, 237, 44,
|
||||
1, 0, 0, 0, 238, 239, 5, 60, 0, 0, 239, 240, 5, 60, 0, 0, 240, 46, 1, 0,
|
||||
0, 0, 241, 242, 5, 62, 0, 0, 242, 243, 5, 62, 0, 0, 243, 48, 1, 0, 0, 0,
|
||||
244, 245, 5, 38, 0, 0, 245, 50, 1, 0, 0, 0, 246, 247, 5, 124, 0, 0, 247,
|
||||
52, 1, 0, 0, 0, 248, 249, 5, 94, 0, 0, 249, 54, 1, 0, 0, 0, 250, 251, 5,
|
||||
38, 0, 0, 251, 256, 5, 38, 0, 0, 252, 253, 5, 97, 0, 0, 253, 254, 5, 110,
|
||||
0, 0, 254, 256, 5, 100, 0, 0, 255, 250, 1, 0, 0, 0, 255, 252, 1, 0, 0,
|
||||
0, 256, 56, 1, 0, 0, 0, 257, 258, 5, 124, 0, 0, 258, 262, 5, 124, 0, 0,
|
||||
259, 260, 5, 111, 0, 0, 260, 262, 5, 114, 0, 0, 261, 257, 1, 0, 0, 0, 261,
|
||||
259, 1, 0, 0, 0, 262, 58, 1, 0, 0, 0, 263, 264, 5, 126, 0, 0, 264, 60,
|
||||
1, 0, 0, 0, 265, 270, 5, 33, 0, 0, 266, 267, 5, 110, 0, 0, 267, 268, 5,
|
||||
111, 0, 0, 268, 270, 5, 116, 0, 0, 269, 265, 1, 0, 0, 0, 269, 266, 1, 0,
|
||||
0, 0, 270, 62, 1, 0, 0, 0, 271, 272, 5, 105, 0, 0, 272, 276, 5, 110, 0,
|
||||
0, 273, 274, 5, 73, 0, 0, 274, 276, 5, 78, 0, 0, 275, 271, 1, 0, 0, 0,
|
||||
275, 273, 1, 0, 0, 0, 276, 64, 1, 0, 0, 0, 277, 282, 5, 91, 0, 0, 278,
|
||||
281, 3, 145, 72, 0, 279, 281, 3, 147, 73, 0, 280, 278, 1, 0, 0, 0, 280,
|
||||
279, 1, 0, 0, 0, 281, 284, 1, 0, 0, 0, 282, 280, 1, 0, 0, 0, 282, 283,
|
||||
1, 0, 0, 0, 283, 285, 1, 0, 0, 0, 284, 282, 1, 0, 0, 0, 285, 286, 5, 93,
|
||||
0, 0, 286, 66, 1, 0, 0, 0, 287, 288, 5, 106, 0, 0, 288, 289, 5, 115, 0,
|
||||
0, 289, 290, 5, 111, 0, 0, 290, 291, 5, 110, 0, 0, 291, 292, 5, 95, 0,
|
||||
0, 292, 293, 5, 99, 0, 0, 293, 294, 5, 111, 0, 0, 294, 295, 5, 110, 0,
|
||||
0, 295, 296, 5, 116, 0, 0, 296, 297, 5, 97, 0, 0, 297, 298, 5, 105, 0,
|
||||
0, 298, 299, 5, 110, 0, 0, 299, 314, 5, 115, 0, 0, 300, 301, 5, 74, 0,
|
||||
0, 301, 302, 5, 83, 0, 0, 302, 303, 5, 79, 0, 0, 303, 304, 5, 78, 0, 0,
|
||||
304, 305, 5, 95, 0, 0, 305, 306, 5, 67, 0, 0, 306, 307, 5, 79, 0, 0, 307,
|
||||
308, 5, 78, 0, 0, 308, 309, 5, 84, 0, 0, 309, 310, 5, 65, 0, 0, 310, 311,
|
||||
5, 73, 0, 0, 311, 312, 5, 78, 0, 0, 312, 314, 5, 83, 0, 0, 313, 287, 1,
|
||||
0, 0, 0, 313, 300, 1, 0, 0, 0, 314, 68, 1, 0, 0, 0, 315, 316, 5, 106, 0,
|
||||
0, 316, 317, 5, 115, 0, 0, 317, 318, 5, 111, 0, 0, 318, 319, 5, 110, 0,
|
||||
0, 319, 320, 5, 95, 0, 0, 320, 321, 5, 99, 0, 0, 321, 322, 5, 111, 0, 0,
|
||||
322, 323, 5, 110, 0, 0, 323, 324, 5, 116, 0, 0, 324, 325, 5, 97, 0, 0,
|
||||
325, 326, 5, 105, 0, 0, 326, 327, 5, 110, 0, 0, 327, 328, 5, 115, 0, 0,
|
||||
328, 329, 5, 95, 0, 0, 329, 330, 5, 97, 0, 0, 330, 331, 5, 108, 0, 0, 331,
|
||||
350, 5, 108, 0, 0, 332, 333, 5, 74, 0, 0, 333, 334, 5, 83, 0, 0, 334, 335,
|
||||
5, 79, 0, 0, 335, 336, 5, 78, 0, 0, 336, 337, 5, 95, 0, 0, 337, 338, 5,
|
||||
67, 0, 0, 338, 339, 5, 79, 0, 0, 339, 340, 5, 78, 0, 0, 340, 341, 5, 84,
|
||||
0, 0, 341, 342, 5, 65, 0, 0, 342, 343, 5, 73, 0, 0, 343, 344, 5, 78, 0,
|
||||
0, 344, 345, 5, 83, 0, 0, 345, 346, 5, 95, 0, 0, 346, 347, 5, 65, 0, 0,
|
||||
347, 348, 5, 76, 0, 0, 348, 350, 5, 76, 0, 0, 349, 315, 1, 0, 0, 0, 349,
|
||||
332, 1, 0, 0, 0, 350, 70, 1, 0, 0, 0, 351, 352, 5, 106, 0, 0, 352, 353,
|
||||
5, 115, 0, 0, 353, 354, 5, 111, 0, 0, 354, 355, 5, 110, 0, 0, 355, 356,
|
||||
5, 95, 0, 0, 356, 357, 5, 99, 0, 0, 357, 358, 5, 111, 0, 0, 358, 359, 5,
|
||||
110, 0, 0, 359, 360, 5, 116, 0, 0, 360, 361, 5, 97, 0, 0, 361, 362, 5,
|
||||
105, 0, 0, 362, 363, 5, 110, 0, 0, 363, 364, 5, 115, 0, 0, 364, 365, 5,
|
||||
95, 0, 0, 365, 366, 5, 97, 0, 0, 366, 367, 5, 110, 0, 0, 367, 386, 5, 121,
|
||||
0, 0, 368, 369, 5, 74, 0, 0, 369, 370, 5, 83, 0, 0, 370, 371, 5, 79, 0,
|
||||
0, 371, 372, 5, 78, 0, 0, 372, 373, 5, 95, 0, 0, 373, 374, 5, 67, 0, 0,
|
||||
374, 375, 5, 79, 0, 0, 375, 376, 5, 78, 0, 0, 376, 377, 5, 84, 0, 0, 377,
|
||||
378, 5, 65, 0, 0, 378, 379, 5, 73, 0, 0, 379, 380, 5, 78, 0, 0, 380, 381,
|
||||
5, 83, 0, 0, 381, 382, 5, 95, 0, 0, 382, 383, 5, 65, 0, 0, 383, 384, 5,
|
||||
78, 0, 0, 384, 386, 5, 89, 0, 0, 385, 351, 1, 0, 0, 0, 385, 368, 1, 0,
|
||||
0, 0, 386, 72, 1, 0, 0, 0, 387, 388, 5, 97, 0, 0, 388, 389, 5, 114, 0,
|
||||
0, 389, 390, 5, 114, 0, 0, 390, 391, 5, 97, 0, 0, 391, 392, 5, 121, 0,
|
||||
0, 392, 393, 5, 95, 0, 0, 393, 394, 5, 99, 0, 0, 394, 395, 5, 111, 0, 0,
|
||||
395, 396, 5, 110, 0, 0, 396, 397, 5, 116, 0, 0, 397, 398, 5, 97, 0, 0,
|
||||
398, 399, 5, 105, 0, 0, 399, 400, 5, 110, 0, 0, 400, 416, 5, 115, 0, 0,
|
||||
401, 402, 5, 65, 0, 0, 402, 403, 5, 82, 0, 0, 403, 404, 5, 82, 0, 0, 404,
|
||||
405, 5, 65, 0, 0, 405, 406, 5, 89, 0, 0, 406, 407, 5, 95, 0, 0, 407, 408,
|
||||
5, 67, 0, 0, 408, 409, 5, 79, 0, 0, 409, 410, 5, 78, 0, 0, 410, 411, 5,
|
||||
84, 0, 0, 411, 412, 5, 65, 0, 0, 412, 413, 5, 73, 0, 0, 413, 414, 5, 78,
|
||||
0, 0, 414, 416, 5, 83, 0, 0, 415, 387, 1, 0, 0, 0, 415, 401, 1, 0, 0, 0,
|
||||
416, 74, 1, 0, 0, 0, 417, 418, 5, 97, 0, 0, 418, 419, 5, 114, 0, 0, 419,
|
||||
420, 5, 114, 0, 0, 420, 421, 5, 97, 0, 0, 421, 422, 5, 121, 0, 0, 422,
|
||||
423, 5, 95, 0, 0, 423, 424, 5, 99, 0, 0, 424, 425, 5, 111, 0, 0, 425, 426,
|
||||
5, 110, 0, 0, 426, 427, 5, 116, 0, 0, 427, 428, 5, 97, 0, 0, 428, 429,
|
||||
5, 105, 0, 0, 429, 430, 5, 110, 0, 0, 430, 431, 5, 115, 0, 0, 431, 432,
|
||||
5, 95, 0, 0, 432, 433, 5, 97, 0, 0, 433, 434, 5, 108, 0, 0, 434, 454, 5,
|
||||
108, 0, 0, 435, 436, 5, 65, 0, 0, 436, 437, 5, 82, 0, 0, 437, 438, 5, 82,
|
||||
0, 0, 438, 439, 5, 65, 0, 0, 439, 440, 5, 89, 0, 0, 440, 441, 5, 95, 0,
|
||||
0, 441, 442, 5, 67, 0, 0, 442, 443, 5, 79, 0, 0, 443, 444, 5, 78, 0, 0,
|
||||
444, 445, 5, 84, 0, 0, 445, 446, 5, 65, 0, 0, 446, 447, 5, 73, 0, 0, 447,
|
||||
448, 5, 78, 0, 0, 448, 449, 5, 83, 0, 0, 449, 450, 5, 95, 0, 0, 450, 451,
|
||||
5, 65, 0, 0, 451, 452, 5, 76, 0, 0, 452, 454, 5, 76, 0, 0, 453, 417, 1,
|
||||
0, 0, 0, 453, 435, 1, 0, 0, 0, 454, 76, 1, 0, 0, 0, 455, 456, 5, 97, 0,
|
||||
0, 456, 457, 5, 114, 0, 0, 457, 458, 5, 114, 0, 0, 458, 459, 5, 97, 0,
|
||||
0, 459, 460, 5, 121, 0, 0, 460, 461, 5, 95, 0, 0, 461, 462, 5, 99, 0, 0,
|
||||
462, 463, 5, 111, 0, 0, 463, 464, 5, 110, 0, 0, 464, 465, 5, 116, 0, 0,
|
||||
465, 466, 5, 97, 0, 0, 466, 467, 5, 105, 0, 0, 467, 468, 5, 110, 0, 0,
|
||||
468, 469, 5, 115, 0, 0, 469, 470, 5, 95, 0, 0, 470, 471, 5, 97, 0, 0, 471,
|
||||
472, 5, 110, 0, 0, 472, 492, 5, 121, 0, 0, 473, 474, 5, 65, 0, 0, 474,
|
||||
475, 5, 82, 0, 0, 475, 476, 5, 82, 0, 0, 476, 477, 5, 65, 0, 0, 477, 478,
|
||||
5, 89, 0, 0, 478, 479, 5, 95, 0, 0, 479, 480, 5, 67, 0, 0, 480, 481, 5,
|
||||
79, 0, 0, 481, 482, 5, 78, 0, 0, 482, 483, 5, 84, 0, 0, 483, 484, 5, 65,
|
||||
0, 0, 484, 485, 5, 73, 0, 0, 485, 486, 5, 78, 0, 0, 486, 487, 5, 83, 0,
|
||||
0, 487, 488, 5, 95, 0, 0, 488, 489, 5, 65, 0, 0, 489, 490, 5, 78, 0, 0,
|
||||
490, 492, 5, 89, 0, 0, 491, 455, 1, 0, 0, 0, 491, 473, 1, 0, 0, 0, 492,
|
||||
78, 1, 0, 0, 0, 493, 494, 5, 97, 0, 0, 494, 495, 5, 114, 0, 0, 495, 496,
|
||||
5, 114, 0, 0, 496, 497, 5, 97, 0, 0, 497, 498, 5, 121, 0, 0, 498, 499,
|
||||
5, 95, 0, 0, 499, 500, 5, 108, 0, 0, 500, 501, 5, 101, 0, 0, 501, 502,
|
||||
5, 110, 0, 0, 502, 503, 5, 103, 0, 0, 503, 504, 5, 116, 0, 0, 504, 518,
|
||||
5, 104, 0, 0, 505, 506, 5, 65, 0, 0, 506, 507, 5, 82, 0, 0, 507, 508, 5,
|
||||
82, 0, 0, 508, 509, 5, 65, 0, 0, 509, 510, 5, 89, 0, 0, 510, 511, 5, 95,
|
||||
0, 0, 511, 512, 5, 76, 0, 0, 512, 513, 5, 69, 0, 0, 513, 514, 5, 78, 0,
|
||||
0, 514, 515, 5, 71, 0, 0, 515, 516, 5, 84, 0, 0, 516, 518, 5, 72, 0, 0,
|
||||
517, 493, 1, 0, 0, 0, 517, 505, 1, 0, 0, 0, 518, 80, 1, 0, 0, 0, 519, 520,
|
||||
5, 116, 0, 0, 520, 521, 5, 114, 0, 0, 521, 522, 5, 117, 0, 0, 522, 547,
|
||||
5, 101, 0, 0, 523, 524, 5, 84, 0, 0, 524, 525, 5, 114, 0, 0, 525, 526,
|
||||
5, 117, 0, 0, 526, 547, 5, 101, 0, 0, 527, 528, 5, 84, 0, 0, 528, 529,
|
||||
5, 82, 0, 0, 529, 530, 5, 85, 0, 0, 530, 547, 5, 69, 0, 0, 531, 532, 5,
|
||||
102, 0, 0, 532, 533, 5, 97, 0, 0, 533, 534, 5, 108, 0, 0, 534, 535, 5,
|
||||
115, 0, 0, 535, 547, 5, 101, 0, 0, 536, 537, 5, 70, 0, 0, 537, 538, 5,
|
||||
97, 0, 0, 538, 539, 5, 108, 0, 0, 539, 540, 5, 115, 0, 0, 540, 547, 5,
|
||||
101, 0, 0, 541, 542, 5, 70, 0, 0, 542, 543, 5, 65, 0, 0, 543, 544, 5, 76,
|
||||
0, 0, 544, 545, 5, 83, 0, 0, 545, 547, 5, 69, 0, 0, 546, 519, 1, 0, 0,
|
||||
0, 546, 523, 1, 0, 0, 0, 546, 527, 1, 0, 0, 0, 546, 531, 1, 0, 0, 0, 546,
|
||||
536, 1, 0, 0, 0, 546, 541, 1, 0, 0, 0, 547, 82, 1, 0, 0, 0, 548, 553, 3,
|
||||
111, 55, 0, 549, 553, 3, 113, 56, 0, 550, 553, 3, 115, 57, 0, 551, 553,
|
||||
3, 109, 54, 0, 552, 548, 1, 0, 0, 0, 552, 549, 1, 0, 0, 0, 552, 550, 1,
|
||||
0, 0, 0, 552, 551, 1, 0, 0, 0, 553, 84, 1, 0, 0, 0, 554, 557, 3, 127, 63,
|
||||
0, 555, 557, 3, 129, 64, 0, 556, 554, 1, 0, 0, 0, 556, 555, 1, 0, 0, 0,
|
||||
557, 86, 1, 0, 0, 0, 558, 563, 3, 105, 52, 0, 559, 562, 3, 105, 52, 0,
|
||||
560, 562, 3, 107, 53, 0, 561, 559, 1, 0, 0, 0, 561, 560, 1, 0, 0, 0, 562,
|
||||
565, 1, 0, 0, 0, 563, 561, 1, 0, 0, 0, 563, 564, 1, 0, 0, 0, 564, 88, 1,
|
||||
0, 0, 0, 565, 563, 1, 0, 0, 0, 566, 567, 5, 36, 0, 0, 567, 568, 5, 109,
|
||||
0, 0, 568, 569, 5, 101, 0, 0, 569, 570, 5, 116, 0, 0, 570, 571, 5, 97,
|
||||
0, 0, 571, 90, 1, 0, 0, 0, 572, 574, 3, 95, 47, 0, 573, 572, 1, 0, 0, 0,
|
||||
573, 574, 1, 0, 0, 0, 574, 585, 1, 0, 0, 0, 575, 577, 5, 34, 0, 0, 576,
|
||||
578, 3, 97, 48, 0, 577, 576, 1, 0, 0, 0, 577, 578, 1, 0, 0, 0, 578, 579,
|
||||
1, 0, 0, 0, 579, 586, 5, 34, 0, 0, 580, 582, 5, 39, 0, 0, 581, 583, 3,
|
||||
99, 49, 0, 582, 581, 1, 0, 0, 0, 582, 583, 1, 0, 0, 0, 583, 584, 1, 0,
|
||||
0, 0, 584, 586, 5, 39, 0, 0, 585, 575, 1, 0, 0, 0, 585, 580, 1, 0, 0, 0,
|
||||
586, 92, 1, 0, 0, 0, 587, 590, 3, 87, 43, 0, 588, 590, 3, 89, 44, 0, 589,
|
||||
587, 1, 0, 0, 0, 589, 588, 1, 0, 0, 0, 590, 598, 1, 0, 0, 0, 591, 594,
|
||||
5, 91, 0, 0, 592, 595, 3, 91, 45, 0, 593, 595, 3, 111, 55, 0, 594, 592,
|
||||
1, 0, 0, 0, 594, 593, 1, 0, 0, 0, 595, 596, 1, 0, 0, 0, 596, 597, 5, 93,
|
||||
0, 0, 597, 599, 1, 0, 0, 0, 598, 591, 1, 0, 0, 0, 599, 600, 1, 0, 0, 0,
|
||||
600, 598, 1, 0, 0, 0, 600, 601, 1, 0, 0, 0, 601, 94, 1, 0, 0, 0, 602, 603,
|
||||
5, 117, 0, 0, 603, 606, 5, 56, 0, 0, 604, 606, 7, 0, 0, 0, 605, 602, 1,
|
||||
0, 0, 0, 605, 604, 1, 0, 0, 0, 606, 96, 1, 0, 0, 0, 607, 609, 3, 101, 50,
|
||||
0, 608, 607, 1, 0, 0, 0, 609, 610, 1, 0, 0, 0, 610, 608, 1, 0, 0, 0, 610,
|
||||
611, 1, 0, 0, 0, 611, 98, 1, 0, 0, 0, 612, 614, 3, 103, 51, 0, 613, 612,
|
||||
1, 0, 0, 0, 614, 615, 1, 0, 0, 0, 615, 613, 1, 0, 0, 0, 615, 616, 1, 0,
|
||||
0, 0, 616, 100, 1, 0, 0, 0, 617, 625, 8, 1, 0, 0, 618, 625, 3, 143, 71,
|
||||
0, 619, 620, 5, 92, 0, 0, 620, 625, 5, 10, 0, 0, 621, 622, 5, 92, 0, 0,
|
||||
622, 623, 5, 13, 0, 0, 623, 625, 5, 10, 0, 0, 624, 617, 1, 0, 0, 0, 624,
|
||||
618, 1, 0, 0, 0, 624, 619, 1, 0, 0, 0, 624, 621, 1, 0, 0, 0, 625, 102,
|
||||
1, 0, 0, 0, 626, 634, 8, 2, 0, 0, 627, 634, 3, 143, 71, 0, 628, 629, 5,
|
||||
92, 0, 0, 629, 634, 5, 10, 0, 0, 630, 631, 5, 92, 0, 0, 631, 632, 5, 13,
|
||||
0, 0, 632, 634, 5, 10, 0, 0, 633, 626, 1, 0, 0, 0, 633, 627, 1, 0, 0, 0,
|
||||
633, 628, 1, 0, 0, 0, 633, 630, 1, 0, 0, 0, 634, 104, 1, 0, 0, 0, 635,
|
||||
636, 7, 3, 0, 0, 636, 106, 1, 0, 0, 0, 637, 638, 7, 4, 0, 0, 638, 108,
|
||||
1, 0, 0, 0, 639, 640, 5, 48, 0, 0, 640, 642, 7, 5, 0, 0, 641, 643, 7, 6,
|
||||
0, 0, 642, 641, 1, 0, 0, 0, 643, 644, 1, 0, 0, 0, 644, 642, 1, 0, 0, 0,
|
||||
644, 645, 1, 0, 0, 0, 645, 110, 1, 0, 0, 0, 646, 650, 3, 117, 58, 0, 647,
|
||||
649, 3, 107, 53, 0, 648, 647, 1, 0, 0, 0, 649, 652, 1, 0, 0, 0, 650, 648,
|
||||
1, 0, 0, 0, 650, 651, 1, 0, 0, 0, 651, 655, 1, 0, 0, 0, 652, 650, 1, 0,
|
||||
0, 0, 653, 655, 5, 48, 0, 0, 654, 646, 1, 0, 0, 0, 654, 653, 1, 0, 0, 0,
|
||||
655, 112, 1, 0, 0, 0, 656, 660, 5, 48, 0, 0, 657, 659, 3, 119, 59, 0, 658,
|
||||
657, 1, 0, 0, 0, 659, 662, 1, 0, 0, 0, 660, 658, 1, 0, 0, 0, 660, 661,
|
||||
1, 0, 0, 0, 661, 114, 1, 0, 0, 0, 662, 660, 1, 0, 0, 0, 663, 664, 5, 48,
|
||||
0, 0, 664, 665, 7, 7, 0, 0, 665, 666, 3, 139, 69, 0, 666, 116, 1, 0, 0,
|
||||
0, 667, 668, 7, 8, 0, 0, 668, 118, 1, 0, 0, 0, 669, 670, 7, 9, 0, 0, 670,
|
||||
120, 1, 0, 0, 0, 671, 672, 7, 10, 0, 0, 672, 122, 1, 0, 0, 0, 673, 674,
|
||||
3, 121, 60, 0, 674, 675, 3, 121, 60, 0, 675, 676, 3, 121, 60, 0, 676, 677,
|
||||
3, 121, 60, 0, 677, 124, 1, 0, 0, 0, 678, 679, 5, 92, 0, 0, 679, 680, 5,
|
||||
117, 0, 0, 680, 681, 1, 0, 0, 0, 681, 689, 3, 123, 61, 0, 682, 683, 5,
|
||||
92, 0, 0, 683, 684, 5, 85, 0, 0, 684, 685, 1, 0, 0, 0, 685, 686, 3, 123,
|
||||
61, 0, 686, 687, 3, 123, 61, 0, 687, 689, 1, 0, 0, 0, 688, 678, 1, 0, 0,
|
||||
0, 688, 682, 1, 0, 0, 0, 689, 126, 1, 0, 0, 0, 690, 692, 3, 131, 65, 0,
|
||||
691, 693, 3, 133, 66, 0, 692, 691, 1, 0, 0, 0, 692, 693, 1, 0, 0, 0, 693,
|
||||
698, 1, 0, 0, 0, 694, 695, 3, 135, 67, 0, 695, 696, 3, 133, 66, 0, 696,
|
||||
698, 1, 0, 0, 0, 697, 690, 1, 0, 0, 0, 697, 694, 1, 0, 0, 0, 698, 128,
|
||||
1, 0, 0, 0, 699, 700, 5, 48, 0, 0, 700, 703, 7, 7, 0, 0, 701, 704, 3, 137,
|
||||
68, 0, 702, 704, 3, 139, 69, 0, 703, 701, 1, 0, 0, 0, 703, 702, 1, 0, 0,
|
||||
0, 704, 705, 1, 0, 0, 0, 705, 706, 3, 141, 70, 0, 706, 130, 1, 0, 0, 0,
|
||||
707, 709, 3, 135, 67, 0, 708, 707, 1, 0, 0, 0, 708, 709, 1, 0, 0, 0, 709,
|
||||
710, 1, 0, 0, 0, 710, 711, 5, 46, 0, 0, 711, 716, 3, 135, 67, 0, 712, 713,
|
||||
3, 135, 67, 0, 713, 714, 5, 46, 0, 0, 714, 716, 1, 0, 0, 0, 715, 708, 1,
|
||||
0, 0, 0, 715, 712, 1, 0, 0, 0, 716, 132, 1, 0, 0, 0, 717, 719, 7, 11, 0,
|
||||
0, 718, 720, 7, 12, 0, 0, 719, 718, 1, 0, 0, 0, 719, 720, 1, 0, 0, 0, 720,
|
||||
721, 1, 0, 0, 0, 721, 722, 3, 135, 67, 0, 722, 134, 1, 0, 0, 0, 723, 725,
|
||||
3, 107, 53, 0, 724, 723, 1, 0, 0, 0, 725, 726, 1, 0, 0, 0, 726, 724, 1,
|
||||
0, 0, 0, 726, 727, 1, 0, 0, 0, 727, 136, 1, 0, 0, 0, 728, 730, 3, 139,
|
||||
69, 0, 729, 728, 1, 0, 0, 0, 729, 730, 1, 0, 0, 0, 730, 731, 1, 0, 0, 0,
|
||||
731, 732, 5, 46, 0, 0, 732, 737, 3, 139, 69, 0, 733, 734, 3, 139, 69, 0,
|
||||
734, 735, 5, 46, 0, 0, 735, 737, 1, 0, 0, 0, 736, 729, 1, 0, 0, 0, 736,
|
||||
733, 1, 0, 0, 0, 737, 138, 1, 0, 0, 0, 738, 740, 3, 121, 60, 0, 739, 738,
|
||||
1, 0, 0, 0, 740, 741, 1, 0, 0, 0, 741, 739, 1, 0, 0, 0, 741, 742, 1, 0,
|
||||
0, 0, 742, 140, 1, 0, 0, 0, 743, 745, 7, 13, 0, 0, 744, 746, 7, 12, 0,
|
||||
0, 745, 744, 1, 0, 0, 0, 745, 746, 1, 0, 0, 0, 746, 747, 1, 0, 0, 0, 747,
|
||||
748, 3, 135, 67, 0, 748, 142, 1, 0, 0, 0, 749, 750, 5, 92, 0, 0, 750, 765,
|
||||
7, 14, 0, 0, 751, 752, 5, 92, 0, 0, 752, 754, 3, 119, 59, 0, 753, 755,
|
||||
3, 119, 59, 0, 754, 753, 1, 0, 0, 0, 754, 755, 1, 0, 0, 0, 755, 757, 1,
|
||||
0, 0, 0, 756, 758, 3, 119, 59, 0, 757, 756, 1, 0, 0, 0, 757, 758, 1, 0,
|
||||
0, 0, 758, 765, 1, 0, 0, 0, 759, 760, 5, 92, 0, 0, 760, 761, 5, 120, 0,
|
||||
0, 761, 762, 1, 0, 0, 0, 762, 765, 3, 139, 69, 0, 763, 765, 3, 125, 62,
|
||||
0, 764, 749, 1, 0, 0, 0, 764, 751, 1, 0, 0, 0, 764, 759, 1, 0, 0, 0, 764,
|
||||
763, 1, 0, 0, 0, 765, 144, 1, 0, 0, 0, 766, 768, 7, 15, 0, 0, 767, 766,
|
||||
1, 0, 0, 0, 768, 769, 1, 0, 0, 0, 769, 767, 1, 0, 0, 0, 769, 770, 1, 0,
|
||||
0, 0, 770, 771, 1, 0, 0, 0, 771, 772, 6, 72, 0, 0, 772, 146, 1, 0, 0, 0,
|
||||
773, 775, 5, 13, 0, 0, 774, 776, 5, 10, 0, 0, 775, 774, 1, 0, 0, 0, 775,
|
||||
776, 1, 0, 0, 0, 776, 779, 1, 0, 0, 0, 777, 779, 5, 10, 0, 0, 778, 773,
|
||||
1, 0, 0, 0, 778, 777, 1, 0, 0, 0, 779, 780, 1, 0, 0, 0, 780, 781, 6, 73,
|
||||
0, 0, 781, 148, 1, 0, 0, 0, 56, 0, 187, 201, 223, 255, 261, 269, 275, 280,
|
||||
282, 313, 349, 385, 415, 453, 491, 517, 546, 552, 556, 561, 563, 573, 577,
|
||||
582, 585, 589, 594, 600, 605, 610, 615, 624, 633, 644, 650, 654, 660, 688,
|
||||
692, 697, 703, 708, 715, 719, 726, 729, 736, 741, 745, 754, 757, 764, 769,
|
||||
775, 778, 1, 6, 0, 0,
|
||||
40, 1, 40, 3, 40, 536, 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, 3, 41, 562, 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, 1, 42, 1, 42, 1, 42, 1, 42, 1, 42, 3, 42, 591, 8, 42, 1, 43, 1,
|
||||
43, 1, 43, 1, 43, 3, 43, 597, 8, 43, 1, 44, 1, 44, 3, 44, 601, 8, 44, 1,
|
||||
45, 1, 45, 1, 45, 5, 45, 606, 8, 45, 10, 45, 12, 45, 609, 9, 45, 1, 46,
|
||||
1, 46, 1, 46, 1, 46, 1, 46, 1, 46, 1, 47, 3, 47, 618, 8, 47, 1, 47, 1,
|
||||
47, 3, 47, 622, 8, 47, 1, 47, 1, 47, 1, 47, 3, 47, 627, 8, 47, 1, 47, 3,
|
||||
47, 630, 8, 47, 1, 48, 1, 48, 3, 48, 634, 8, 48, 1, 48, 1, 48, 1, 48, 3,
|
||||
48, 639, 8, 48, 1, 48, 1, 48, 4, 48, 643, 8, 48, 11, 48, 12, 48, 644, 1,
|
||||
49, 1, 49, 1, 49, 3, 49, 650, 8, 49, 1, 50, 4, 50, 653, 8, 50, 11, 50,
|
||||
12, 50, 654, 1, 51, 4, 51, 658, 8, 51, 11, 51, 12, 51, 659, 1, 52, 1, 52,
|
||||
1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 3, 52, 669, 8, 52, 1, 53, 1, 53, 1,
|
||||
53, 1, 53, 1, 53, 1, 53, 1, 53, 3, 53, 678, 8, 53, 1, 54, 1, 54, 1, 55,
|
||||
1, 55, 1, 56, 1, 56, 1, 56, 4, 56, 687, 8, 56, 11, 56, 12, 56, 688, 1,
|
||||
57, 1, 57, 5, 57, 693, 8, 57, 10, 57, 12, 57, 696, 9, 57, 1, 57, 3, 57,
|
||||
699, 8, 57, 1, 58, 1, 58, 5, 58, 703, 8, 58, 10, 58, 12, 58, 706, 9, 58,
|
||||
1, 59, 1, 59, 1, 59, 1, 59, 1, 60, 1, 60, 1, 61, 1, 61, 1, 62, 1, 62, 1,
|
||||
63, 1, 63, 1, 63, 1, 63, 1, 63, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64, 1, 64,
|
||||
1, 64, 1, 64, 1, 64, 1, 64, 3, 64, 733, 8, 64, 1, 65, 1, 65, 3, 65, 737,
|
||||
8, 65, 1, 65, 1, 65, 1, 65, 3, 65, 742, 8, 65, 1, 66, 1, 66, 1, 66, 1,
|
||||
66, 3, 66, 748, 8, 66, 1, 66, 1, 66, 1, 67, 3, 67, 753, 8, 67, 1, 67, 1,
|
||||
67, 1, 67, 1, 67, 1, 67, 3, 67, 760, 8, 67, 1, 68, 1, 68, 3, 68, 764, 8,
|
||||
68, 1, 68, 1, 68, 1, 69, 4, 69, 769, 8, 69, 11, 69, 12, 69, 770, 1, 70,
|
||||
3, 70, 774, 8, 70, 1, 70, 1, 70, 1, 70, 1, 70, 1, 70, 3, 70, 781, 8, 70,
|
||||
1, 71, 4, 71, 784, 8, 71, 11, 71, 12, 71, 785, 1, 72, 1, 72, 3, 72, 790,
|
||||
8, 72, 1, 72, 1, 72, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 3, 73, 799, 8,
|
||||
73, 1, 73, 3, 73, 802, 8, 73, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 3, 73,
|
||||
809, 8, 73, 1, 74, 4, 74, 812, 8, 74, 11, 74, 12, 74, 813, 1, 74, 1, 74,
|
||||
1, 75, 1, 75, 3, 75, 820, 8, 75, 1, 75, 3, 75, 823, 8, 75, 1, 75, 1, 75,
|
||||
0, 0, 76, 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, 0, 101, 0, 103, 0, 105, 0, 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, 50, 151, 51, 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, 869,
|
||||
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, 149, 1, 0, 0,
|
||||
0, 0, 151, 1, 0, 0, 0, 1, 153, 1, 0, 0, 0, 3, 155, 1, 0, 0, 0, 5, 157,
|
||||
1, 0, 0, 0, 7, 159, 1, 0, 0, 0, 9, 161, 1, 0, 0, 0, 11, 163, 1, 0, 0, 0,
|
||||
13, 165, 1, 0, 0, 0, 15, 167, 1, 0, 0, 0, 17, 169, 1, 0, 0, 0, 19, 172,
|
||||
1, 0, 0, 0, 21, 174, 1, 0, 0, 0, 23, 177, 1, 0, 0, 0, 25, 180, 1, 0, 0,
|
||||
0, 27, 191, 1, 0, 0, 0, 29, 205, 1, 0, 0, 0, 31, 227, 1, 0, 0, 0, 33, 229,
|
||||
1, 0, 0, 0, 35, 231, 1, 0, 0, 0, 37, 233, 1, 0, 0, 0, 39, 235, 1, 0, 0,
|
||||
0, 41, 237, 1, 0, 0, 0, 43, 239, 1, 0, 0, 0, 45, 242, 1, 0, 0, 0, 47, 245,
|
||||
1, 0, 0, 0, 49, 248, 1, 0, 0, 0, 51, 250, 1, 0, 0, 0, 53, 252, 1, 0, 0,
|
||||
0, 55, 259, 1, 0, 0, 0, 57, 265, 1, 0, 0, 0, 59, 281, 1, 0, 0, 0, 61, 305,
|
||||
1, 0, 0, 0, 63, 307, 1, 0, 0, 0, 65, 313, 1, 0, 0, 0, 67, 319, 1, 0, 0,
|
||||
0, 69, 321, 1, 0, 0, 0, 71, 357, 1, 0, 0, 0, 73, 393, 1, 0, 0, 0, 75, 429,
|
||||
1, 0, 0, 0, 77, 459, 1, 0, 0, 0, 79, 497, 1, 0, 0, 0, 81, 535, 1, 0, 0,
|
||||
0, 83, 561, 1, 0, 0, 0, 85, 590, 1, 0, 0, 0, 87, 596, 1, 0, 0, 0, 89, 600,
|
||||
1, 0, 0, 0, 91, 602, 1, 0, 0, 0, 93, 610, 1, 0, 0, 0, 95, 617, 1, 0, 0,
|
||||
0, 97, 633, 1, 0, 0, 0, 99, 649, 1, 0, 0, 0, 101, 652, 1, 0, 0, 0, 103,
|
||||
657, 1, 0, 0, 0, 105, 668, 1, 0, 0, 0, 107, 677, 1, 0, 0, 0, 109, 679,
|
||||
1, 0, 0, 0, 111, 681, 1, 0, 0, 0, 113, 683, 1, 0, 0, 0, 115, 698, 1, 0,
|
||||
0, 0, 117, 700, 1, 0, 0, 0, 119, 707, 1, 0, 0, 0, 121, 711, 1, 0, 0, 0,
|
||||
123, 713, 1, 0, 0, 0, 125, 715, 1, 0, 0, 0, 127, 717, 1, 0, 0, 0, 129,
|
||||
732, 1, 0, 0, 0, 131, 741, 1, 0, 0, 0, 133, 743, 1, 0, 0, 0, 135, 759,
|
||||
1, 0, 0, 0, 137, 761, 1, 0, 0, 0, 139, 768, 1, 0, 0, 0, 141, 780, 1, 0,
|
||||
0, 0, 143, 783, 1, 0, 0, 0, 145, 787, 1, 0, 0, 0, 147, 808, 1, 0, 0, 0,
|
||||
149, 811, 1, 0, 0, 0, 151, 822, 1, 0, 0, 0, 153, 154, 5, 40, 0, 0, 154,
|
||||
2, 1, 0, 0, 0, 155, 156, 5, 41, 0, 0, 156, 4, 1, 0, 0, 0, 157, 158, 5,
|
||||
91, 0, 0, 158, 6, 1, 0, 0, 0, 159, 160, 5, 44, 0, 0, 160, 8, 1, 0, 0, 0,
|
||||
161, 162, 5, 93, 0, 0, 162, 10, 1, 0, 0, 0, 163, 164, 5, 123, 0, 0, 164,
|
||||
12, 1, 0, 0, 0, 165, 166, 5, 125, 0, 0, 166, 14, 1, 0, 0, 0, 167, 168,
|
||||
5, 60, 0, 0, 168, 16, 1, 0, 0, 0, 169, 170, 5, 60, 0, 0, 170, 171, 5, 61,
|
||||
0, 0, 171, 18, 1, 0, 0, 0, 172, 173, 5, 62, 0, 0, 173, 20, 1, 0, 0, 0,
|
||||
174, 175, 5, 62, 0, 0, 175, 176, 5, 61, 0, 0, 176, 22, 1, 0, 0, 0, 177,
|
||||
178, 5, 61, 0, 0, 178, 179, 5, 61, 0, 0, 179, 24, 1, 0, 0, 0, 180, 181,
|
||||
5, 33, 0, 0, 181, 182, 5, 61, 0, 0, 182, 26, 1, 0, 0, 0, 183, 184, 5, 108,
|
||||
0, 0, 184, 185, 5, 105, 0, 0, 185, 186, 5, 107, 0, 0, 186, 192, 5, 101,
|
||||
0, 0, 187, 188, 5, 76, 0, 0, 188, 189, 5, 73, 0, 0, 189, 190, 5, 75, 0,
|
||||
0, 190, 192, 5, 69, 0, 0, 191, 183, 1, 0, 0, 0, 191, 187, 1, 0, 0, 0, 192,
|
||||
28, 1, 0, 0, 0, 193, 194, 5, 101, 0, 0, 194, 195, 5, 120, 0, 0, 195, 196,
|
||||
5, 105, 0, 0, 196, 197, 5, 115, 0, 0, 197, 198, 5, 116, 0, 0, 198, 206,
|
||||
5, 115, 0, 0, 199, 200, 5, 69, 0, 0, 200, 201, 5, 88, 0, 0, 201, 202, 5,
|
||||
73, 0, 0, 202, 203, 5, 83, 0, 0, 203, 204, 5, 84, 0, 0, 204, 206, 5, 83,
|
||||
0, 0, 205, 193, 1, 0, 0, 0, 205, 199, 1, 0, 0, 0, 206, 30, 1, 0, 0, 0,
|
||||
207, 208, 5, 116, 0, 0, 208, 209, 5, 101, 0, 0, 209, 210, 5, 120, 0, 0,
|
||||
210, 211, 5, 116, 0, 0, 211, 212, 5, 95, 0, 0, 212, 213, 5, 109, 0, 0,
|
||||
213, 214, 5, 97, 0, 0, 214, 215, 5, 116, 0, 0, 215, 216, 5, 99, 0, 0, 216,
|
||||
228, 5, 104, 0, 0, 217, 218, 5, 84, 0, 0, 218, 219, 5, 69, 0, 0, 219, 220,
|
||||
5, 88, 0, 0, 220, 221, 5, 84, 0, 0, 221, 222, 5, 95, 0, 0, 222, 223, 5,
|
||||
77, 0, 0, 223, 224, 5, 65, 0, 0, 224, 225, 5, 84, 0, 0, 225, 226, 5, 67,
|
||||
0, 0, 226, 228, 5, 72, 0, 0, 227, 207, 1, 0, 0, 0, 227, 217, 1, 0, 0, 0,
|
||||
228, 32, 1, 0, 0, 0, 229, 230, 5, 43, 0, 0, 230, 34, 1, 0, 0, 0, 231, 232,
|
||||
5, 45, 0, 0, 232, 36, 1, 0, 0, 0, 233, 234, 5, 42, 0, 0, 234, 38, 1, 0,
|
||||
0, 0, 235, 236, 5, 47, 0, 0, 236, 40, 1, 0, 0, 0, 237, 238, 5, 37, 0, 0,
|
||||
238, 42, 1, 0, 0, 0, 239, 240, 5, 42, 0, 0, 240, 241, 5, 42, 0, 0, 241,
|
||||
44, 1, 0, 0, 0, 242, 243, 5, 60, 0, 0, 243, 244, 5, 60, 0, 0, 244, 46,
|
||||
1, 0, 0, 0, 245, 246, 5, 62, 0, 0, 246, 247, 5, 62, 0, 0, 247, 48, 1, 0,
|
||||
0, 0, 248, 249, 5, 38, 0, 0, 249, 50, 1, 0, 0, 0, 250, 251, 5, 124, 0,
|
||||
0, 251, 52, 1, 0, 0, 0, 252, 253, 5, 94, 0, 0, 253, 54, 1, 0, 0, 0, 254,
|
||||
255, 5, 38, 0, 0, 255, 260, 5, 38, 0, 0, 256, 257, 5, 97, 0, 0, 257, 258,
|
||||
5, 110, 0, 0, 258, 260, 5, 100, 0, 0, 259, 254, 1, 0, 0, 0, 259, 256, 1,
|
||||
0, 0, 0, 260, 56, 1, 0, 0, 0, 261, 262, 5, 124, 0, 0, 262, 266, 5, 124,
|
||||
0, 0, 263, 264, 5, 111, 0, 0, 264, 266, 5, 114, 0, 0, 265, 261, 1, 0, 0,
|
||||
0, 265, 263, 1, 0, 0, 0, 266, 58, 1, 0, 0, 0, 267, 268, 5, 105, 0, 0, 268,
|
||||
269, 5, 115, 0, 0, 269, 270, 5, 32, 0, 0, 270, 271, 5, 110, 0, 0, 271,
|
||||
272, 5, 117, 0, 0, 272, 273, 5, 108, 0, 0, 273, 282, 5, 108, 0, 0, 274,
|
||||
275, 5, 73, 0, 0, 275, 276, 5, 83, 0, 0, 276, 277, 5, 32, 0, 0, 277, 278,
|
||||
5, 78, 0, 0, 278, 279, 5, 85, 0, 0, 279, 280, 5, 76, 0, 0, 280, 282, 5,
|
||||
76, 0, 0, 281, 267, 1, 0, 0, 0, 281, 274, 1, 0, 0, 0, 282, 60, 1, 0, 0,
|
||||
0, 283, 284, 5, 105, 0, 0, 284, 285, 5, 115, 0, 0, 285, 286, 5, 32, 0,
|
||||
0, 286, 287, 5, 110, 0, 0, 287, 288, 5, 111, 0, 0, 288, 289, 5, 116, 0,
|
||||
0, 289, 290, 5, 32, 0, 0, 290, 291, 5, 110, 0, 0, 291, 292, 5, 117, 0,
|
||||
0, 292, 293, 5, 108, 0, 0, 293, 306, 5, 108, 0, 0, 294, 295, 5, 73, 0,
|
||||
0, 295, 296, 5, 83, 0, 0, 296, 297, 5, 32, 0, 0, 297, 298, 5, 78, 0, 0,
|
||||
298, 299, 5, 79, 0, 0, 299, 300, 5, 84, 0, 0, 300, 301, 5, 32, 0, 0, 301,
|
||||
302, 5, 78, 0, 0, 302, 303, 5, 85, 0, 0, 303, 304, 5, 76, 0, 0, 304, 306,
|
||||
5, 76, 0, 0, 305, 283, 1, 0, 0, 0, 305, 294, 1, 0, 0, 0, 306, 62, 1, 0,
|
||||
0, 0, 307, 308, 5, 126, 0, 0, 308, 64, 1, 0, 0, 0, 309, 314, 5, 33, 0,
|
||||
0, 310, 311, 5, 110, 0, 0, 311, 312, 5, 111, 0, 0, 312, 314, 5, 116, 0,
|
||||
0, 313, 309, 1, 0, 0, 0, 313, 310, 1, 0, 0, 0, 314, 66, 1, 0, 0, 0, 315,
|
||||
316, 5, 105, 0, 0, 316, 320, 5, 110, 0, 0, 317, 318, 5, 73, 0, 0, 318,
|
||||
320, 5, 78, 0, 0, 319, 315, 1, 0, 0, 0, 319, 317, 1, 0, 0, 0, 320, 68,
|
||||
1, 0, 0, 0, 321, 326, 5, 91, 0, 0, 322, 325, 3, 149, 74, 0, 323, 325, 3,
|
||||
151, 75, 0, 324, 322, 1, 0, 0, 0, 324, 323, 1, 0, 0, 0, 325, 328, 1, 0,
|
||||
0, 0, 326, 324, 1, 0, 0, 0, 326, 327, 1, 0, 0, 0, 327, 329, 1, 0, 0, 0,
|
||||
328, 326, 1, 0, 0, 0, 329, 330, 5, 93, 0, 0, 330, 70, 1, 0, 0, 0, 331,
|
||||
332, 5, 106, 0, 0, 332, 333, 5, 115, 0, 0, 333, 334, 5, 111, 0, 0, 334,
|
||||
335, 5, 110, 0, 0, 335, 336, 5, 95, 0, 0, 336, 337, 5, 99, 0, 0, 337, 338,
|
||||
5, 111, 0, 0, 338, 339, 5, 110, 0, 0, 339, 340, 5, 116, 0, 0, 340, 341,
|
||||
5, 97, 0, 0, 341, 342, 5, 105, 0, 0, 342, 343, 5, 110, 0, 0, 343, 358,
|
||||
5, 115, 0, 0, 344, 345, 5, 74, 0, 0, 345, 346, 5, 83, 0, 0, 346, 347, 5,
|
||||
79, 0, 0, 347, 348, 5, 78, 0, 0, 348, 349, 5, 95, 0, 0, 349, 350, 5, 67,
|
||||
0, 0, 350, 351, 5, 79, 0, 0, 351, 352, 5, 78, 0, 0, 352, 353, 5, 84, 0,
|
||||
0, 353, 354, 5, 65, 0, 0, 354, 355, 5, 73, 0, 0, 355, 356, 5, 78, 0, 0,
|
||||
356, 358, 5, 83, 0, 0, 357, 331, 1, 0, 0, 0, 357, 344, 1, 0, 0, 0, 358,
|
||||
72, 1, 0, 0, 0, 359, 360, 5, 106, 0, 0, 360, 361, 5, 115, 0, 0, 361, 362,
|
||||
5, 111, 0, 0, 362, 363, 5, 110, 0, 0, 363, 364, 5, 95, 0, 0, 364, 365,
|
||||
5, 99, 0, 0, 365, 366, 5, 111, 0, 0, 366, 367, 5, 110, 0, 0, 367, 368,
|
||||
5, 116, 0, 0, 368, 369, 5, 97, 0, 0, 369, 370, 5, 105, 0, 0, 370, 371,
|
||||
5, 110, 0, 0, 371, 372, 5, 115, 0, 0, 372, 373, 5, 95, 0, 0, 373, 374,
|
||||
5, 97, 0, 0, 374, 375, 5, 108, 0, 0, 375, 394, 5, 108, 0, 0, 376, 377,
|
||||
5, 74, 0, 0, 377, 378, 5, 83, 0, 0, 378, 379, 5, 79, 0, 0, 379, 380, 5,
|
||||
78, 0, 0, 380, 381, 5, 95, 0, 0, 381, 382, 5, 67, 0, 0, 382, 383, 5, 79,
|
||||
0, 0, 383, 384, 5, 78, 0, 0, 384, 385, 5, 84, 0, 0, 385, 386, 5, 65, 0,
|
||||
0, 386, 387, 5, 73, 0, 0, 387, 388, 5, 78, 0, 0, 388, 389, 5, 83, 0, 0,
|
||||
389, 390, 5, 95, 0, 0, 390, 391, 5, 65, 0, 0, 391, 392, 5, 76, 0, 0, 392,
|
||||
394, 5, 76, 0, 0, 393, 359, 1, 0, 0, 0, 393, 376, 1, 0, 0, 0, 394, 74,
|
||||
1, 0, 0, 0, 395, 396, 5, 106, 0, 0, 396, 397, 5, 115, 0, 0, 397, 398, 5,
|
||||
111, 0, 0, 398, 399, 5, 110, 0, 0, 399, 400, 5, 95, 0, 0, 400, 401, 5,
|
||||
99, 0, 0, 401, 402, 5, 111, 0, 0, 402, 403, 5, 110, 0, 0, 403, 404, 5,
|
||||
116, 0, 0, 404, 405, 5, 97, 0, 0, 405, 406, 5, 105, 0, 0, 406, 407, 5,
|
||||
110, 0, 0, 407, 408, 5, 115, 0, 0, 408, 409, 5, 95, 0, 0, 409, 410, 5,
|
||||
97, 0, 0, 410, 411, 5, 110, 0, 0, 411, 430, 5, 121, 0, 0, 412, 413, 5,
|
||||
74, 0, 0, 413, 414, 5, 83, 0, 0, 414, 415, 5, 79, 0, 0, 415, 416, 5, 78,
|
||||
0, 0, 416, 417, 5, 95, 0, 0, 417, 418, 5, 67, 0, 0, 418, 419, 5, 79, 0,
|
||||
0, 419, 420, 5, 78, 0, 0, 420, 421, 5, 84, 0, 0, 421, 422, 5, 65, 0, 0,
|
||||
422, 423, 5, 73, 0, 0, 423, 424, 5, 78, 0, 0, 424, 425, 5, 83, 0, 0, 425,
|
||||
426, 5, 95, 0, 0, 426, 427, 5, 65, 0, 0, 427, 428, 5, 78, 0, 0, 428, 430,
|
||||
5, 89, 0, 0, 429, 395, 1, 0, 0, 0, 429, 412, 1, 0, 0, 0, 430, 76, 1, 0,
|
||||
0, 0, 431, 432, 5, 97, 0, 0, 432, 433, 5, 114, 0, 0, 433, 434, 5, 114,
|
||||
0, 0, 434, 435, 5, 97, 0, 0, 435, 436, 5, 121, 0, 0, 436, 437, 5, 95, 0,
|
||||
0, 437, 438, 5, 99, 0, 0, 438, 439, 5, 111, 0, 0, 439, 440, 5, 110, 0,
|
||||
0, 440, 441, 5, 116, 0, 0, 441, 442, 5, 97, 0, 0, 442, 443, 5, 105, 0,
|
||||
0, 443, 444, 5, 110, 0, 0, 444, 460, 5, 115, 0, 0, 445, 446, 5, 65, 0,
|
||||
0, 446, 447, 5, 82, 0, 0, 447, 448, 5, 82, 0, 0, 448, 449, 5, 65, 0, 0,
|
||||
449, 450, 5, 89, 0, 0, 450, 451, 5, 95, 0, 0, 451, 452, 5, 67, 0, 0, 452,
|
||||
453, 5, 79, 0, 0, 453, 454, 5, 78, 0, 0, 454, 455, 5, 84, 0, 0, 455, 456,
|
||||
5, 65, 0, 0, 456, 457, 5, 73, 0, 0, 457, 458, 5, 78, 0, 0, 458, 460, 5,
|
||||
83, 0, 0, 459, 431, 1, 0, 0, 0, 459, 445, 1, 0, 0, 0, 460, 78, 1, 0, 0,
|
||||
0, 461, 462, 5, 97, 0, 0, 462, 463, 5, 114, 0, 0, 463, 464, 5, 114, 0,
|
||||
0, 464, 465, 5, 97, 0, 0, 465, 466, 5, 121, 0, 0, 466, 467, 5, 95, 0, 0,
|
||||
467, 468, 5, 99, 0, 0, 468, 469, 5, 111, 0, 0, 469, 470, 5, 110, 0, 0,
|
||||
470, 471, 5, 116, 0, 0, 471, 472, 5, 97, 0, 0, 472, 473, 5, 105, 0, 0,
|
||||
473, 474, 5, 110, 0, 0, 474, 475, 5, 115, 0, 0, 475, 476, 5, 95, 0, 0,
|
||||
476, 477, 5, 97, 0, 0, 477, 478, 5, 108, 0, 0, 478, 498, 5, 108, 0, 0,
|
||||
479, 480, 5, 65, 0, 0, 480, 481, 5, 82, 0, 0, 481, 482, 5, 82, 0, 0, 482,
|
||||
483, 5, 65, 0, 0, 483, 484, 5, 89, 0, 0, 484, 485, 5, 95, 0, 0, 485, 486,
|
||||
5, 67, 0, 0, 486, 487, 5, 79, 0, 0, 487, 488, 5, 78, 0, 0, 488, 489, 5,
|
||||
84, 0, 0, 489, 490, 5, 65, 0, 0, 490, 491, 5, 73, 0, 0, 491, 492, 5, 78,
|
||||
0, 0, 492, 493, 5, 83, 0, 0, 493, 494, 5, 95, 0, 0, 494, 495, 5, 65, 0,
|
||||
0, 495, 496, 5, 76, 0, 0, 496, 498, 5, 76, 0, 0, 497, 461, 1, 0, 0, 0,
|
||||
497, 479, 1, 0, 0, 0, 498, 80, 1, 0, 0, 0, 499, 500, 5, 97, 0, 0, 500,
|
||||
501, 5, 114, 0, 0, 501, 502, 5, 114, 0, 0, 502, 503, 5, 97, 0, 0, 503,
|
||||
504, 5, 121, 0, 0, 504, 505, 5, 95, 0, 0, 505, 506, 5, 99, 0, 0, 506, 507,
|
||||
5, 111, 0, 0, 507, 508, 5, 110, 0, 0, 508, 509, 5, 116, 0, 0, 509, 510,
|
||||
5, 97, 0, 0, 510, 511, 5, 105, 0, 0, 511, 512, 5, 110, 0, 0, 512, 513,
|
||||
5, 115, 0, 0, 513, 514, 5, 95, 0, 0, 514, 515, 5, 97, 0, 0, 515, 516, 5,
|
||||
110, 0, 0, 516, 536, 5, 121, 0, 0, 517, 518, 5, 65, 0, 0, 518, 519, 5,
|
||||
82, 0, 0, 519, 520, 5, 82, 0, 0, 520, 521, 5, 65, 0, 0, 521, 522, 5, 89,
|
||||
0, 0, 522, 523, 5, 95, 0, 0, 523, 524, 5, 67, 0, 0, 524, 525, 5, 79, 0,
|
||||
0, 525, 526, 5, 78, 0, 0, 526, 527, 5, 84, 0, 0, 527, 528, 5, 65, 0, 0,
|
||||
528, 529, 5, 73, 0, 0, 529, 530, 5, 78, 0, 0, 530, 531, 5, 83, 0, 0, 531,
|
||||
532, 5, 95, 0, 0, 532, 533, 5, 65, 0, 0, 533, 534, 5, 78, 0, 0, 534, 536,
|
||||
5, 89, 0, 0, 535, 499, 1, 0, 0, 0, 535, 517, 1, 0, 0, 0, 536, 82, 1, 0,
|
||||
0, 0, 537, 538, 5, 97, 0, 0, 538, 539, 5, 114, 0, 0, 539, 540, 5, 114,
|
||||
0, 0, 540, 541, 5, 97, 0, 0, 541, 542, 5, 121, 0, 0, 542, 543, 5, 95, 0,
|
||||
0, 543, 544, 5, 108, 0, 0, 544, 545, 5, 101, 0, 0, 545, 546, 5, 110, 0,
|
||||
0, 546, 547, 5, 103, 0, 0, 547, 548, 5, 116, 0, 0, 548, 562, 5, 104, 0,
|
||||
0, 549, 550, 5, 65, 0, 0, 550, 551, 5, 82, 0, 0, 551, 552, 5, 82, 0, 0,
|
||||
552, 553, 5, 65, 0, 0, 553, 554, 5, 89, 0, 0, 554, 555, 5, 95, 0, 0, 555,
|
||||
556, 5, 76, 0, 0, 556, 557, 5, 69, 0, 0, 557, 558, 5, 78, 0, 0, 558, 559,
|
||||
5, 71, 0, 0, 559, 560, 5, 84, 0, 0, 560, 562, 5, 72, 0, 0, 561, 537, 1,
|
||||
0, 0, 0, 561, 549, 1, 0, 0, 0, 562, 84, 1, 0, 0, 0, 563, 564, 5, 116, 0,
|
||||
0, 564, 565, 5, 114, 0, 0, 565, 566, 5, 117, 0, 0, 566, 591, 5, 101, 0,
|
||||
0, 567, 568, 5, 84, 0, 0, 568, 569, 5, 114, 0, 0, 569, 570, 5, 117, 0,
|
||||
0, 570, 591, 5, 101, 0, 0, 571, 572, 5, 84, 0, 0, 572, 573, 5, 82, 0, 0,
|
||||
573, 574, 5, 85, 0, 0, 574, 591, 5, 69, 0, 0, 575, 576, 5, 102, 0, 0, 576,
|
||||
577, 5, 97, 0, 0, 577, 578, 5, 108, 0, 0, 578, 579, 5, 115, 0, 0, 579,
|
||||
591, 5, 101, 0, 0, 580, 581, 5, 70, 0, 0, 581, 582, 5, 97, 0, 0, 582, 583,
|
||||
5, 108, 0, 0, 583, 584, 5, 115, 0, 0, 584, 591, 5, 101, 0, 0, 585, 586,
|
||||
5, 70, 0, 0, 586, 587, 5, 65, 0, 0, 587, 588, 5, 76, 0, 0, 588, 589, 5,
|
||||
83, 0, 0, 589, 591, 5, 69, 0, 0, 590, 563, 1, 0, 0, 0, 590, 567, 1, 0,
|
||||
0, 0, 590, 571, 1, 0, 0, 0, 590, 575, 1, 0, 0, 0, 590, 580, 1, 0, 0, 0,
|
||||
590, 585, 1, 0, 0, 0, 591, 86, 1, 0, 0, 0, 592, 597, 3, 115, 57, 0, 593,
|
||||
597, 3, 117, 58, 0, 594, 597, 3, 119, 59, 0, 595, 597, 3, 113, 56, 0, 596,
|
||||
592, 1, 0, 0, 0, 596, 593, 1, 0, 0, 0, 596, 594, 1, 0, 0, 0, 596, 595,
|
||||
1, 0, 0, 0, 597, 88, 1, 0, 0, 0, 598, 601, 3, 131, 65, 0, 599, 601, 3,
|
||||
133, 66, 0, 600, 598, 1, 0, 0, 0, 600, 599, 1, 0, 0, 0, 601, 90, 1, 0,
|
||||
0, 0, 602, 607, 3, 109, 54, 0, 603, 606, 3, 109, 54, 0, 604, 606, 3, 111,
|
||||
55, 0, 605, 603, 1, 0, 0, 0, 605, 604, 1, 0, 0, 0, 606, 609, 1, 0, 0, 0,
|
||||
607, 605, 1, 0, 0, 0, 607, 608, 1, 0, 0, 0, 608, 92, 1, 0, 0, 0, 609, 607,
|
||||
1, 0, 0, 0, 610, 611, 5, 36, 0, 0, 611, 612, 5, 109, 0, 0, 612, 613, 5,
|
||||
101, 0, 0, 613, 614, 5, 116, 0, 0, 614, 615, 5, 97, 0, 0, 615, 94, 1, 0,
|
||||
0, 0, 616, 618, 3, 99, 49, 0, 617, 616, 1, 0, 0, 0, 617, 618, 1, 0, 0,
|
||||
0, 618, 629, 1, 0, 0, 0, 619, 621, 5, 34, 0, 0, 620, 622, 3, 101, 50, 0,
|
||||
621, 620, 1, 0, 0, 0, 621, 622, 1, 0, 0, 0, 622, 623, 1, 0, 0, 0, 623,
|
||||
630, 5, 34, 0, 0, 624, 626, 5, 39, 0, 0, 625, 627, 3, 103, 51, 0, 626,
|
||||
625, 1, 0, 0, 0, 626, 627, 1, 0, 0, 0, 627, 628, 1, 0, 0, 0, 628, 630,
|
||||
5, 39, 0, 0, 629, 619, 1, 0, 0, 0, 629, 624, 1, 0, 0, 0, 630, 96, 1, 0,
|
||||
0, 0, 631, 634, 3, 91, 45, 0, 632, 634, 3, 93, 46, 0, 633, 631, 1, 0, 0,
|
||||
0, 633, 632, 1, 0, 0, 0, 634, 642, 1, 0, 0, 0, 635, 638, 5, 91, 0, 0, 636,
|
||||
639, 3, 95, 47, 0, 637, 639, 3, 115, 57, 0, 638, 636, 1, 0, 0, 0, 638,
|
||||
637, 1, 0, 0, 0, 639, 640, 1, 0, 0, 0, 640, 641, 5, 93, 0, 0, 641, 643,
|
||||
1, 0, 0, 0, 642, 635, 1, 0, 0, 0, 643, 644, 1, 0, 0, 0, 644, 642, 1, 0,
|
||||
0, 0, 644, 645, 1, 0, 0, 0, 645, 98, 1, 0, 0, 0, 646, 647, 5, 117, 0, 0,
|
||||
647, 650, 5, 56, 0, 0, 648, 650, 7, 0, 0, 0, 649, 646, 1, 0, 0, 0, 649,
|
||||
648, 1, 0, 0, 0, 650, 100, 1, 0, 0, 0, 651, 653, 3, 105, 52, 0, 652, 651,
|
||||
1, 0, 0, 0, 653, 654, 1, 0, 0, 0, 654, 652, 1, 0, 0, 0, 654, 655, 1, 0,
|
||||
0, 0, 655, 102, 1, 0, 0, 0, 656, 658, 3, 107, 53, 0, 657, 656, 1, 0, 0,
|
||||
0, 658, 659, 1, 0, 0, 0, 659, 657, 1, 0, 0, 0, 659, 660, 1, 0, 0, 0, 660,
|
||||
104, 1, 0, 0, 0, 661, 669, 8, 1, 0, 0, 662, 669, 3, 147, 73, 0, 663, 664,
|
||||
5, 92, 0, 0, 664, 669, 5, 10, 0, 0, 665, 666, 5, 92, 0, 0, 666, 667, 5,
|
||||
13, 0, 0, 667, 669, 5, 10, 0, 0, 668, 661, 1, 0, 0, 0, 668, 662, 1, 0,
|
||||
0, 0, 668, 663, 1, 0, 0, 0, 668, 665, 1, 0, 0, 0, 669, 106, 1, 0, 0, 0,
|
||||
670, 678, 8, 2, 0, 0, 671, 678, 3, 147, 73, 0, 672, 673, 5, 92, 0, 0, 673,
|
||||
678, 5, 10, 0, 0, 674, 675, 5, 92, 0, 0, 675, 676, 5, 13, 0, 0, 676, 678,
|
||||
5, 10, 0, 0, 677, 670, 1, 0, 0, 0, 677, 671, 1, 0, 0, 0, 677, 672, 1, 0,
|
||||
0, 0, 677, 674, 1, 0, 0, 0, 678, 108, 1, 0, 0, 0, 679, 680, 7, 3, 0, 0,
|
||||
680, 110, 1, 0, 0, 0, 681, 682, 7, 4, 0, 0, 682, 112, 1, 0, 0, 0, 683,
|
||||
684, 5, 48, 0, 0, 684, 686, 7, 5, 0, 0, 685, 687, 7, 6, 0, 0, 686, 685,
|
||||
1, 0, 0, 0, 687, 688, 1, 0, 0, 0, 688, 686, 1, 0, 0, 0, 688, 689, 1, 0,
|
||||
0, 0, 689, 114, 1, 0, 0, 0, 690, 694, 3, 121, 60, 0, 691, 693, 3, 111,
|
||||
55, 0, 692, 691, 1, 0, 0, 0, 693, 696, 1, 0, 0, 0, 694, 692, 1, 0, 0, 0,
|
||||
694, 695, 1, 0, 0, 0, 695, 699, 1, 0, 0, 0, 696, 694, 1, 0, 0, 0, 697,
|
||||
699, 5, 48, 0, 0, 698, 690, 1, 0, 0, 0, 698, 697, 1, 0, 0, 0, 699, 116,
|
||||
1, 0, 0, 0, 700, 704, 5, 48, 0, 0, 701, 703, 3, 123, 61, 0, 702, 701, 1,
|
||||
0, 0, 0, 703, 706, 1, 0, 0, 0, 704, 702, 1, 0, 0, 0, 704, 705, 1, 0, 0,
|
||||
0, 705, 118, 1, 0, 0, 0, 706, 704, 1, 0, 0, 0, 707, 708, 5, 48, 0, 0, 708,
|
||||
709, 7, 7, 0, 0, 709, 710, 3, 143, 71, 0, 710, 120, 1, 0, 0, 0, 711, 712,
|
||||
7, 8, 0, 0, 712, 122, 1, 0, 0, 0, 713, 714, 7, 9, 0, 0, 714, 124, 1, 0,
|
||||
0, 0, 715, 716, 7, 10, 0, 0, 716, 126, 1, 0, 0, 0, 717, 718, 3, 125, 62,
|
||||
0, 718, 719, 3, 125, 62, 0, 719, 720, 3, 125, 62, 0, 720, 721, 3, 125,
|
||||
62, 0, 721, 128, 1, 0, 0, 0, 722, 723, 5, 92, 0, 0, 723, 724, 5, 117, 0,
|
||||
0, 724, 725, 1, 0, 0, 0, 725, 733, 3, 127, 63, 0, 726, 727, 5, 92, 0, 0,
|
||||
727, 728, 5, 85, 0, 0, 728, 729, 1, 0, 0, 0, 729, 730, 3, 127, 63, 0, 730,
|
||||
731, 3, 127, 63, 0, 731, 733, 1, 0, 0, 0, 732, 722, 1, 0, 0, 0, 732, 726,
|
||||
1, 0, 0, 0, 733, 130, 1, 0, 0, 0, 734, 736, 3, 135, 67, 0, 735, 737, 3,
|
||||
137, 68, 0, 736, 735, 1, 0, 0, 0, 736, 737, 1, 0, 0, 0, 737, 742, 1, 0,
|
||||
0, 0, 738, 739, 3, 139, 69, 0, 739, 740, 3, 137, 68, 0, 740, 742, 1, 0,
|
||||
0, 0, 741, 734, 1, 0, 0, 0, 741, 738, 1, 0, 0, 0, 742, 132, 1, 0, 0, 0,
|
||||
743, 744, 5, 48, 0, 0, 744, 747, 7, 7, 0, 0, 745, 748, 3, 141, 70, 0, 746,
|
||||
748, 3, 143, 71, 0, 747, 745, 1, 0, 0, 0, 747, 746, 1, 0, 0, 0, 748, 749,
|
||||
1, 0, 0, 0, 749, 750, 3, 145, 72, 0, 750, 134, 1, 0, 0, 0, 751, 753, 3,
|
||||
139, 69, 0, 752, 751, 1, 0, 0, 0, 752, 753, 1, 0, 0, 0, 753, 754, 1, 0,
|
||||
0, 0, 754, 755, 5, 46, 0, 0, 755, 760, 3, 139, 69, 0, 756, 757, 3, 139,
|
||||
69, 0, 757, 758, 5, 46, 0, 0, 758, 760, 1, 0, 0, 0, 759, 752, 1, 0, 0,
|
||||
0, 759, 756, 1, 0, 0, 0, 760, 136, 1, 0, 0, 0, 761, 763, 7, 11, 0, 0, 762,
|
||||
764, 7, 12, 0, 0, 763, 762, 1, 0, 0, 0, 763, 764, 1, 0, 0, 0, 764, 765,
|
||||
1, 0, 0, 0, 765, 766, 3, 139, 69, 0, 766, 138, 1, 0, 0, 0, 767, 769, 3,
|
||||
111, 55, 0, 768, 767, 1, 0, 0, 0, 769, 770, 1, 0, 0, 0, 770, 768, 1, 0,
|
||||
0, 0, 770, 771, 1, 0, 0, 0, 771, 140, 1, 0, 0, 0, 772, 774, 3, 143, 71,
|
||||
0, 773, 772, 1, 0, 0, 0, 773, 774, 1, 0, 0, 0, 774, 775, 1, 0, 0, 0, 775,
|
||||
776, 5, 46, 0, 0, 776, 781, 3, 143, 71, 0, 777, 778, 3, 143, 71, 0, 778,
|
||||
779, 5, 46, 0, 0, 779, 781, 1, 0, 0, 0, 780, 773, 1, 0, 0, 0, 780, 777,
|
||||
1, 0, 0, 0, 781, 142, 1, 0, 0, 0, 782, 784, 3, 125, 62, 0, 783, 782, 1,
|
||||
0, 0, 0, 784, 785, 1, 0, 0, 0, 785, 783, 1, 0, 0, 0, 785, 786, 1, 0, 0,
|
||||
0, 786, 144, 1, 0, 0, 0, 787, 789, 7, 13, 0, 0, 788, 790, 7, 12, 0, 0,
|
||||
789, 788, 1, 0, 0, 0, 789, 790, 1, 0, 0, 0, 790, 791, 1, 0, 0, 0, 791,
|
||||
792, 3, 139, 69, 0, 792, 146, 1, 0, 0, 0, 793, 794, 5, 92, 0, 0, 794, 809,
|
||||
7, 14, 0, 0, 795, 796, 5, 92, 0, 0, 796, 798, 3, 123, 61, 0, 797, 799,
|
||||
3, 123, 61, 0, 798, 797, 1, 0, 0, 0, 798, 799, 1, 0, 0, 0, 799, 801, 1,
|
||||
0, 0, 0, 800, 802, 3, 123, 61, 0, 801, 800, 1, 0, 0, 0, 801, 802, 1, 0,
|
||||
0, 0, 802, 809, 1, 0, 0, 0, 803, 804, 5, 92, 0, 0, 804, 805, 5, 120, 0,
|
||||
0, 805, 806, 1, 0, 0, 0, 806, 809, 3, 143, 71, 0, 807, 809, 3, 129, 64,
|
||||
0, 808, 793, 1, 0, 0, 0, 808, 795, 1, 0, 0, 0, 808, 803, 1, 0, 0, 0, 808,
|
||||
807, 1, 0, 0, 0, 809, 148, 1, 0, 0, 0, 810, 812, 7, 15, 0, 0, 811, 810,
|
||||
1, 0, 0, 0, 812, 813, 1, 0, 0, 0, 813, 811, 1, 0, 0, 0, 813, 814, 1, 0,
|
||||
0, 0, 814, 815, 1, 0, 0, 0, 815, 816, 6, 74, 0, 0, 816, 150, 1, 0, 0, 0,
|
||||
817, 819, 5, 13, 0, 0, 818, 820, 5, 10, 0, 0, 819, 818, 1, 0, 0, 0, 819,
|
||||
820, 1, 0, 0, 0, 820, 823, 1, 0, 0, 0, 821, 823, 5, 10, 0, 0, 822, 817,
|
||||
1, 0, 0, 0, 822, 821, 1, 0, 0, 0, 823, 824, 1, 0, 0, 0, 824, 825, 6, 75,
|
||||
0, 0, 825, 152, 1, 0, 0, 0, 58, 0, 191, 205, 227, 259, 265, 281, 305, 313,
|
||||
319, 324, 326, 357, 393, 429, 459, 497, 535, 561, 590, 596, 600, 605, 607,
|
||||
617, 621, 626, 629, 633, 638, 644, 649, 654, 659, 668, 677, 688, 694, 698,
|
||||
704, 732, 736, 741, 747, 752, 759, 763, 770, 773, 780, 785, 789, 798, 801,
|
||||
808, 813, 819, 822, 1, 6, 0, 0,
|
||||
}
|
||||
deserializer := antlr.NewATNDeserializer(nil)
|
||||
staticData.atn = deserializer.Deserialize(staticData.serializedATN)
|
||||
|
@ -498,24 +519,26 @@ const (
|
|||
PlanLexerBXOR = 27
|
||||
PlanLexerAND = 28
|
||||
PlanLexerOR = 29
|
||||
PlanLexerBNOT = 30
|
||||
PlanLexerNOT = 31
|
||||
PlanLexerIN = 32
|
||||
PlanLexerEmptyArray = 33
|
||||
PlanLexerJSONContains = 34
|
||||
PlanLexerJSONContainsAll = 35
|
||||
PlanLexerJSONContainsAny = 36
|
||||
PlanLexerArrayContains = 37
|
||||
PlanLexerArrayContainsAll = 38
|
||||
PlanLexerArrayContainsAny = 39
|
||||
PlanLexerArrayLength = 40
|
||||
PlanLexerBooleanConstant = 41
|
||||
PlanLexerIntegerConstant = 42
|
||||
PlanLexerFloatingConstant = 43
|
||||
PlanLexerIdentifier = 44
|
||||
PlanLexerMeta = 45
|
||||
PlanLexerStringLiteral = 46
|
||||
PlanLexerJSONIdentifier = 47
|
||||
PlanLexerWhitespace = 48
|
||||
PlanLexerNewline = 49
|
||||
PlanLexerISNULL = 30
|
||||
PlanLexerISNOTNULL = 31
|
||||
PlanLexerBNOT = 32
|
||||
PlanLexerNOT = 33
|
||||
PlanLexerIN = 34
|
||||
PlanLexerEmptyArray = 35
|
||||
PlanLexerJSONContains = 36
|
||||
PlanLexerJSONContainsAll = 37
|
||||
PlanLexerJSONContainsAny = 38
|
||||
PlanLexerArrayContains = 39
|
||||
PlanLexerArrayContainsAll = 40
|
||||
PlanLexerArrayContainsAny = 41
|
||||
PlanLexerArrayLength = 42
|
||||
PlanLexerBooleanConstant = 43
|
||||
PlanLexerIntegerConstant = 44
|
||||
PlanLexerFloatingConstant = 45
|
||||
PlanLexerIdentifier = 46
|
||||
PlanLexerMeta = 47
|
||||
PlanLexerStringLiteral = 48
|
||||
PlanLexerJSONIdentifier = 49
|
||||
PlanLexerWhitespace = 50
|
||||
PlanLexerNewline = 51
|
||||
)
|
||||
|
|
|
@ -34,92 +34,95 @@ func planParserInit() {
|
|||
staticData.LiteralNames = []string{
|
||||
"", "'('", "')'", "'['", "','", "']'", "'{'", "'}'", "'<'", "'<='",
|
||||
"'>'", "'>='", "'=='", "'!='", "", "", "", "'+'", "'-'", "'*'", "'/'",
|
||||
"'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "'~'", "",
|
||||
"", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'",
|
||||
"'%'", "'**'", "'<<'", "'>>'", "'&'", "'|'", "'^'", "", "", "", "",
|
||||
"'~'", "", "", "", "", "", "", "", "", "", "", "", "", "", "", "'$meta'",
|
||||
}
|
||||
staticData.SymbolicNames = []string{
|
||||
"", "", "", "", "", "", "LBRACE", "RBRACE", "LT", "LE", "GT", "GE",
|
||||
"EQ", "NE", "LIKE", "EXISTS", "TEXTMATCH", "ADD", "SUB", "MUL", "DIV",
|
||||
"MOD", "POW", "SHL", "SHR", "BAND", "BOR", "BXOR", "AND", "OR", "BNOT",
|
||||
"NOT", "IN", "EmptyArray", "JSONContains", "JSONContainsAll", "JSONContainsAny",
|
||||
"ArrayContains", "ArrayContainsAll", "ArrayContainsAny", "ArrayLength",
|
||||
"BooleanConstant", "IntegerConstant", "FloatingConstant", "Identifier",
|
||||
"Meta", "StringLiteral", "JSONIdentifier", "Whitespace", "Newline",
|
||||
"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{
|
||||
"expr",
|
||||
}
|
||||
staticData.PredictionContextCache = antlr.NewPredictionContextCache()
|
||||
staticData.serializedATN = []int32{
|
||||
4, 1, 49, 142, 2, 0, 7, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
4, 1, 51, 146, 2, 0, 7, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 21,
|
||||
8, 0, 10, 0, 12, 0, 24, 9, 0, 1, 0, 3, 0, 27, 8, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 5, 0, 70, 8, 0, 10, 0, 12, 0, 73, 9, 0, 1, 0, 3, 0, 76, 8, 0,
|
||||
3, 0, 78, 8, 0, 1, 0, 1, 0, 1, 0, 3, 0, 83, 8, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 99,
|
||||
8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0,
|
||||
1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0,
|
||||
1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0,
|
||||
1, 0, 5, 0, 137, 8, 0, 10, 0, 12, 0, 140, 9, 0, 1, 0, 0, 1, 0, 1, 0, 0,
|
||||
13, 1, 0, 44, 45, 2, 0, 17, 18, 30, 31, 2, 0, 34, 34, 37, 37, 2, 0, 35,
|
||||
35, 38, 38, 2, 0, 36, 36, 39, 39, 2, 0, 44, 44, 47, 47, 1, 0, 19, 21, 1,
|
||||
0, 17, 18, 1, 0, 23, 24, 1, 0, 8, 9, 1, 0, 10, 11, 1, 0, 8, 11, 1, 0, 12,
|
||||
13, 178, 0, 82, 1, 0, 0, 0, 2, 3, 6, 0, -1, 0, 3, 83, 5, 42, 0, 0, 4, 83,
|
||||
5, 43, 0, 0, 5, 83, 5, 41, 0, 0, 6, 83, 5, 46, 0, 0, 7, 83, 7, 0, 0, 0,
|
||||
8, 83, 5, 47, 0, 0, 9, 10, 5, 6, 0, 0, 10, 11, 5, 44, 0, 0, 11, 83, 5,
|
||||
7, 0, 0, 12, 13, 5, 1, 0, 0, 13, 14, 3, 0, 0, 0, 14, 15, 5, 2, 0, 0, 15,
|
||||
83, 1, 0, 0, 0, 16, 17, 5, 3, 0, 0, 17, 22, 3, 0, 0, 0, 18, 19, 5, 4, 0,
|
||||
0, 19, 21, 3, 0, 0, 0, 20, 18, 1, 0, 0, 0, 21, 24, 1, 0, 0, 0, 22, 20,
|
||||
1, 0, 0, 0, 22, 23, 1, 0, 0, 0, 23, 26, 1, 0, 0, 0, 24, 22, 1, 0, 0, 0,
|
||||
25, 27, 5, 4, 0, 0, 26, 25, 1, 0, 0, 0, 26, 27, 1, 0, 0, 0, 27, 28, 1,
|
||||
0, 0, 0, 28, 29, 5, 5, 0, 0, 29, 83, 1, 0, 0, 0, 30, 83, 5, 33, 0, 0, 31,
|
||||
32, 5, 16, 0, 0, 32, 33, 5, 1, 0, 0, 33, 34, 5, 44, 0, 0, 34, 35, 5, 4,
|
||||
0, 0, 35, 36, 5, 46, 0, 0, 36, 83, 5, 2, 0, 0, 37, 38, 7, 1, 0, 0, 38,
|
||||
83, 3, 0, 0, 20, 39, 40, 7, 2, 0, 0, 40, 41, 5, 1, 0, 0, 41, 42, 3, 0,
|
||||
0, 0, 42, 43, 5, 4, 0, 0, 43, 44, 3, 0, 0, 0, 44, 45, 5, 2, 0, 0, 45, 83,
|
||||
1, 0, 0, 0, 46, 47, 7, 3, 0, 0, 47, 48, 5, 1, 0, 0, 48, 49, 3, 0, 0, 0,
|
||||
49, 50, 5, 4, 0, 0, 50, 51, 3, 0, 0, 0, 51, 52, 5, 2, 0, 0, 52, 83, 1,
|
||||
0, 0, 0, 53, 54, 7, 4, 0, 0, 54, 55, 5, 1, 0, 0, 55, 56, 3, 0, 0, 0, 56,
|
||||
57, 5, 4, 0, 0, 57, 58, 3, 0, 0, 0, 58, 59, 5, 2, 0, 0, 59, 83, 1, 0, 0,
|
||||
0, 60, 61, 5, 40, 0, 0, 61, 62, 5, 1, 0, 0, 62, 63, 7, 5, 0, 0, 63, 83,
|
||||
5, 2, 0, 0, 64, 65, 5, 44, 0, 0, 65, 77, 5, 1, 0, 0, 66, 71, 3, 0, 0, 0,
|
||||
67, 68, 5, 4, 0, 0, 68, 70, 3, 0, 0, 0, 69, 67, 1, 0, 0, 0, 70, 73, 1,
|
||||
0, 0, 0, 71, 69, 1, 0, 0, 0, 71, 72, 1, 0, 0, 0, 72, 75, 1, 0, 0, 0, 73,
|
||||
71, 1, 0, 0, 0, 74, 76, 5, 4, 0, 0, 75, 74, 1, 0, 0, 0, 75, 76, 1, 0, 0,
|
||||
0, 76, 78, 1, 0, 0, 0, 77, 66, 1, 0, 0, 0, 77, 78, 1, 0, 0, 0, 78, 79,
|
||||
1, 0, 0, 0, 79, 83, 5, 2, 0, 0, 80, 81, 5, 15, 0, 0, 81, 83, 3, 0, 0, 1,
|
||||
82, 2, 1, 0, 0, 0, 82, 4, 1, 0, 0, 0, 82, 5, 1, 0, 0, 0, 82, 6, 1, 0, 0,
|
||||
0, 82, 7, 1, 0, 0, 0, 82, 8, 1, 0, 0, 0, 82, 9, 1, 0, 0, 0, 82, 12, 1,
|
||||
0, 0, 0, 82, 16, 1, 0, 0, 0, 82, 30, 1, 0, 0, 0, 82, 31, 1, 0, 0, 0, 82,
|
||||
37, 1, 0, 0, 0, 82, 39, 1, 0, 0, 0, 82, 46, 1, 0, 0, 0, 82, 53, 1, 0, 0,
|
||||
0, 82, 60, 1, 0, 0, 0, 82, 64, 1, 0, 0, 0, 82, 80, 1, 0, 0, 0, 83, 138,
|
||||
1, 0, 0, 0, 84, 85, 10, 21, 0, 0, 85, 86, 5, 22, 0, 0, 86, 137, 3, 0, 0,
|
||||
22, 87, 88, 10, 19, 0, 0, 88, 89, 7, 6, 0, 0, 89, 137, 3, 0, 0, 20, 90,
|
||||
91, 10, 18, 0, 0, 91, 92, 7, 7, 0, 0, 92, 137, 3, 0, 0, 19, 93, 94, 10,
|
||||
17, 0, 0, 94, 95, 7, 8, 0, 0, 95, 137, 3, 0, 0, 18, 96, 98, 10, 16, 0,
|
||||
0, 97, 99, 5, 31, 0, 0, 98, 97, 1, 0, 0, 0, 98, 99, 1, 0, 0, 0, 99, 100,
|
||||
1, 0, 0, 0, 100, 101, 5, 32, 0, 0, 101, 137, 3, 0, 0, 17, 102, 103, 10,
|
||||
10, 0, 0, 103, 104, 7, 9, 0, 0, 104, 105, 7, 5, 0, 0, 105, 106, 7, 9, 0,
|
||||
0, 106, 137, 3, 0, 0, 11, 107, 108, 10, 9, 0, 0, 108, 109, 7, 10, 0, 0,
|
||||
109, 110, 7, 5, 0, 0, 110, 111, 7, 10, 0, 0, 111, 137, 3, 0, 0, 10, 112,
|
||||
113, 10, 8, 0, 0, 113, 114, 7, 11, 0, 0, 114, 137, 3, 0, 0, 9, 115, 116,
|
||||
10, 7, 0, 0, 116, 117, 7, 12, 0, 0, 117, 137, 3, 0, 0, 8, 118, 119, 10,
|
||||
6, 0, 0, 119, 120, 5, 25, 0, 0, 120, 137, 3, 0, 0, 7, 121, 122, 10, 5,
|
||||
0, 0, 122, 123, 5, 27, 0, 0, 123, 137, 3, 0, 0, 6, 124, 125, 10, 4, 0,
|
||||
0, 125, 126, 5, 26, 0, 0, 126, 137, 3, 0, 0, 5, 127, 128, 10, 3, 0, 0,
|
||||
128, 129, 5, 28, 0, 0, 129, 137, 3, 0, 0, 4, 130, 131, 10, 2, 0, 0, 131,
|
||||
132, 5, 29, 0, 0, 132, 137, 3, 0, 0, 3, 133, 134, 10, 23, 0, 0, 134, 135,
|
||||
5, 14, 0, 0, 135, 137, 5, 46, 0, 0, 136, 84, 1, 0, 0, 0, 136, 87, 1, 0,
|
||||
0, 0, 136, 90, 1, 0, 0, 0, 136, 93, 1, 0, 0, 0, 136, 96, 1, 0, 0, 0, 136,
|
||||
102, 1, 0, 0, 0, 136, 107, 1, 0, 0, 0, 136, 112, 1, 0, 0, 0, 136, 115,
|
||||
1, 0, 0, 0, 136, 118, 1, 0, 0, 0, 136, 121, 1, 0, 0, 0, 136, 124, 1, 0,
|
||||
0, 0, 136, 127, 1, 0, 0, 0, 136, 130, 1, 0, 0, 0, 136, 133, 1, 0, 0, 0,
|
||||
137, 140, 1, 0, 0, 0, 138, 136, 1, 0, 0, 0, 138, 139, 1, 0, 0, 0, 139,
|
||||
1, 1, 0, 0, 0, 140, 138, 1, 0, 0, 0, 9, 22, 26, 71, 75, 77, 82, 98, 136,
|
||||
138,
|
||||
3, 0, 78, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 3, 0, 87, 8,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 3, 0, 103, 8, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1,
|
||||
0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 5, 0, 141, 8, 0, 10, 0, 12, 0, 144, 9,
|
||||
0, 1, 0, 0, 1, 0, 1, 0, 0, 13, 1, 0, 46, 47, 2, 0, 17, 18, 32, 33, 2, 0,
|
||||
36, 36, 39, 39, 2, 0, 37, 37, 40, 40, 2, 0, 38, 38, 41, 41, 2, 0, 46, 46,
|
||||
49, 49, 1, 0, 19, 21, 1, 0, 17, 18, 1, 0, 23, 24, 1, 0, 8, 9, 1, 0, 10,
|
||||
11, 1, 0, 8, 11, 1, 0, 12, 13, 184, 0, 86, 1, 0, 0, 0, 2, 3, 6, 0, -1,
|
||||
0, 3, 87, 5, 44, 0, 0, 4, 87, 5, 45, 0, 0, 5, 87, 5, 43, 0, 0, 6, 87, 5,
|
||||
48, 0, 0, 7, 87, 7, 0, 0, 0, 8, 87, 5, 49, 0, 0, 9, 10, 5, 6, 0, 0, 10,
|
||||
11, 5, 46, 0, 0, 11, 87, 5, 7, 0, 0, 12, 13, 5, 1, 0, 0, 13, 14, 3, 0,
|
||||
0, 0, 14, 15, 5, 2, 0, 0, 15, 87, 1, 0, 0, 0, 16, 17, 5, 3, 0, 0, 17, 22,
|
||||
3, 0, 0, 0, 18, 19, 5, 4, 0, 0, 19, 21, 3, 0, 0, 0, 20, 18, 1, 0, 0, 0,
|
||||
21, 24, 1, 0, 0, 0, 22, 20, 1, 0, 0, 0, 22, 23, 1, 0, 0, 0, 23, 26, 1,
|
||||
0, 0, 0, 24, 22, 1, 0, 0, 0, 25, 27, 5, 4, 0, 0, 26, 25, 1, 0, 0, 0, 26,
|
||||
27, 1, 0, 0, 0, 27, 28, 1, 0, 0, 0, 28, 29, 5, 5, 0, 0, 29, 87, 1, 0, 0,
|
||||
0, 30, 87, 5, 35, 0, 0, 31, 32, 5, 16, 0, 0, 32, 33, 5, 1, 0, 0, 33, 34,
|
||||
5, 46, 0, 0, 34, 35, 5, 4, 0, 0, 35, 36, 5, 48, 0, 0, 36, 87, 5, 2, 0,
|
||||
0, 37, 38, 7, 1, 0, 0, 38, 87, 3, 0, 0, 22, 39, 40, 7, 2, 0, 0, 40, 41,
|
||||
5, 1, 0, 0, 41, 42, 3, 0, 0, 0, 42, 43, 5, 4, 0, 0, 43, 44, 3, 0, 0, 0,
|
||||
44, 45, 5, 2, 0, 0, 45, 87, 1, 0, 0, 0, 46, 47, 7, 3, 0, 0, 47, 48, 5,
|
||||
1, 0, 0, 48, 49, 3, 0, 0, 0, 49, 50, 5, 4, 0, 0, 50, 51, 3, 0, 0, 0, 51,
|
||||
52, 5, 2, 0, 0, 52, 87, 1, 0, 0, 0, 53, 54, 7, 4, 0, 0, 54, 55, 5, 1, 0,
|
||||
0, 55, 56, 3, 0, 0, 0, 56, 57, 5, 4, 0, 0, 57, 58, 3, 0, 0, 0, 58, 59,
|
||||
5, 2, 0, 0, 59, 87, 1, 0, 0, 0, 60, 61, 5, 42, 0, 0, 61, 62, 5, 1, 0, 0,
|
||||
62, 63, 7, 5, 0, 0, 63, 87, 5, 2, 0, 0, 64, 65, 5, 46, 0, 0, 65, 77, 5,
|
||||
1, 0, 0, 66, 71, 3, 0, 0, 0, 67, 68, 5, 4, 0, 0, 68, 70, 3, 0, 0, 0, 69,
|
||||
67, 1, 0, 0, 0, 70, 73, 1, 0, 0, 0, 71, 69, 1, 0, 0, 0, 71, 72, 1, 0, 0,
|
||||
0, 72, 75, 1, 0, 0, 0, 73, 71, 1, 0, 0, 0, 74, 76, 5, 4, 0, 0, 75, 74,
|
||||
1, 0, 0, 0, 75, 76, 1, 0, 0, 0, 76, 78, 1, 0, 0, 0, 77, 66, 1, 0, 0, 0,
|
||||
77, 78, 1, 0, 0, 0, 78, 79, 1, 0, 0, 0, 79, 87, 5, 2, 0, 0, 80, 81, 5,
|
||||
46, 0, 0, 81, 87, 5, 30, 0, 0, 82, 83, 5, 46, 0, 0, 83, 87, 5, 31, 0, 0,
|
||||
84, 85, 5, 15, 0, 0, 85, 87, 3, 0, 0, 1, 86, 2, 1, 0, 0, 0, 86, 4, 1, 0,
|
||||
0, 0, 86, 5, 1, 0, 0, 0, 86, 6, 1, 0, 0, 0, 86, 7, 1, 0, 0, 0, 86, 8, 1,
|
||||
0, 0, 0, 86, 9, 1, 0, 0, 0, 86, 12, 1, 0, 0, 0, 86, 16, 1, 0, 0, 0, 86,
|
||||
30, 1, 0, 0, 0, 86, 31, 1, 0, 0, 0, 86, 37, 1, 0, 0, 0, 86, 39, 1, 0, 0,
|
||||
0, 86, 46, 1, 0, 0, 0, 86, 53, 1, 0, 0, 0, 86, 60, 1, 0, 0, 0, 86, 64,
|
||||
1, 0, 0, 0, 86, 80, 1, 0, 0, 0, 86, 82, 1, 0, 0, 0, 86, 84, 1, 0, 0, 0,
|
||||
87, 142, 1, 0, 0, 0, 88, 89, 10, 23, 0, 0, 89, 90, 5, 22, 0, 0, 90, 141,
|
||||
3, 0, 0, 24, 91, 92, 10, 21, 0, 0, 92, 93, 7, 6, 0, 0, 93, 141, 3, 0, 0,
|
||||
22, 94, 95, 10, 20, 0, 0, 95, 96, 7, 7, 0, 0, 96, 141, 3, 0, 0, 21, 97,
|
||||
98, 10, 19, 0, 0, 98, 99, 7, 8, 0, 0, 99, 141, 3, 0, 0, 20, 100, 102, 10,
|
||||
18, 0, 0, 101, 103, 5, 33, 0, 0, 102, 101, 1, 0, 0, 0, 102, 103, 1, 0,
|
||||
0, 0, 103, 104, 1, 0, 0, 0, 104, 105, 5, 34, 0, 0, 105, 141, 3, 0, 0, 19,
|
||||
106, 107, 10, 12, 0, 0, 107, 108, 7, 9, 0, 0, 108, 109, 7, 5, 0, 0, 109,
|
||||
110, 7, 9, 0, 0, 110, 141, 3, 0, 0, 13, 111, 112, 10, 11, 0, 0, 112, 113,
|
||||
7, 10, 0, 0, 113, 114, 7, 5, 0, 0, 114, 115, 7, 10, 0, 0, 115, 141, 3,
|
||||
0, 0, 12, 116, 117, 10, 10, 0, 0, 117, 118, 7, 11, 0, 0, 118, 141, 3, 0,
|
||||
0, 11, 119, 120, 10, 9, 0, 0, 120, 121, 7, 12, 0, 0, 121, 141, 3, 0, 0,
|
||||
10, 122, 123, 10, 8, 0, 0, 123, 124, 5, 25, 0, 0, 124, 141, 3, 0, 0, 9,
|
||||
125, 126, 10, 7, 0, 0, 126, 127, 5, 27, 0, 0, 127, 141, 3, 0, 0, 8, 128,
|
||||
129, 10, 6, 0, 0, 129, 130, 5, 26, 0, 0, 130, 141, 3, 0, 0, 7, 131, 132,
|
||||
10, 5, 0, 0, 132, 133, 5, 28, 0, 0, 133, 141, 3, 0, 0, 6, 134, 135, 10,
|
||||
4, 0, 0, 135, 136, 5, 29, 0, 0, 136, 141, 3, 0, 0, 5, 137, 138, 10, 25,
|
||||
0, 0, 138, 139, 5, 14, 0, 0, 139, 141, 5, 48, 0, 0, 140, 88, 1, 0, 0, 0,
|
||||
140, 91, 1, 0, 0, 0, 140, 94, 1, 0, 0, 0, 140, 97, 1, 0, 0, 0, 140, 100,
|
||||
1, 0, 0, 0, 140, 106, 1, 0, 0, 0, 140, 111, 1, 0, 0, 0, 140, 116, 1, 0,
|
||||
0, 0, 140, 119, 1, 0, 0, 0, 140, 122, 1, 0, 0, 0, 140, 125, 1, 0, 0, 0,
|
||||
140, 128, 1, 0, 0, 0, 140, 131, 1, 0, 0, 0, 140, 134, 1, 0, 0, 0, 140,
|
||||
137, 1, 0, 0, 0, 141, 144, 1, 0, 0, 0, 142, 140, 1, 0, 0, 0, 142, 143,
|
||||
1, 0, 0, 0, 143, 1, 1, 0, 0, 0, 144, 142, 1, 0, 0, 0, 9, 22, 26, 71, 75,
|
||||
77, 86, 102, 140, 142,
|
||||
}
|
||||
deserializer := antlr.NewATNDeserializer(nil)
|
||||
staticData.atn = deserializer.Deserialize(staticData.serializedATN)
|
||||
|
@ -187,26 +190,28 @@ const (
|
|||
PlanParserBXOR = 27
|
||||
PlanParserAND = 28
|
||||
PlanParserOR = 29
|
||||
PlanParserBNOT = 30
|
||||
PlanParserNOT = 31
|
||||
PlanParserIN = 32
|
||||
PlanParserEmptyArray = 33
|
||||
PlanParserJSONContains = 34
|
||||
PlanParserJSONContainsAll = 35
|
||||
PlanParserJSONContainsAny = 36
|
||||
PlanParserArrayContains = 37
|
||||
PlanParserArrayContainsAll = 38
|
||||
PlanParserArrayContainsAny = 39
|
||||
PlanParserArrayLength = 40
|
||||
PlanParserBooleanConstant = 41
|
||||
PlanParserIntegerConstant = 42
|
||||
PlanParserFloatingConstant = 43
|
||||
PlanParserIdentifier = 44
|
||||
PlanParserMeta = 45
|
||||
PlanParserStringLiteral = 46
|
||||
PlanParserJSONIdentifier = 47
|
||||
PlanParserWhitespace = 48
|
||||
PlanParserNewline = 49
|
||||
PlanParserISNULL = 30
|
||||
PlanParserISNOTNULL = 31
|
||||
PlanParserBNOT = 32
|
||||
PlanParserNOT = 33
|
||||
PlanParserIN = 34
|
||||
PlanParserEmptyArray = 35
|
||||
PlanParserJSONContains = 36
|
||||
PlanParserJSONContainsAll = 37
|
||||
PlanParserJSONContainsAny = 38
|
||||
PlanParserArrayContains = 39
|
||||
PlanParserArrayContainsAll = 40
|
||||
PlanParserArrayContainsAny = 41
|
||||
PlanParserArrayLength = 42
|
||||
PlanParserBooleanConstant = 43
|
||||
PlanParserIntegerConstant = 44
|
||||
PlanParserFloatingConstant = 45
|
||||
PlanParserIdentifier = 46
|
||||
PlanParserMeta = 47
|
||||
PlanParserStringLiteral = 48
|
||||
PlanParserJSONIdentifier = 49
|
||||
PlanParserWhitespace = 50
|
||||
PlanParserNewline = 51
|
||||
)
|
||||
|
||||
// PlanParserRULE_expr is the PlanParser rule.
|
||||
|
@ -556,6 +561,42 @@ func (s *LogicalOrContext) Accept(visitor antlr.ParseTreeVisitor) interface{} {
|
|||
}
|
||||
}
|
||||
|
||||
type IsNotNullContext struct {
|
||||
ExprContext
|
||||
}
|
||||
|
||||
func NewIsNotNullContext(parser antlr.Parser, ctx antlr.ParserRuleContext) *IsNotNullContext {
|
||||
var p = new(IsNotNullContext)
|
||||
|
||||
InitEmptyExprContext(&p.ExprContext)
|
||||
p.parser = parser
|
||||
p.CopyAll(ctx.(*ExprContext))
|
||||
|
||||
return p
|
||||
}
|
||||
|
||||
func (s *IsNotNullContext) GetRuleContext() antlr.RuleContext {
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *IsNotNullContext) Identifier() antlr.TerminalNode {
|
||||
return s.GetToken(PlanParserIdentifier, 0)
|
||||
}
|
||||
|
||||
func (s *IsNotNullContext) ISNOTNULL() antlr.TerminalNode {
|
||||
return s.GetToken(PlanParserISNOTNULL, 0)
|
||||
}
|
||||
|
||||
func (s *IsNotNullContext) Accept(visitor antlr.ParseTreeVisitor) interface{} {
|
||||
switch t := visitor.(type) {
|
||||
case PlanVisitor:
|
||||
return t.VisitIsNotNull(s)
|
||||
|
||||
default:
|
||||
return t.VisitChildren(s)
|
||||
}
|
||||
}
|
||||
|
||||
type MulDivModContext struct {
|
||||
ExprContext
|
||||
op antlr.Token
|
||||
|
@ -2271,6 +2312,42 @@ func (s *BitAndContext) Accept(visitor antlr.ParseTreeVisitor) interface{} {
|
|||
}
|
||||
}
|
||||
|
||||
type IsNullContext struct {
|
||||
ExprContext
|
||||
}
|
||||
|
||||
func NewIsNullContext(parser antlr.Parser, ctx antlr.ParserRuleContext) *IsNullContext {
|
||||
var p = new(IsNullContext)
|
||||
|
||||
InitEmptyExprContext(&p.ExprContext)
|
||||
p.parser = parser
|
||||
p.CopyAll(ctx.(*ExprContext))
|
||||
|
||||
return p
|
||||
}
|
||||
|
||||
func (s *IsNullContext) GetRuleContext() antlr.RuleContext {
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *IsNullContext) Identifier() antlr.TerminalNode {
|
||||
return s.GetToken(PlanParserIdentifier, 0)
|
||||
}
|
||||
|
||||
func (s *IsNullContext) ISNULL() antlr.TerminalNode {
|
||||
return s.GetToken(PlanParserISNULL, 0)
|
||||
}
|
||||
|
||||
func (s *IsNullContext) Accept(visitor antlr.ParseTreeVisitor) interface{} {
|
||||
switch t := visitor.(type) {
|
||||
case PlanVisitor:
|
||||
return t.VisitIsNull(s)
|
||||
|
||||
default:
|
||||
return t.VisitChildren(s)
|
||||
}
|
||||
}
|
||||
|
||||
type PowerContext struct {
|
||||
ExprContext
|
||||
}
|
||||
|
@ -2362,7 +2439,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
var _alt int
|
||||
|
||||
p.EnterOuterAlt(localctx, 1)
|
||||
p.SetState(82)
|
||||
p.SetState(86)
|
||||
p.GetErrorHandler().Sync(p)
|
||||
if p.HasError() {
|
||||
goto errorExit
|
||||
|
@ -2662,7 +2739,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
|
||||
_la = p.GetTokenStream().LA(1)
|
||||
|
||||
if !((int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&3221618688) != 0) {
|
||||
if !((int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&12885295104) != 0) {
|
||||
var _ri = p.GetErrorHandler().RecoverInline(p)
|
||||
|
||||
localctx.(*UnaryContext).op = _ri
|
||||
|
@ -2673,7 +2750,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
{
|
||||
p.SetState(38)
|
||||
p.expr(20)
|
||||
p.expr(22)
|
||||
}
|
||||
|
||||
case 13:
|
||||
|
@ -2887,7 +2964,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
_la = p.GetTokenStream().LA(1)
|
||||
|
||||
if (int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&281469608493130) != 0 {
|
||||
if (int64(_la) & ^0x3f) == 0 && ((int64(1)<<_la)&1125878432497738) != 0 {
|
||||
{
|
||||
p.SetState(66)
|
||||
p.expr(0)
|
||||
|
@ -2957,12 +3034,12 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
|
||||
case 18:
|
||||
localctx = NewExistsContext(p, localctx)
|
||||
localctx = NewIsNullContext(p, localctx)
|
||||
p.SetParserRuleContext(localctx)
|
||||
_prevctx = localctx
|
||||
{
|
||||
p.SetState(80)
|
||||
p.Match(PlanParserEXISTS)
|
||||
p.Match(PlanParserIdentifier)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
goto errorExit
|
||||
|
@ -2970,6 +3047,48 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
{
|
||||
p.SetState(81)
|
||||
p.Match(PlanParserISNULL)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
goto errorExit
|
||||
}
|
||||
}
|
||||
|
||||
case 19:
|
||||
localctx = NewIsNotNullContext(p, localctx)
|
||||
p.SetParserRuleContext(localctx)
|
||||
_prevctx = localctx
|
||||
{
|
||||
p.SetState(82)
|
||||
p.Match(PlanParserIdentifier)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
goto errorExit
|
||||
}
|
||||
}
|
||||
{
|
||||
p.SetState(83)
|
||||
p.Match(PlanParserISNOTNULL)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
goto errorExit
|
||||
}
|
||||
}
|
||||
|
||||
case 20:
|
||||
localctx = NewExistsContext(p, localctx)
|
||||
p.SetParserRuleContext(localctx)
|
||||
_prevctx = localctx
|
||||
{
|
||||
p.SetState(84)
|
||||
p.Match(PlanParserEXISTS)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
goto errorExit
|
||||
}
|
||||
}
|
||||
{
|
||||
p.SetState(85)
|
||||
p.expr(1)
|
||||
}
|
||||
|
||||
|
@ -2977,7 +3096,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
goto errorExit
|
||||
}
|
||||
p.GetParserRuleContext().SetStop(p.GetTokenStream().LT(-1))
|
||||
p.SetState(138)
|
||||
p.SetState(142)
|
||||
p.GetErrorHandler().Sync(p)
|
||||
if p.HasError() {
|
||||
goto errorExit
|
||||
|
@ -2992,7 +3111,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
p.TriggerExitRuleEvent()
|
||||
}
|
||||
_prevctx = localctx
|
||||
p.SetState(136)
|
||||
p.SetState(140)
|
||||
p.GetErrorHandler().Sync(p)
|
||||
if p.HasError() {
|
||||
goto errorExit
|
||||
|
@ -3002,14 +3121,14 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
case 1:
|
||||
localctx = NewPowerContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(84)
|
||||
p.SetState(88)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 21)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 21)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 23)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 23)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(85)
|
||||
p.SetState(89)
|
||||
p.Match(PlanParserPOW)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3017,21 +3136,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(86)
|
||||
p.expr(22)
|
||||
p.SetState(90)
|
||||
p.expr(24)
|
||||
}
|
||||
|
||||
case 2:
|
||||
localctx = NewMulDivModContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(87)
|
||||
p.SetState(91)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 19)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 19)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 21)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 21)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(88)
|
||||
p.SetState(92)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3049,21 +3168,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(89)
|
||||
p.expr(20)
|
||||
p.SetState(93)
|
||||
p.expr(22)
|
||||
}
|
||||
|
||||
case 3:
|
||||
localctx = NewAddSubContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(90)
|
||||
p.SetState(94)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 18)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 18)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 20)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 20)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(91)
|
||||
p.SetState(95)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3081,21 +3200,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(92)
|
||||
p.expr(19)
|
||||
p.SetState(96)
|
||||
p.expr(21)
|
||||
}
|
||||
|
||||
case 4:
|
||||
localctx = NewShiftContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(93)
|
||||
p.SetState(97)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 17)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 17)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 19)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 19)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(94)
|
||||
p.SetState(98)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3113,20 +3232,20 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(95)
|
||||
p.expr(18)
|
||||
p.SetState(99)
|
||||
p.expr(20)
|
||||
}
|
||||
|
||||
case 5:
|
||||
localctx = NewTermContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(96)
|
||||
p.SetState(100)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 16)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 16)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 18)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 18)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
p.SetState(98)
|
||||
p.SetState(102)
|
||||
p.GetErrorHandler().Sync(p)
|
||||
if p.HasError() {
|
||||
goto errorExit
|
||||
|
@ -3135,7 +3254,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
|
||||
if _la == PlanParserNOT {
|
||||
{
|
||||
p.SetState(97)
|
||||
p.SetState(101)
|
||||
|
||||
var _m = p.Match(PlanParserNOT)
|
||||
|
||||
|
@ -3148,7 +3267,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
|
||||
}
|
||||
{
|
||||
p.SetState(100)
|
||||
p.SetState(104)
|
||||
p.Match(PlanParserIN)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3156,21 +3275,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(101)
|
||||
p.expr(17)
|
||||
p.SetState(105)
|
||||
p.expr(19)
|
||||
}
|
||||
|
||||
case 6:
|
||||
localctx = NewRangeContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(102)
|
||||
p.SetState(106)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 10)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 10)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 12)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 12)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(103)
|
||||
p.SetState(107)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3188,7 +3307,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(104)
|
||||
p.SetState(108)
|
||||
_la = p.GetTokenStream().LA(1)
|
||||
|
||||
if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) {
|
||||
|
@ -3199,7 +3318,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(105)
|
||||
p.SetState(109)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3217,21 +3336,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(106)
|
||||
p.expr(11)
|
||||
p.SetState(110)
|
||||
p.expr(13)
|
||||
}
|
||||
|
||||
case 7:
|
||||
localctx = NewReverseRangeContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(107)
|
||||
p.SetState(111)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 9)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 9)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 11)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 11)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(108)
|
||||
p.SetState(112)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3249,7 +3368,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(109)
|
||||
p.SetState(113)
|
||||
_la = p.GetTokenStream().LA(1)
|
||||
|
||||
if !(_la == PlanParserIdentifier || _la == PlanParserJSONIdentifier) {
|
||||
|
@ -3260,7 +3379,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(110)
|
||||
p.SetState(114)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3278,21 +3397,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(111)
|
||||
p.expr(10)
|
||||
p.SetState(115)
|
||||
p.expr(12)
|
||||
}
|
||||
|
||||
case 8:
|
||||
localctx = NewRelationalContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(112)
|
||||
p.SetState(116)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 8)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 8)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 10)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 10)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(113)
|
||||
p.SetState(117)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3310,21 +3429,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(114)
|
||||
p.expr(9)
|
||||
p.SetState(118)
|
||||
p.expr(11)
|
||||
}
|
||||
|
||||
case 9:
|
||||
localctx = NewEqualityContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(115)
|
||||
p.SetState(119)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 7)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 7)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 9)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 9)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(116)
|
||||
p.SetState(120)
|
||||
|
||||
var _lt = p.GetTokenStream().LT(1)
|
||||
|
||||
|
@ -3342,21 +3461,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(117)
|
||||
p.expr(8)
|
||||
p.SetState(121)
|
||||
p.expr(10)
|
||||
}
|
||||
|
||||
case 10:
|
||||
localctx = NewBitAndContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(118)
|
||||
p.SetState(122)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 6)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 6)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 8)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 8)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(119)
|
||||
p.SetState(123)
|
||||
p.Match(PlanParserBAND)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3364,21 +3483,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(120)
|
||||
p.expr(7)
|
||||
p.SetState(124)
|
||||
p.expr(9)
|
||||
}
|
||||
|
||||
case 11:
|
||||
localctx = NewBitXorContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(121)
|
||||
p.SetState(125)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 5)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 5)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 7)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 7)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(122)
|
||||
p.SetState(126)
|
||||
p.Match(PlanParserBXOR)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3386,21 +3505,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(123)
|
||||
p.expr(6)
|
||||
p.SetState(127)
|
||||
p.expr(8)
|
||||
}
|
||||
|
||||
case 12:
|
||||
localctx = NewBitOrContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(124)
|
||||
p.SetState(128)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 4)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 4)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 6)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 6)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(125)
|
||||
p.SetState(129)
|
||||
p.Match(PlanParserBOR)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3408,21 +3527,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(126)
|
||||
p.expr(5)
|
||||
p.SetState(130)
|
||||
p.expr(7)
|
||||
}
|
||||
|
||||
case 13:
|
||||
localctx = NewLogicalAndContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(127)
|
||||
p.SetState(131)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 3)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 3)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 5)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 5)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(128)
|
||||
p.SetState(132)
|
||||
p.Match(PlanParserAND)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3430,21 +3549,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(129)
|
||||
p.expr(4)
|
||||
p.SetState(133)
|
||||
p.expr(6)
|
||||
}
|
||||
|
||||
case 14:
|
||||
localctx = NewLogicalOrContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(130)
|
||||
p.SetState(134)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 2)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 2)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 4)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 4)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(131)
|
||||
p.SetState(135)
|
||||
p.Match(PlanParserOR)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3452,21 +3571,21 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(132)
|
||||
p.expr(3)
|
||||
p.SetState(136)
|
||||
p.expr(5)
|
||||
}
|
||||
|
||||
case 15:
|
||||
localctx = NewLikeContext(p, NewExprContext(p, _parentctx, _parentState))
|
||||
p.PushNewRecursionContext(localctx, _startState, PlanParserRULE_expr)
|
||||
p.SetState(133)
|
||||
p.SetState(137)
|
||||
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 23)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 23)", ""))
|
||||
if !(p.Precpred(p.GetParserRuleContext(), 25)) {
|
||||
p.SetError(antlr.NewFailedPredicateException(p, "p.Precpred(p.GetParserRuleContext(), 25)", ""))
|
||||
goto errorExit
|
||||
}
|
||||
{
|
||||
p.SetState(134)
|
||||
p.SetState(138)
|
||||
p.Match(PlanParserLIKE)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3474,7 +3593,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
}
|
||||
{
|
||||
p.SetState(135)
|
||||
p.SetState(139)
|
||||
p.Match(PlanParserStringLiteral)
|
||||
if p.HasError() {
|
||||
// Recognition error - abort rule
|
||||
|
@ -3487,7 +3606,7 @@ func (p *PlanParser) expr(_p int) (localctx IExprContext) {
|
|||
}
|
||||
|
||||
}
|
||||
p.SetState(140)
|
||||
p.SetState(144)
|
||||
p.GetErrorHandler().Sync(p)
|
||||
if p.HasError() {
|
||||
goto errorExit
|
||||
|
@ -3528,49 +3647,49 @@ func (p *PlanParser) Sempred(localctx antlr.RuleContext, ruleIndex, predIndex in
|
|||
func (p *PlanParser) Expr_Sempred(localctx antlr.RuleContext, predIndex int) bool {
|
||||
switch predIndex {
|
||||
case 0:
|
||||
return p.Precpred(p.GetParserRuleContext(), 21)
|
||||
return p.Precpred(p.GetParserRuleContext(), 23)
|
||||
|
||||
case 1:
|
||||
return p.Precpred(p.GetParserRuleContext(), 19)
|
||||
return p.Precpred(p.GetParserRuleContext(), 21)
|
||||
|
||||
case 2:
|
||||
return p.Precpred(p.GetParserRuleContext(), 18)
|
||||
return p.Precpred(p.GetParserRuleContext(), 20)
|
||||
|
||||
case 3:
|
||||
return p.Precpred(p.GetParserRuleContext(), 17)
|
||||
return p.Precpred(p.GetParserRuleContext(), 19)
|
||||
|
||||
case 4:
|
||||
return p.Precpred(p.GetParserRuleContext(), 16)
|
||||
return p.Precpred(p.GetParserRuleContext(), 18)
|
||||
|
||||
case 5:
|
||||
return p.Precpred(p.GetParserRuleContext(), 10)
|
||||
return p.Precpred(p.GetParserRuleContext(), 12)
|
||||
|
||||
case 6:
|
||||
return p.Precpred(p.GetParserRuleContext(), 9)
|
||||
return p.Precpred(p.GetParserRuleContext(), 11)
|
||||
|
||||
case 7:
|
||||
return p.Precpred(p.GetParserRuleContext(), 8)
|
||||
return p.Precpred(p.GetParserRuleContext(), 10)
|
||||
|
||||
case 8:
|
||||
return p.Precpred(p.GetParserRuleContext(), 7)
|
||||
return p.Precpred(p.GetParserRuleContext(), 9)
|
||||
|
||||
case 9:
|
||||
return p.Precpred(p.GetParserRuleContext(), 6)
|
||||
return p.Precpred(p.GetParserRuleContext(), 8)
|
||||
|
||||
case 10:
|
||||
return p.Precpred(p.GetParserRuleContext(), 5)
|
||||
return p.Precpred(p.GetParserRuleContext(), 7)
|
||||
|
||||
case 11:
|
||||
return p.Precpred(p.GetParserRuleContext(), 4)
|
||||
return p.Precpred(p.GetParserRuleContext(), 6)
|
||||
|
||||
case 12:
|
||||
return p.Precpred(p.GetParserRuleContext(), 3)
|
||||
return p.Precpred(p.GetParserRuleContext(), 5)
|
||||
|
||||
case 13:
|
||||
return p.Precpred(p.GetParserRuleContext(), 2)
|
||||
return p.Precpred(p.GetParserRuleContext(), 4)
|
||||
|
||||
case 14:
|
||||
return p.Precpred(p.GetParserRuleContext(), 23)
|
||||
return p.Precpred(p.GetParserRuleContext(), 25)
|
||||
|
||||
default:
|
||||
panic("No predicate with index: " + fmt.Sprint(predIndex))
|
||||
|
|
|
@ -25,6 +25,9 @@ type PlanVisitor interface {
|
|||
// Visit a parse tree produced by PlanParser#LogicalOr.
|
||||
VisitLogicalOr(ctx *LogicalOrContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by PlanParser#IsNotNull.
|
||||
VisitIsNotNull(ctx *IsNotNullContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by PlanParser#MulDivMod.
|
||||
VisitMulDivMod(ctx *MulDivModContext) interface{}
|
||||
|
||||
|
@ -103,6 +106,9 @@ type PlanVisitor interface {
|
|||
// Visit a parse tree produced by PlanParser#BitAnd.
|
||||
VisitBitAnd(ctx *BitAndContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by PlanParser#IsNull.
|
||||
VisitIsNull(ctx *IsNullContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by PlanParser#Power.
|
||||
VisitPower(ctx *PowerContext) interface{}
|
||||
}
|
||||
|
|
|
@ -51,6 +51,7 @@ func (v *ParserVisitor) translateIdentifier(identifier string) (*ExprWithType, e
|
|||
IsPartitionKey: field.IsPartitionKey,
|
||||
IsClusteringKey: field.IsClusteringKey,
|
||||
ElementType: field.GetElementType(),
|
||||
Nullable: field.GetNullable(),
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -1180,6 +1181,46 @@ func (v *ParserVisitor) VisitEmptyArray(ctx *parser.EmptyArrayContext) interface
|
|||
}
|
||||
}
|
||||
|
||||
func (v *ParserVisitor) VisitIsNotNull(ctx *parser.IsNotNullContext) interface{} {
|
||||
column, err := v.translateIdentifier(ctx.Identifier().GetText())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
expr := &planpb.Expr{
|
||||
Expr: &planpb.Expr_NullExpr{
|
||||
NullExpr: &planpb.NullExpr{
|
||||
ColumnInfo: toColumnInfo(column),
|
||||
Op: planpb.NullExpr_IsNotNull,
|
||||
},
|
||||
},
|
||||
}
|
||||
return &ExprWithType{
|
||||
expr: expr,
|
||||
dataType: schemapb.DataType_Bool,
|
||||
}
|
||||
}
|
||||
|
||||
func (v *ParserVisitor) VisitIsNull(ctx *parser.IsNullContext) interface{} {
|
||||
column, err := v.translateIdentifier(ctx.Identifier().GetText())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
expr := &planpb.Expr{
|
||||
Expr: &planpb.Expr_NullExpr{
|
||||
NullExpr: &planpb.NullExpr{
|
||||
ColumnInfo: toColumnInfo(column),
|
||||
Op: planpb.NullExpr_IsNull,
|
||||
},
|
||||
},
|
||||
}
|
||||
return &ExprWithType{
|
||||
expr: expr,
|
||||
dataType: schemapb.DataType_Bool,
|
||||
}
|
||||
}
|
||||
|
||||
func (v *ParserVisitor) VisitJSONContains(ctx *parser.JSONContainsContext) interface{} {
|
||||
field := ctx.Expr(0).Accept(v)
|
||||
if err := getError(field); err != nil {
|
||||
|
|
|
@ -16,7 +16,7 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
func newTestSchema() *schemapb.CollectionSchema {
|
||||
func newTestSchema(EnableDynamicField bool) *schemapb.CollectionSchema {
|
||||
fields := []*schemapb.FieldSchema{
|
||||
{FieldID: 0, Name: "FieldID", IsPrimaryKey: false, Description: "field no.1", DataType: schemapb.DataType_Int64},
|
||||
}
|
||||
|
@ -31,10 +31,13 @@ func newTestSchema() *schemapb.CollectionSchema {
|
|||
}
|
||||
fields = append(fields, newField)
|
||||
}
|
||||
fields = append(fields, &schemapb.FieldSchema{
|
||||
FieldID: 130, Name: common.MetaFieldName, IsPrimaryKey: false, Description: "dynamic field", DataType: schemapb.DataType_JSON,
|
||||
IsDynamic: true,
|
||||
})
|
||||
if EnableDynamicField {
|
||||
fields = append(fields, &schemapb.FieldSchema{
|
||||
FieldID: 130, Name: common.MetaFieldName, IsPrimaryKey: false, Description: "dynamic field", DataType: schemapb.DataType_JSON,
|
||||
IsDynamic: true,
|
||||
})
|
||||
}
|
||||
|
||||
fields = append(fields, &schemapb.FieldSchema{
|
||||
FieldID: 131, Name: "StringArrayField", IsPrimaryKey: false, Description: "string array field",
|
||||
DataType: schemapb.DataType_Array,
|
||||
|
@ -46,12 +49,12 @@ func newTestSchema() *schemapb.CollectionSchema {
|
|||
Description: "schema for test used",
|
||||
AutoID: true,
|
||||
Fields: fields,
|
||||
EnableDynamicField: true,
|
||||
EnableDynamicField: EnableDynamicField,
|
||||
}
|
||||
}
|
||||
|
||||
func newTestSchemaHelper(t *testing.T) *typeutil.SchemaHelper {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
require.NoError(t, err)
|
||||
return schemaHelper
|
||||
|
@ -71,7 +74,7 @@ func assertInvalidExpr(t *testing.T, helper *typeutil.SchemaHelper, exprStr stri
|
|||
}
|
||||
|
||||
func TestExpr_Term(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -105,7 +108,7 @@ func TestExpr_Term(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Call(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -142,7 +145,7 @@ func TestExpr_Call(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Compare(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -162,7 +165,7 @@ func TestExpr_Compare(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_UnaryRange(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -185,7 +188,7 @@ func TestExpr_UnaryRange(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Like(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -211,7 +214,7 @@ func TestExpr_Like(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_TextMatch(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -231,8 +234,51 @@ func TestExpr_TextMatch(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestExpr_IsNull(t *testing.T) {
|
||||
schema := newTestSchema(false)
|
||||
schema.EnableDynamicField = false
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
exprStrs := []string{
|
||||
`VarCharField is null`,
|
||||
`VarCharField IS NULL`,
|
||||
}
|
||||
for _, exprStr := range exprStrs {
|
||||
assertValidExpr(t, helper, exprStr)
|
||||
}
|
||||
|
||||
unsupported := []string{
|
||||
`not_exist is null`,
|
||||
}
|
||||
for _, exprStr := range unsupported {
|
||||
assertInvalidExpr(t, helper, exprStr)
|
||||
}
|
||||
}
|
||||
|
||||
func TestExpr_IsNotNull(t *testing.T) {
|
||||
schema := newTestSchema(false)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
exprStrs := []string{
|
||||
`VarCharField is not null`,
|
||||
`VarCharField IS NOT NULL`,
|
||||
}
|
||||
for _, exprStr := range exprStrs {
|
||||
assertValidExpr(t, helper, exprStr)
|
||||
}
|
||||
|
||||
unsupported := []string{
|
||||
`not_exist is not null`,
|
||||
}
|
||||
for _, exprStr := range unsupported {
|
||||
assertInvalidExpr(t, helper, exprStr)
|
||||
}
|
||||
}
|
||||
|
||||
func TestExpr_BinaryRange(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -275,7 +321,7 @@ func TestExpr_BinaryRange(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_castValue(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -297,7 +343,7 @@ func TestExpr_castValue(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_BinaryArith(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -333,7 +379,7 @@ func TestExpr_BinaryArith(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Value(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -354,7 +400,7 @@ func TestExpr_Value(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Identifier(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -381,7 +427,7 @@ func TestExpr_Identifier(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Constant(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -453,7 +499,7 @@ func TestExpr_Constant(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Combinations(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -521,7 +567,7 @@ func TestCreateSparseFloatVectorSearchPlan(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestExpr_Invalid(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
helper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -713,7 +759,7 @@ func (l *errorListenerTest) Error() error {
|
|||
}
|
||||
|
||||
func Test_FixErrorListenerNotRemoved(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -728,7 +774,7 @@ func Test_FixErrorListenerNotRemoved(t *testing.T) {
|
|||
}
|
||||
|
||||
func Test_handleExpr(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -739,7 +785,7 @@ func Test_handleExpr(t *testing.T) {
|
|||
}
|
||||
|
||||
func Test_handleExpr_empty(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -750,7 +796,7 @@ func Test_handleExpr_empty(t *testing.T) {
|
|||
|
||||
// test if handleExpr is thread-safe.
|
||||
func Test_handleExpr_17126_26662(t *testing.T) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
assert.NoError(t, err)
|
||||
normal := `VarCharField == "abcd\"defg"`
|
||||
|
@ -1124,6 +1170,8 @@ c'`,
|
|||
}
|
||||
}
|
||||
|
||||
// todo add null test
|
||||
|
||||
func Test_JSONContainsAll(t *testing.T) {
|
||||
schema := newTestSchemaHelper(t)
|
||||
expr := ""
|
||||
|
@ -1373,7 +1421,7 @@ func TestConcurrency(t *testing.T) {
|
|||
}
|
||||
|
||||
func BenchmarkPlanCache(b *testing.B) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
require.NoError(b, err)
|
||||
|
||||
|
@ -1409,7 +1457,7 @@ func randomChineseString(length int) string {
|
|||
}
|
||||
|
||||
func BenchmarkWithString(b *testing.B) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
require.NoError(b, err)
|
||||
|
||||
|
@ -1448,7 +1496,7 @@ func Test_convertHanToASCII(t *testing.T) {
|
|||
}
|
||||
|
||||
func BenchmarkTemplateWithString(b *testing.B) {
|
||||
schema := newTestSchema()
|
||||
schema := newTestSchema(true)
|
||||
schemaHelper, err := typeutil.CreateSchemaHelper(schema)
|
||||
require.NoError(b, err)
|
||||
|
||||
|
|
|
@ -16,6 +16,7 @@ func extractColumnInfo(info *planpb.ColumnInfo) interface{} {
|
|||
js["data_type"] = info.GetDataType().String()
|
||||
js["auto_id"] = info.GetIsAutoID()
|
||||
js["is_pk"] = info.GetIsPrimaryKey()
|
||||
js["nullable"] = info.GetNullable()
|
||||
return js
|
||||
}
|
||||
|
||||
|
@ -64,6 +65,8 @@ func (v *ShowExprVisitor) VisitExpr(expr *planpb.Expr) interface{} {
|
|||
js["expr"] = v.VisitValueExpr(realExpr.ValueExpr)
|
||||
case *planpb.Expr_ColumnExpr:
|
||||
js["expr"] = v.VisitColumnExpr(realExpr.ColumnExpr)
|
||||
case *planpb.Expr_NullExpr:
|
||||
js["expr"] = v.VisitNullExpr(realExpr.NullExpr)
|
||||
default:
|
||||
js["expr"] = ""
|
||||
}
|
||||
|
@ -172,6 +175,14 @@ func (v *ShowExprVisitor) VisitColumnExpr(expr *planpb.ColumnExpr) interface{} {
|
|||
return js
|
||||
}
|
||||
|
||||
func (v *ShowExprVisitor) VisitNullExpr(expr *planpb.NullExpr) interface{} {
|
||||
js := make(map[string]interface{})
|
||||
js["expr_type"] = "null"
|
||||
js["op"] = expr.Op.String()
|
||||
js["column_info"] = extractColumnInfo(expr.GetColumnInfo())
|
||||
return js
|
||||
}
|
||||
|
||||
func NewShowExprVisitor() LogicalExprVisitor {
|
||||
return &ShowExprVisitor{}
|
||||
}
|
||||
|
|
|
@ -85,6 +85,7 @@ message ColumnInfo {
|
|||
bool is_partition_key = 6;
|
||||
schema.DataType element_type = 7;
|
||||
bool is_clustering_key = 8;
|
||||
bool nullable = 9;
|
||||
}
|
||||
|
||||
message ColumnExpr {
|
||||
|
@ -153,6 +154,16 @@ message JSONContainsExpr {
|
|||
string template_variable_name = 5;
|
||||
}
|
||||
|
||||
message NullExpr {
|
||||
ColumnInfo column_info = 1;
|
||||
enum NullOp {
|
||||
Invalid = 0;
|
||||
IsNull = 1;
|
||||
IsNotNull = 2;
|
||||
}
|
||||
NullOp op = 2;
|
||||
}
|
||||
|
||||
message UnaryExpr {
|
||||
enum UnaryOp {
|
||||
Invalid = 0;
|
||||
|
@ -213,6 +224,7 @@ message Expr {
|
|||
AlwaysTrueExpr always_true_expr = 12;
|
||||
JSONContainsExpr json_contains_expr = 13;
|
||||
CallExpr call_expr = 14;
|
||||
NullExpr null_expr = 15;
|
||||
};
|
||||
bool is_template = 20;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue