From 40f512c949f4b35a7a12adf9dd4ac997846024cd Mon Sep 17 00:00:00 2001 From: shen yushi Date: Fri, 21 Jun 2024 23:50:25 +0800 Subject: [PATCH] Bmp idx 3 (#1361) ### What problem does this PR solve? 1. Add knn query by sparse vector's bmp index 2. Add filter knn query for sparse vector. Issue link:#1355 ### Type of change - [x] New Feature (non-breaking change which adds functionality) - [x] Test cases --- .gitignore | 2 + .../local_infinity/sparse/bmp_benchmark.cpp | 151 +- .../sparse/sparse_benchmark_util.h | 14 +- src/executor/operator/physical_optimize.cpp | 18 +- src/executor/operator/physical_optimize.cppm | 14 +- .../physical_match_sparse_scan.cpp | 245 +- .../physical_match_sparse_scan.cppm | 7 + src/executor/operator_state.cppm | 5 +- src/executor/physical_planner.cpp | 4 +- .../table/match_sparse_scan_data.cppm | 34 +- src/parser/parser.cpp | 2768 +++++++++-------- src/parser/parser.y | 20 + src/parser/statement/optimize_statement.h | 3 + src/planner/logical_planner.cpp | 7 +- src/planner/node/logical_optimize.cppm | 10 +- src/scheduler/fragment_context.cpp | 5 +- .../file_worker/bmp_index_file_worker.cpp | 3 +- src/storage/common/block_index.cpp | 8 +- src/storage/common/block_index.cppm | 2 +- src/storage/knn_index/sparse/bmp_alg.cpp | 85 +- src/storage/knn_index/sparse/bmp_alg.cppm | 107 +- .../knn_index/sparse/bmp_alg_serialize.cpp | 4 + src/storage/knn_index/sparse/bmp_posting.cpp | 4 +- src/storage/knn_index/sparse/bmp_util.cpp | 81 + src/storage/knn_index/sparse/bmp_util.cppm | 18 + .../meta/entry/segment_index_entry.cpp | 41 + .../meta/entry/segment_index_entry.cppm | 3 + src/storage/meta/entry/table_index_entry.cpp | 13 + src/storage/meta/entry/table_index_entry.cppm | 4 +- .../knnindex/knn_sparse/test_bmp_index.cpp | 9 +- .../dql/knn/sparse/test_knn_sparse_bmp.slt | 54 + .../knn/sparse/test_knn_sparse_bmp_filter.slt | 53 + .../sparse/test_knn_sparse_bmp_realtime.slt | 55 + 33 files changed, 2274 insertions(+), 1577 deletions(-) create mode 100644 src/storage/knn_index/sparse/bmp_util.cpp create mode 100644 test/sql/dql/knn/sparse/test_knn_sparse_bmp_filter.slt diff --git a/.gitignore b/.gitignore index a05decc1d6..5f7ab545f0 100644 --- a/.gitignore +++ b/.gitignore @@ -129,3 +129,5 @@ callgrind.out.* python/benchmark/datasets/ .venv/ + +dist/ diff --git a/benchmark/local_infinity/sparse/bmp_benchmark.cpp b/benchmark/local_infinity/sparse/bmp_benchmark.cpp index 28c5ef18bc..e27741ac8d 100644 --- a/benchmark/local_infinity/sparse/bmp_benchmark.cpp +++ b/benchmark/local_infinity/sparse/bmp_benchmark.cpp @@ -49,80 +49,121 @@ int main(int argc, char *argv[]) { case ModeType::kImport: { SparseMatrix data_mat = DecodeSparseDataset(opt.data_path_); profiler.Begin(); - BMPAlg index(data_mat.ncol_, opt.block_size_); - for (SparseMatrixIter iter(data_mat); iter.HasNext(); iter.Next()) { - SparseVecRef vec = iter.val(); - u32 doc_id = iter.row_id(); - Vector indices(vec.nnz_); - for (i32 i = 0; i < vec.nnz_; i++) { - indices[i] = static_cast(vec.indices_[i]); + + auto inner = [&](auto &index) { + for (SparseMatrixIter iter(data_mat); iter.HasNext(); iter.Next()) { + SparseVecRef vec = iter.val(); + u32 doc_id = iter.row_id(); + Vector indices(vec.nnz_); + for (i32 i = 0; i < vec.nnz_; i++) { + indices[i] = static_cast(vec.indices_[i]); + } + SparseVecRef vec1(vec.nnz_, indices.data(), vec.data_); + index.AddDoc(vec1, doc_id); + + if (LogInterval != 0 && doc_id % LogInterval == 0) { + std::cout << fmt::format("Imported {} docs\n", doc_id); + } } - SparseVecRef vec1(vec.nnz_, indices.data(), vec.data_); - index.AddDoc(vec1, doc_id); + data_mat.Clear(); + + BMPOptimizeOptions optimize_options{.topk_ = opt.topk_}; + std::cout << "Optimizing index...\n"; + index.Optimize(optimize_options); + std::cout << "Index built\n"; + + profiler.End(); - if (LogInterval != 0 && doc_id % LogInterval == 0) { - std::cout << fmt::format("Imported {} docs\n", doc_id); + std::cout << fmt::format("Import data time: {}\n", profiler.ElapsedToString(1000)); + auto [file_handler, status] = + fs.OpenFile(opt.index_save_path_.string(), FileFlags::WRITE_FLAG | FileFlags::CREATE_FLAG, FileLockType::kNoLock); + if (!status.ok()) { + UnrecoverableError(fmt::format("Failed to open file: {}", opt.index_save_path_.string())); + } + index.Save(*file_handler); + }; + switch (opt.type_) { + case BMPCompressType::kCompressed: { + BMPAlg index(data_mat.ncol_, opt.block_size_); + inner(index); + break; + } + case BMPCompressType::kRaw: { + BMPAlg index(data_mat.ncol_, opt.block_size_); + inner(index); + break; + } + default: { + UnrecoverableError("Unknown compress type"); } } - data_mat.Clear(); - std::cout << "Optimizing index...\n"; - index.Optimize(opt.topk_); - std::cout << "Index built\n"; - profiler.End(); - - std::cout << fmt::format("Import data time: {}\n", profiler.ElapsedToString(1000)); - auto [file_handler, status] = - fs.OpenFile(opt.index_save_path_.string(), FileFlags::WRITE_FLAG | FileFlags::CREATE_FLAG, FileLockType::kNoLock); - if (!status.ok()) { - UnrecoverableError(fmt::format("Failed to open file: {}", opt.index_save_path_.string())); - } - index.Save(*file_handler); break; } case ModeType::kQuery: { i64 query_n = opt.query_n_; i32 thread_n = opt.thread_n_; + BmpSearchOptions search_options{.alpha_ = opt.alpha_, .beta_ = opt.beta_, .use_tail_ = true, .use_lock_ = false}; auto [file_handler, status] = fs.OpenFile(opt.index_save_path_.string(), FileFlags::READ_FLAG, FileLockType::kNoLock); if (!status.ok()) { UnrecoverableError(fmt::format("Failed to open file: {}", opt.index_save_path_.string())); } - auto index = BMPAlg::Load(*file_handler); - auto [top_k, all_query_n, _1, _2] = DecodeGroundtruth(opt.groundtruth_path_, true); - if ((int)top_k != opt.topk_) { - UnrecoverableError(fmt::format("Topk mismatch: {} vs {}", top_k, opt.topk_)); - } - Vector, Vector>> query_result; - { - SparseMatrix query_mat = DecodeSparseDataset(opt.query_path_); - if (all_query_n != query_mat.nrow_) { - UnrecoverableError(fmt::format("Query number mismatch: {} vs {}", query_n, query_mat.nrow_)); + auto inner = [&](auto &index) { + auto [top_k, all_query_n, _1, _2] = DecodeGroundtruth(opt.groundtruth_path_, true); + if ((int)top_k != opt.topk_) { + UnrecoverableError(fmt::format("Topk mismatch: {} vs {}", top_k, opt.topk_)); } - if (query_n > all_query_n) { - UnrecoverableError(fmt::format("Query number: {} is larger than all query number: {}", query_n, all_query_n)); + Vector, Vector>> query_result; + { + SparseMatrix query_mat = DecodeSparseDataset(opt.query_path_); + if (all_query_n != query_mat.nrow_) { + UnrecoverableError(fmt::format("Query number mismatch: {} vs {}", query_n, query_mat.nrow_)); + } + if (query_n > all_query_n) { + UnrecoverableError(fmt::format("Query number: {} is larger than all query number: {}", query_n, all_query_n)); + } + if (query_n == 0) { + query_n = all_query_n; + } + + profiler.Begin(); + query_result = Search(thread_n, + query_mat, + top_k, + query_n, + [&](const SparseVecRef &query, u32 topk) -> Pair, Vector> { + Vector indices(query.nnz_); + for (i32 i = 0; i < query.nnz_; i++) { + indices[i] = static_cast(query.indices_[i]); + } + SparseVecRef query1(query.nnz_, indices.data(), query.data_); + return index.SearchKnn(query1, topk, search_options); + }); + profiler.End(); + std::cout << fmt::format("Search time: {}\n", profiler.ElapsedToString(1000)); } - if (query_n == 0) { - query_n = all_query_n; + { + auto [_1, _2, gt_indices, gt_scores] = DecodeGroundtruth(opt.groundtruth_path_, false); + f32 recall = CheckGroundtruth(gt_indices.get(), gt_scores.get(), query_result, top_k); + std::cout << fmt::format("Recall: {}\n", recall); } + }; - profiler.Begin(); - query_result = - Search(thread_n, query_mat, top_k, query_n, [&](const SparseVecRef &query, u32 topk) -> Pair, Vector> { - Vector indices(query.nnz_); - for (i32 i = 0; i < query.nnz_; i++) { - indices[i] = static_cast(query.indices_[i]); - } - SparseVecRef query1(query.nnz_, indices.data(), query.data_); - return index.SearchKnn(query1, topk, opt.alpha_, opt.beta_); - }); - profiler.End(); - std::cout << fmt::format("Search time: {}\n", profiler.ElapsedToString(1000)); - } - { - auto [_1, _2, gt_indices, gt_scores] = DecodeGroundtruth(opt.groundtruth_path_, false); - f32 recall = CheckGroundtruth(gt_indices.get(), gt_scores.get(), query_result, top_k); - std::cout << fmt::format("Recall: {}\n", recall); + switch (opt.type_) { + case BMPCompressType::kCompressed: { + auto index = BMPAlg::Load(*file_handler); + inner(index); + break; + } + case BMPCompressType::kRaw: { + auto index = BMPAlg::Load(*file_handler); + inner(index); + break; + } + default: { + UnrecoverableError("Unknown compress type"); + } } break; } diff --git a/benchmark/local_infinity/sparse/sparse_benchmark_util.h b/benchmark/local_infinity/sparse/sparse_benchmark_util.h index cdbd4f86c8..deee231265 100644 --- a/benchmark/local_infinity/sparse/sparse_benchmark_util.h +++ b/benchmark/local_infinity/sparse/sparse_benchmark_util.h @@ -25,6 +25,7 @@ import third_party; import infinity_exception; import sparse_util; import compilation_config; +import bmp_util; using namespace infinity; @@ -248,17 +249,26 @@ struct LinScanOption : public BenchmarkOption { struct BMPOption : public BenchmarkOption { public: void ParseInner(CLI::App &app_) override { + Map bmp_compress_type_map = { + {"compress", BMPCompressType::kCompressed}, + {"raw", BMPCompressType::kRaw}, + }; + + app_.add_option("--type", type_, "BMP compress type") + ->required(false) + ->transform(CLI::CheckedTransformer(bmp_compress_type_map, CLI::ignore_case)); app_.add_option("--topk", topk_, "Topk")->required(false)->transform(CLI::Range(1, 1024)); app_.add_option("--block_size", block_size_, "Block size")->required(false)->transform(CLI::Range(1, 1024)); app_.add_option("--alpha", alpha_, "Alpha")->required(false)->transform(CLI::Range(0.0, 100.0)); app_.add_option("--beta", beta_, "Beta")->required(false)->transform(CLI::Range(0.0, 100.0)); } - String IndexName() const override { return fmt::format("bmp_block{}", block_size_); } + String IndexName() const override { return fmt::format("bmp_block{}_type{}", block_size_, static_cast(type_)); } public: + BMPCompressType type_ = BMPCompressType::kCompressed; i32 topk_ = 10; - i8 block_size_ = 8; + u8 block_size_ = 8; f32 alpha_ = 1.0; f32 beta_ = 1.0; }; diff --git a/src/executor/operator/physical_optimize.cpp b/src/executor/operator/physical_optimize.cpp index d7a151be81..b6be07247e 100644 --- a/src/executor/operator/physical_optimize.cpp +++ b/src/executor/operator/physical_optimize.cpp @@ -37,7 +37,12 @@ namespace infinity { void PhysicalOptimize::Init() {} bool PhysicalOptimize::Execute(QueryContext *query_context, OperatorState *operator_state) { - OptimizeIndex(query_context, operator_state); + if (index_name_.empty()) { + OptimizeIndex(query_context, operator_state); + } else { + OptimizeAIndex(query_context, operator_state); + } + operator_state->SetComplete(); return true; } @@ -58,4 +63,15 @@ void PhysicalOptimize::OptimizeIndex(QueryContext *query_context, OperatorState LOG_INFO(fmt::format("OptimizeIndex {}.{} end", db_name_, table_name_)); } +void PhysicalOptimize::OptimizeAIndex(QueryContext *query_context, OperatorState *operator_state) { + auto txn = query_context->GetTxn(); + auto [table_index_entry, status] = txn->GetIndexByName(db_name_, table_name_, index_name_); + if (!status.ok()) { + operator_state->status_ = status; + RecoverableError(status); + return; + } + table_index_entry->OptimizeIndex(txn, opt_params_); +} + } // namespace infinity diff --git a/src/executor/operator/physical_optimize.cppm b/src/executor/operator/physical_optimize.cppm index 1da112e16d..1a42b90b23 100644 --- a/src/executor/operator/physical_optimize.cppm +++ b/src/executor/operator/physical_optimize.cppm @@ -29,14 +29,20 @@ import internal_types; import optimize_statement; import data_type; import logger; +import statement_common; namespace infinity { export class PhysicalOptimize final : public PhysicalOperator { public: - explicit PhysicalOptimize(u64 id, String db_name, String table_name, SharedPtr> load_metas) + explicit PhysicalOptimize(u64 id, + String db_name, + String table_name, + String index_name, + Vector> opt_params, + SharedPtr> load_metas) : PhysicalOperator(PhysicalOperatorType::kOptimize, nullptr, nullptr, id, load_metas), db_name_(std::move(db_name)), - table_name_(std::move(table_name)) {} + table_name_(std::move(table_name)), index_name_(std::move(index_name)), opt_params_(std::move(opt_params)) {} ~PhysicalOptimize() override = default; @@ -58,9 +64,13 @@ public: private: void OptimizeIndex(QueryContext *query_context, OperatorState *operator_state); + void OptimizeAIndex(QueryContext *query_context, OperatorState *operator_state); + private: String db_name_{}; String table_name_{}; + String index_name_{}; + Vector> opt_params_; SharedPtr> output_names_{}; SharedPtr>> output_types_{}; diff --git a/src/executor/operator/physical_scan/physical_match_sparse_scan.cpp b/src/executor/operator/physical_scan/physical_match_sparse_scan.cpp index 19308480c0..edab171d38 100644 --- a/src/executor/operator/physical_scan/physical_match_sparse_scan.cpp +++ b/src/executor/operator/physical_scan/physical_match_sparse_scan.cpp @@ -26,6 +26,9 @@ import physical_operator_type; import table_entry; import match_sparse_expression; import base_table_ref; +import block_index; +import column_expression; +import create_index_info; import data_type; import common_query_filter; import default_values; @@ -50,6 +53,13 @@ import fix_heap; import global_block_id; import bitmask; import txn; +import bmp_index_file_worker; +import segment_index_entry; +import buffer_handle; +import sparse_util; +import bmp_util; +import knn_filter; +import segment_entry; namespace infinity { @@ -88,6 +98,79 @@ SharedPtr>> PhysicalMatchSparseScan::GetOutputTypes() return result_types; } +SizeT PhysicalMatchSparseScan::TaskletCount() { + SizeT ret = base_table_ref_->block_index_->BlockCount(); + if (base_table_ref_->index_index_.get() != nullptr) { + const auto &index_snapshots = base_table_ref_->index_index_->index_snapshots_vec_; + if (index_snapshots.size() != 1) { + UnrecoverableError("Multiple index snapshots are not supported."); + } + ret = index_snapshots[0]->segment_index_entries_.size(); + } + return ret; +} + +SizeT PhysicalMatchSparseScan::GetTaskletCount(QueryContext *query_context) { + Txn *txn = query_context->GetTxn(); + TableEntry *table_entry = base_table_ref_->table_entry_ptr_; + BlockIndex *block_index = base_table_ref_->block_index_.get(); + SizeT ret = block_index->BlockCount(); + + ColumnExpression *column_expr = static_cast(match_sparse_expr_->arguments()[0].get()); + SizeT knn_column_id = column_expr->binding().column_idx; + { + auto map_guard = table_entry->IndexMetaMap(); + for (const auto &[idx_name, table_index_meta] : *map_guard) { + auto [table_index_entry, status] = table_index_meta->GetEntryNolock(txn->TxnID(), txn->BeginTS()); + if (!status.ok()) { + RecoverableError(status); + } + const String column_name = table_index_entry->index_base()->column_name(); + SizeT column_id = table_entry->GetColumnIdByName(column_name); + if (column_id != knn_column_id) { + continue; + } + if (auto index_type = table_index_entry->index_base()->index_type_; index_type != IndexType::kBMP) { + LOG_TRACE(fmt::format("KnnScan: PlanWithIndex(): Skipping non-knn index.")); + continue; + } + if (base_table_ref_->index_index_.get() == nullptr) { + base_table_ref_->index_index_ = MakeShared(); + } + IndexIndex *index_index = base_table_ref_->index_index_.get(); + auto index_snapshot = index_index->Insert(table_index_entry, txn); + ret = index_snapshot->segment_index_entries_.size(); + break; + } + } + + return ret; +} + +Vector>> PhysicalMatchSparseScan::PlanWithIndex(Vector>> &block_groups, + i64 parallel_count) { + Vector>> segment_groups(parallel_count); + for (i64 i = 0; i < parallel_count; ++i) { + segment_groups[i] = MakeShared>(); + } + if ((i64)block_groups.size() != parallel_count) { + UnrecoverableError("block_groups.size() != parallel_count"); + } + IndexIndex *index_index = base_table_ref_->index_index_.get(); + if (index_index != nullptr) { + block_groups.assign(parallel_count, MakeShared>()); + SizeT group_idx = 0; + for (const auto &[idx_name, index_snapshot] : index_index->index_snapshots_) { + for (const auto &[segment_id, segment_index_entry] : index_snapshot->segment_index_entries_) { + segment_groups[group_idx]->push_back(segment_id); + group_idx = (group_idx + 1) % parallel_count; + } + break; + } + } + return segment_groups; +} + bool PhysicalMatchSparseScan::Execute(QueryContext *query_context, OperatorState *operator_state) { auto *match_sparse_scan_state = static_cast(operator_state); MatchSparseScanFunctionData &function_data = match_sparse_scan_state->match_sparse_scan_function_data_; @@ -272,14 +355,31 @@ void PhysicalMatchSparseScan::ExecuteInnerT(DistFunc *dist_func, BufferManager *buffer_mgr = query_context->storage()->buffer_manager(); const Vector &block_ids = *function_data.global_block_ids_; - const BlockIndex *block_index = function_data.block_index_; + const Vector &segment_ids = *function_data.segment_ids_; auto &block_ids_idx = function_data.current_block_ids_idx_; + auto &segment_ids_idx = function_data.current_segment_ids_idx_; + + const ColumnVector &query_vector = *function_data.query_data_->column_vectors[0]; + + auto get_ele = [](const ColumnVector &column_vector, SizeT idx) -> SparseVecRef { + const auto *ele = reinterpret_cast(column_vector.data()) + idx; + const auto &[nnz, chunk_id, chunk_offset] = *ele; + if (nnz == 0) { + return SparseVecRef(0, nullptr, nullptr); + } + const char *sparse_ptr = column_vector.buffer_->fix_heap_mgr_->GetRawPtrFromChunk(chunk_id, chunk_offset); + const auto *indices = reinterpret_cast(sparse_ptr); + const auto *data = reinterpret_cast(sparse_ptr + nnz * sizeof(typename DistFunc::IndexT)); + return SparseVecRef(nnz, indices, data); + }; auto task_id = block_ids_idx; while (task_id < block_ids.size()) { - ++block_ids_idx; + block_ids_idx++; + LOG_DEBUG(fmt::format("MatchSparseScan: block {}", task_id)); const auto [segment_id, block_id] = block_ids[task_id]; + const BlockIndex *block_index = base_table_ref_->block_index_.get(); BlockOffset row_cnt = block_index->GetBlockOffset(segment_id, block_id); const BlockEntry *block_entry = block_index->GetBlockEntry(segment_id, block_id); LOG_DEBUG(fmt::format("MatchSparseScan: segment_id: {}, block_id: {}", segment_id, block_id)); @@ -292,35 +392,17 @@ void PhysicalMatchSparseScan::ExecuteInnerT(DistFunc *dist_func, auto *block_column_entry = block_entry->GetColumnBlockEntry(search_column_id_); ColumnVector column_vector = block_column_entry->GetColumnVector(buffer_mgr); - const ColumnVector &query_vector = *function_data.query_data_->column_vectors[0]; - const auto *query_data_begin = reinterpret_cast(query_vector.data()); for (SizeT query_id = 0; query_id < query_n; ++query_id) { - const auto *query_data = query_data_begin + query_id; - const auto &[query_nnz, query_chunk_id, query_chunk_offset] = *query_data; - const char *query_sparse_ptr = nullptr; - if (query_nnz) { - query_sparse_ptr = query_vector.buffer_->fix_heap_mgr_->GetRawPtrFromChunk(query_chunk_id, query_chunk_offset); - } - - const auto *data_begin = reinterpret_cast(column_vector.data()); - FixHeapManager *heap_mgr = column_vector.buffer_->fix_heap_mgr_.get(); + auto query_sparse = get_ele(query_vector, query_id); for (BlockOffset i = 0; i < row_cnt; ++i) { if (!bitmask.IsTrue(i)) { continue; } - const auto *data = data_begin + i; - const auto &[nnz, chunk_id, chunk_offset] = *data; + auto sparse = get_ele(column_vector, i); - ResultType d = 0; - const char *sparse_ptr = nullptr; - if (nnz) { - sparse_ptr = heap_mgr->GetRawPtrFromChunk(chunk_id, chunk_offset); - } - if (query_sparse_ptr != nullptr && sparse_ptr != nullptr) { - d = dist_func->Calculate(query_sparse_ptr, query_nnz, sparse_ptr, nnz); - } + ResultType d = dist_func->Calculate(query_sparse, sparse); RowID row_id(segment_id, block_id * DEFAULT_BLOCK_CAPACITY + i); merge_heap->Search(query_id, &d, &row_id, 1); @@ -328,7 +410,122 @@ void PhysicalMatchSparseScan::ExecuteInnerT(DistFunc *dist_func, } break; } - if (block_ids_idx >= block_ids.size()) { + task_id = segment_ids_idx; + while (task_id < segment_ids.size()) { + segment_ids_idx++; + LOG_DEBUG(fmt::format("MatchSparseScan: segment {}", task_id)); + SegmentID segment_id = segment_ids[task_id]; + + const IndexIndex *index_index = base_table_ref_->index_index_.get(); + const BlockIndex *block_index = base_table_ref_->block_index_.get(); + IndexSnapshot *index_snapshot = index_index->index_snapshots_vec_[0]; + auto iter = index_snapshot->segment_index_entries_.find(segment_id); + if (iter == index_snapshot->segment_index_entries_.end()) { + UnrecoverableError(fmt::format("Cannot find segment index with id: {}", segment_id)); + } + SegmentIndexEntry *segment_index_entry = iter->second; + const auto &index_base = segment_index_entry->table_index_entry()->index_base(); + if (index_base->index_type_ != IndexType::kBMP) { + UnrecoverableError(fmt::format("IndexType: {} is not supported.", (i8)index_base->index_type_)); + } + + auto it = common_query_filter_->filter_result_.find(segment_id); + if (it == common_query_filter_->filter_result_.end()) { + break; + } + SizeT segment_row_count = 0; + SegmentEntry *segment_entry = nullptr; + { + auto segment_it = block_index->segment_block_index_.find(segment_id); + if (segment_it == block_index->segment_block_index_.end()) { + UnrecoverableError(fmt::format("Cannot find segment with id: {}", segment_id)); + } + segment_entry = segment_it->second.segment_entry_; + segment_row_count = segment_it->second.segment_offset_; + } + Bitmask bitmask; + const std::variant, Bitmask> &filter_result = it->second; + if (std::holds_alternative>(filter_result)) { + const Vector &filter_result_vector = std::get>(filter_result); + bitmask.Initialize(std::ceil(segment_row_count)); + bitmask.SetAllFalse(); + for (u32 row_id : filter_result_vector) { + bitmask.SetTrue(row_id); + } + } else { + bitmask.ShallowCopy(std::get(filter_result)); + } + bool use_bitmask = !bitmask.IsAllTrue(); + + auto bmp_search = [&](AbstractBMP index, SizeT query_id, bool with_lock, const auto &filter) { + auto query = get_ele(query_vector, query_id); + std::visit( + [&](auto &&index) { + using T = std::decay_t; + if constexpr (std::is_same_v) { + UnrecoverableError("Invalid index type."); + } else { + using IndexT = std::decay_t; + if constexpr (std::is_same_v && + std::is_same_v) { + if (use_bitmask) { + + } else { + } + + BmpSearchOptions options = BMPUtil::ParseBmpSearchOptions(match_sparse_expr_->opt_params_); + options.use_lock_ = with_lock; + auto [doc_ids, scores] = index->SearchKnn(query, topn, options, filter); + for (SizeT i = 0; i < topn; ++i) { + RowID row_id(segment_id, doc_ids[i]); + ResultType d = scores[i]; + merge_heap->Search(query_id, &d, &row_id, 1); + } + } else { + UnrecoverableError("Invalid index type."); + } + } + }, + index); + }; + + auto bmp_scan = [&](const auto &filter) { + const auto [chunk_index_entries, memory_index_entry] = segment_index_entry->GetBMPIndexSnapshot(); + for (SizeT query_id = 0; query_id < query_n; ++query_id) { + for (auto chunk_index_entry : chunk_index_entries) { + BufferHandle buffer_handle = chunk_index_entry->GetIndex(); + auto abstract_bmp = static_cast(buffer_handle.GetFileWorkerMut())->GetAbstractIndex(); + bmp_search(abstract_bmp, query_id, false, filter); + } + if (memory_index_entry.get() != nullptr) { + BufferHandle buffer_handle = memory_index_entry->GetIndex(); + auto abstract_bmp = static_cast(buffer_handle.GetFileWorkerMut())->GetAbstractIndex(); + bmp_search(abstract_bmp, query_id, true, filter); + } + } + }; + + if (use_bitmask) { + if (segment_entry->CheckAnyDelete(begin_ts)) { + DeleteWithBitmaskFilter filter(bitmask, segment_entry, begin_ts); + bmp_scan(filter); + } else { + BitmaskFilter filter(bitmask); + bmp_scan(filter); + } + } else { + SegmentOffset max_segment_offset = block_index->GetSegmentOffset(segment_id); + if (segment_entry->CheckAnyDelete(begin_ts)) { + DeleteFilter filter(segment_entry, begin_ts, max_segment_offset); + bmp_scan(filter); + } else { + bmp_scan(nullptr); + } + } + + break; + } + if (block_ids_idx == block_ids.size() && segment_ids_idx == segment_ids.size()) { LOG_DEBUG(fmt::format("MatchSparseScan: {} task finished", block_ids_idx)); merge_heap->End(); i64 result_n = std::min(topn, (SizeT)merge_heap->total_count()); diff --git a/src/executor/operator/physical_scan/physical_match_sparse_scan.cppm b/src/executor/operator/physical_scan/physical_match_sparse_scan.cppm index d3eb4f8195..187a7a37a0 100644 --- a/src/executor/operator/physical_scan/physical_match_sparse_scan.cppm +++ b/src/executor/operator/physical_scan/physical_match_sparse_scan.cppm @@ -29,6 +29,7 @@ import physical_scan_base; import sparse_info; import match_sparse_expr; import physical_filter_scan_base; +import global_block_id; namespace infinity { struct LoadMeta; @@ -52,6 +53,12 @@ public: SharedPtr>> GetOutputTypes() const override; + SizeT TaskletCount() override; + + SizeT GetTaskletCount(QueryContext *query_context); + + Vector>> PlanWithIndex(Vector>> &block_groups, i64 parallel_count); + private: template void ExecuteInner(QueryContext *query_context, diff --git a/src/executor/operator_state.cppm b/src/executor/operator_state.cppm index abbee4efcd..ccf7271f0d 100644 --- a/src/executor/operator_state.cppm +++ b/src/executor/operator_state.cppm @@ -506,10 +506,11 @@ export struct MatchTensorScanSourceState : public SourceState { }; export struct MatchSparseScanSourceState : public SourceState { - explicit MatchSparseScanSourceState(SharedPtr> global_ids) - : SourceState(SourceStateType::kMatchSparseScan), global_ids_(std::move(global_ids)) {} + explicit MatchSparseScanSourceState(SharedPtr> global_ids, SharedPtr> segment_ids) + : SourceState(SourceStateType::kMatchSparseScan), global_ids_(std::move(global_ids)), segment_ids_(std::move(segment_ids)) {} SharedPtr> global_ids_; + SharedPtr> segment_ids_; }; export struct IndexScanSourceState : public SourceState { diff --git a/src/executor/physical_planner.cpp b/src/executor/physical_planner.cpp index dc8ec97774..b7607f2efb 100644 --- a/src/executor/physical_planner.cpp +++ b/src/executor/physical_planner.cpp @@ -920,6 +920,8 @@ UniquePtr PhysicalPlanner::BuildOptimize(const SharedPtr(logical_optimize->node_id(), logical_optimize->schema_name(), logical_optimize->object_name(), + logical_optimize->index_name_, + std::move(logical_optimize->opt_params_), logical_operator->load_metas()); } @@ -973,7 +975,7 @@ UniquePtr PhysicalPlanner::BuildMatchSparseScan(const SharedPt std::static_pointer_cast(logical_match_sparse->query_expression_), logical_match_sparse->common_query_filter_, logical_operator->load_metas()); - if (match_sparse_scan_op->TaskletCount() == 1) { + if (match_sparse_scan_op->GetTaskletCount(query_context_ptr_) == 1) { return match_sparse_scan_op; } auto merge_match_sparse_op = diff --git a/src/function/table/match_sparse_scan_data.cppm b/src/function/table/match_sparse_scan_data.cppm index 3e2baa278b..3b77094b4d 100644 --- a/src/function/table/match_sparse_scan_data.cppm +++ b/src/function/table/match_sparse_scan_data.cppm @@ -28,6 +28,7 @@ import infinity_exception; import third_party; import knn_result_handler; import sparse_vector_distance; +import sparse_util; namespace infinity { @@ -36,6 +37,9 @@ export class SparseDistanceBase {}; export template class SparseDistance : public SparseDistanceBase { public: + using IndexT = IndexType; + using DataT = DataType; + SparseDistance(SparseMetricType metric_type) { switch (metric_type) { case SparseMetricType::kInnerProduct: { @@ -48,15 +52,8 @@ public: } } - ResultType Calculate(const char *raw1, SizeT nnz1, const char *raw2, SizeT nnz2) { - if (raw1 == nullptr || raw2 == nullptr) { - return 0; - } - const IndexType *index1 = reinterpret_cast(raw1); - const DataType *data1 = reinterpret_cast(raw1 + nnz1 * sizeof(IndexType)); - const IndexType *index2 = reinterpret_cast(raw2); - const DataType *data2 = reinterpret_cast(raw2 + nnz2 * sizeof(IndexType)); - return Calculate(data1, index1, nnz1, data2, index2, nnz2); + ResultType Calculate(const SparseVecRef &vec1, const SparseVecRef &vec2) { + return Calculate(vec1.data_, vec1.indices_, vec1.nnz_, vec2.data_, vec2.indices_, vec2.nnz_); } ResultType Calculate(const DataType *data, const IndexType *index, SizeT nnz, const DataType *data2, const IndexType *index2, SizeT nnz2) { @@ -73,6 +70,9 @@ public: export template class SparseBitDistance : public SparseDistanceBase { public: + using IndexT = IndexType; + using DataT = bool; // not used + SparseBitDistance(SparseMetricType metric_type) { switch (metric_type) { case SparseMetricType::kInnerProduct: { @@ -85,13 +85,8 @@ public: } } - ResultType Calculate(const char *raw1, SizeT nnz1, const char *raw2, SizeT nnz2) { - if (raw1 == nullptr || raw2 == nullptr) { - return 0; - } - const IndexType *index1 = reinterpret_cast(raw1); - const IndexType *index2 = reinterpret_cast(raw2); - return Calculate(index1, nnz1, index2, nnz2); + ResultType Calculate(const SparseVecRef &vec1, const SparseVecRef &vec2) { + return Calculate(vec1.indices_, vec1.nnz_, vec2.indices_, vec2.nnz_); } ResultType Calculate(const IndexType *index1, SizeT nnz1, const IndexType *index2, SizeT nnz2) { return dist_func_(index1, nnz1, index2, nnz2); } @@ -106,17 +101,18 @@ export class MatchSparseScanFunctionData : public TableFunctionData { public: MatchSparseScanFunctionData() = default; - MatchSparseScanFunctionData(const BlockIndex *block_index, const SharedPtr> &global_block_ids) - : block_index_(block_index), global_block_ids_(global_block_ids), query_data_(DataBlock::Make()) {} + MatchSparseScanFunctionData(const SharedPtr> &global_block_ids, const SharedPtr> &segment_ids) + : global_block_ids_(global_block_ids), segment_ids_(segment_ids), query_data_(DataBlock::Make()) {} public: - const BlockIndex *block_index_{}; SharedPtr> global_block_ids_; + SharedPtr> segment_ids_; bool evaluated_ = false; SharedPtr query_data_{}; u32 current_block_ids_idx_ = 0; + u32 current_segment_ids_idx_ = 0; UniquePtr merge_knn_base_{}; UniquePtr sparse_distance_{}; }; diff --git a/src/parser/parser.cpp b/src/parser/parser.cpp index 16a19c47bc..1860d7fde5 100644 --- a/src/parser/parser.cpp +++ b/src/parser/parser.cpp @@ -753,18 +753,18 @@ union yyalloc #endif /* !YYCOPY_NEEDED */ /* YYFINAL -- State number of the termination state. */ -#define YYFINAL 89 +#define YYFINAL 90 /* YYLAST -- Last index in YYTABLE. */ -#define YYLAST 1145 +#define YYLAST 1102 /* YYNTOKENS -- Number of terminals. */ #define YYNTOKENS 192 /* YYNNTS -- Number of nonterminals. */ #define YYNNTS 113 /* YYNRULES -- Number of rules. */ -#define YYNRULES 425 +#define YYNRULES 426 /* YYNSTATES -- Number of states. */ -#define YYNSTATES 895 +#define YYNSTATES 899 /* YYMAXUTOK -- Last valid token kind. */ #define YYMAXUTOK 430 @@ -855,25 +855,25 @@ static const yytype_int16 yyrline[] = 1618, 1622, 1626, 1630, 1637, 1641, 1645, 1649, 1654, 1658, 1663, 1667, 1671, 1677, 1683, 1689, 1700, 1711, 1722, 1734, 1746, 1759, 1773, 1784, 1798, 1814, 1835, 1839, 1843, 1851, - 1865, 1871, 1876, 1882, 1888, 1896, 1902, 1908, 1914, 1920, - 1928, 1934, 1940, 1946, 1952, 1960, 1966, 1973, 1990, 1994, - 1999, 2003, 2030, 2036, 2040, 2041, 2042, 2043, 2044, 2046, - 2049, 2055, 2058, 2059, 2060, 2061, 2062, 2063, 2064, 2065, - 2066, 2067, 2069, 2072, 2078, 2097, 2139, 2185, 2203, 2221, - 2229, 2240, 2246, 2255, 2261, 2273, 2276, 2279, 2282, 2285, - 2288, 2292, 2296, 2301, 2309, 2317, 2326, 2333, 2340, 2347, - 2354, 2361, 2369, 2377, 2385, 2393, 2401, 2409, 2417, 2425, - 2433, 2441, 2449, 2457, 2487, 2495, 2504, 2512, 2521, 2529, - 2535, 2542, 2548, 2555, 2560, 2567, 2574, 2582, 2606, 2612, - 2618, 2625, 2633, 2640, 2647, 2652, 2662, 2667, 2672, 2677, - 2682, 2687, 2692, 2697, 2702, 2707, 2710, 2713, 2717, 2720, - 2723, 2726, 2730, 2733, 2736, 2740, 2744, 2749, 2754, 2757, - 2761, 2765, 2772, 2779, 2783, 2790, 2797, 2801, 2805, 2809, - 2812, 2816, 2820, 2825, 2830, 2834, 2839, 2844, 2850, 2856, - 2862, 2868, 2874, 2880, 2886, 2892, 2898, 2904, 2910, 2921, - 2925, 2930, 2958, 2968, 2974, 2978, 2979, 2981, 2982, 2984, - 2985, 2997, 3005, 3009, 3012, 3016, 3019, 3023, 3027, 3032, - 3038, 3048, 3055, 3066, 3118, 3167 + 1862, 1885, 1891, 1896, 1902, 1908, 1916, 1922, 1928, 1934, + 1940, 1948, 1954, 1960, 1966, 1972, 1980, 1986, 1993, 2010, + 2014, 2019, 2023, 2050, 2056, 2060, 2061, 2062, 2063, 2064, + 2066, 2069, 2075, 2078, 2079, 2080, 2081, 2082, 2083, 2084, + 2085, 2086, 2087, 2089, 2092, 2098, 2117, 2159, 2205, 2223, + 2241, 2249, 2260, 2266, 2275, 2281, 2293, 2296, 2299, 2302, + 2305, 2308, 2312, 2316, 2321, 2329, 2337, 2346, 2353, 2360, + 2367, 2374, 2381, 2389, 2397, 2405, 2413, 2421, 2429, 2437, + 2445, 2453, 2461, 2469, 2477, 2507, 2515, 2524, 2532, 2541, + 2549, 2555, 2562, 2568, 2575, 2580, 2587, 2594, 2602, 2626, + 2632, 2638, 2645, 2653, 2660, 2667, 2672, 2682, 2687, 2692, + 2697, 2702, 2707, 2712, 2717, 2722, 2727, 2730, 2733, 2737, + 2740, 2743, 2746, 2750, 2753, 2756, 2760, 2764, 2769, 2774, + 2777, 2781, 2785, 2792, 2799, 2803, 2810, 2817, 2821, 2825, + 2829, 2832, 2836, 2840, 2845, 2850, 2854, 2859, 2864, 2870, + 2876, 2882, 2888, 2894, 2900, 2906, 2912, 2918, 2924, 2930, + 2941, 2945, 2950, 2978, 2988, 2994, 2998, 2999, 3001, 3002, + 3004, 3005, 3017, 3025, 3029, 3032, 3036, 3039, 3043, 3047, + 3052, 3058, 3068, 3075, 3086, 3138, 3187 }; #endif @@ -959,12 +959,12 @@ yysymbol_name (yysymbol_kind_t yysymbol) } #endif -#define YYPACT_NINF (-642) +#define YYPACT_NINF (-499) #define yypact_value_is_default(Yyn) \ ((Yyn) == YYPACT_NINF) -#define YYTABLE_NINF (-413) +#define YYTABLE_NINF (-414) #define yytable_value_is_error(Yyn) \ ((Yyn) == YYTABLE_NINF) @@ -973,96 +973,96 @@ yysymbol_name (yysymbol_kind_t yysymbol) STATE-NUM. */ static const yytype_int16 yypact[] = { - 64, 317, 26, 344, 85, -11, 85, 192, 507, 597, - 65, 153, 100, 85, 128, -26, -48, 160, 1, -642, - -642, -642, -642, -642, -642, -642, -642, 358, -642, -642, - 225, -642, -642, -642, -642, -642, 112, 112, 112, 112, - 37, 85, 167, 167, 167, 167, 167, 61, 237, 85, - 42, 260, 280, 293, -642, -642, -642, -642, -642, -642, - -642, 246, 334, 85, -642, -642, -642, -642, -642, 273, - 51, 124, -642, 340, -642, 203, -642, 85, -642, -642, - -642, -642, -642, 307, 187, -642, 391, 223, 224, -642, - 30, -642, 396, -642, -642, 3, 370, -642, 374, 387, - 470, 85, 85, 85, 474, 425, 297, 411, 500, 85, - 85, 85, 506, 512, 518, 488, 535, 535, 465, 63, - 78, 93, -642, -642, -642, -642, -642, -642, -642, 358, - -642, -642, -642, -642, -642, -642, 352, -642, -642, 545, - -642, 562, -642, 560, -642, 402, 128, 535, -642, -642, - -642, -642, 3, -642, -642, -642, 465, 527, 531, 514, - -642, -10, -642, 297, -642, 85, 620, 29, -642, -642, - -642, -642, -642, 564, -642, 458, -42, -642, 465, -642, - -642, 568, 579, 481, -642, -642, 606, 501, 484, 486, - 316, 636, 661, 672, 674, -642, -642, 678, 503, 295, - 505, 515, 604, 604, -642, 12, 350, -105, -642, -3, - 516, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, -642, 517, -642, -642, -642, -120, -642, - -642, -5, -642, 50, -642, -642, -642, 97, -642, 104, - -642, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, -642, -642, -642, 697, 696, -642, -642, - -642, -642, -642, -642, 225, -642, -642, 528, 529, -40, - 465, 465, 632, -642, -48, 31, 647, 533, -642, -93, - 534, -642, 85, 465, 518, -642, 178, 536, 544, 209, - -642, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, 604, 550, 381, 641, 465, 465, 56, 346, - -642, -642, -642, -642, 606, -642, 746, 565, 572, 573, - 574, 760, 761, 241, 241, -642, 575, -642, -642, -642, - -642, 580, 174, 689, 465, 768, 465, 465, -16, 586, - 27, 604, 604, 604, 604, 604, 604, 604, 604, 604, - 604, 604, 604, 604, 604, 21, -642, 589, -642, 771, - -642, 773, -642, 774, -642, 776, 736, 129, 600, -642, - -642, 7, 622, 599, -642, 34, 178, 465, -642, 358, - 753, 675, 610, 11, -642, -642, -642, -48, 620, 612, - -642, 798, 465, 617, -642, 178, -642, 471, 471, 465, - -642, 219, 641, 650, 621, 73, 94, 353, -642, 465, - 465, 731, 465, 806, 28, 465, 254, 276, 423, -642, - -642, 535, -642, -642, -642, 663, 627, 604, 350, 688, - -642, 659, 659, 121, 121, 648, 659, 659, 121, 121, - 241, 241, -642, -642, -642, -642, -642, -642, 624, -642, - 625, -642, -642, -642, 811, 812, -642, -642, -48, 629, - 666, -642, 47, -642, 302, 488, 465, -642, -642, -642, - 178, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, 634, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, 635, 637, 646, 679, 681, 206, 682, - 620, 794, 31, 358, 288, 620, -642, 329, 685, 829, - 831, -642, 348, -642, 393, 394, -642, 677, -642, 753, - 465, -642, 465, -33, 116, 604, 71, 686, -642, -39, - 80, -642, 868, -642, 873, -642, -642, 799, 350, 659, - 691, 399, -642, 604, 874, 876, 830, 832, 38, 7, - 825, -642, -642, -642, -642, -642, -642, 826, -642, 883, - -642, -642, -642, -642, -642, -642, -642, -642, 698, 833, - -642, 884, 366, 419, 480, 520, 619, 763, 705, 769, - -642, -642, 152, -642, 764, 620, 410, 708, -642, -642, - 737, 412, -642, 465, -642, -642, -642, 471, -642, -642, - -642, 709, 178, 58, -642, 465, 189, 713, 894, 589, - 715, 714, 716, 718, 719, 430, -642, -642, 381, 896, - 899, 47, 666, 7, 7, 721, 302, 845, 852, 434, - 720, 722, 723, 724, 725, 726, 727, 728, 729, 730, - 732, 733, 734, 735, 738, 739, 740, 741, 742, 743, - 744, 745, 747, 748, 749, 750, 751, 752, 754, 755, - 756, 757, 758, 759, 762, 765, 766, 767, 770, 772, - -642, 906, -642, 10, -642, -642, -642, 435, -642, 906, - 908, 778, 446, -642, -642, -642, 178, -642, 493, 775, - 460, 777, 15, 779, -642, -642, -642, -642, -642, 471, - -642, -642, -642, -642, -642, -642, 857, 620, -642, 465, - 465, -642, -642, 920, 921, 937, 944, 945, 947, 948, - 952, 953, 955, 957, 958, 960, 962, 964, 965, 966, - 967, 968, 969, 970, 971, 972, 973, 974, 975, 976, - 977, 978, 979, 980, 981, 982, 983, 984, 985, 986, - 987, 988, 989, 990, 821, 466, -642, -642, -642, 467, - 918, 996, -642, -642, 997, -642, 998, 999, 1000, 468, - 465, 479, 810, 178, 818, 819, 820, 822, 823, 824, - 827, 828, 834, 835, 836, 837, 838, 839, 840, 841, - 842, 843, 844, 846, 847, 848, 849, 850, 851, 853, - 854, 855, 856, 858, 859, 860, 861, 862, 863, 864, - 865, 866, 867, 869, 870, 312, -642, 906, -642, -642, - 918, 871, 872, 875, 487, -642, 178, -642, -642, -642, - -642, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, -642, -642, -642, -642, -642, -642, -642, - -642, -642, -642, -642, 1004, -642, 1008, 918, 1007, 496, - 877, -642, 879, 918, 1010, 1013, 881, 918, -642, 882, - -642, -642, -642, 918, -642 + 670, 260, 50, 287, 38, 80, 38, -68, 907, 575, + 60, 281, 95, 147, 176, 27, -49, 195, 18, -499, + -499, -499, -499, -499, -499, -499, -499, 288, -499, -499, + 203, -499, -499, -499, -499, -499, 143, 143, 143, 143, + -2, 38, 151, 151, 151, 151, 151, 48, 246, 38, + 3, 244, 268, 306, -499, -499, -499, -499, -499, -499, + -499, 693, 317, 38, -499, -499, -499, -499, -499, 257, + 88, 127, -499, 334, -499, 236, -499, 38, -499, -499, + -499, -499, -37, -499, 243, 172, -499, 402, 229, 241, + -499, 44, -499, 421, -499, -499, 5, 380, -499, 379, + 376, 455, 38, 38, 38, 456, 410, 291, 407, 475, + 38, 38, 38, 481, 497, 499, 438, 508, 508, 451, + 58, 70, 81, -499, -499, -499, -499, -499, -499, -499, + 288, -499, -499, -499, -499, -499, -499, 207, -499, -499, + 515, -499, 529, -499, 530, -499, 38, 355, 176, 508, + -499, -499, -499, -499, 5, -499, -499, -499, 451, 487, + 473, 469, -499, -42, -499, 291, -499, 38, 547, 55, + -499, -499, -499, -499, -499, 488, -499, 375, -33, -499, + 451, -499, -499, 477, 478, 373, -499, -499, 734, 485, + 377, 378, 301, 558, 561, 562, 564, -499, -499, 566, + 384, 155, 387, 388, 543, 543, -499, 20, 335, -122, + -499, -18, 615, -499, -499, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, -499, 389, -499, -499, -499, + -81, -499, -499, 43, -499, 53, -499, -499, -499, 54, + -499, 72, -499, -499, -499, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, -499, -499, -499, 572, 570, + -499, -499, -499, -499, -499, -499, 498, 203, -499, -499, + 392, 396, -41, 451, 451, 513, -499, -49, 30, 528, + 400, -499, 78, 411, -499, 38, 451, 499, -499, 177, + 414, 417, 194, -499, -499, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, 543, 420, 624, 531, 451, + 451, 96, 276, -499, -499, -499, -499, 734, -499, 604, + 423, 424, 426, 427, 613, 618, 341, 341, -499, 433, + -499, -499, -499, -499, 439, -54, 549, 451, 630, 451, + 451, -34, 448, -1, 543, 543, 543, 543, 543, 543, + 543, 543, 543, 543, 543, 543, 543, 543, 11, -499, + 454, -499, 633, -499, 634, -499, 635, -499, 638, 596, + 302, 458, -499, 462, -499, -499, 8, 482, 460, -499, + 28, 177, 451, -499, 288, 790, 532, 466, 124, -499, + -499, -499, -49, 547, 467, -499, 651, 451, 468, -499, + 177, -499, 457, 457, 451, -499, 132, 531, 509, 476, + 31, -22, 297, -499, 451, 451, 582, 451, 658, 29, + 451, 157, 210, 337, -499, -499, 508, -499, -499, -499, + 518, 484, 543, 335, 542, -499, 681, 681, 105, 105, + 637, 681, 681, 105, 105, 341, 341, -499, -499, -499, + -499, -499, -499, 496, -499, 500, -499, -499, -499, 669, + 672, -499, 673, -499, -49, 511, 533, -499, 21, -499, + 224, 438, 451, -499, -499, -499, 177, -499, -499, -499, + -499, -499, -499, -499, -499, -499, -499, -499, 502, -499, + -499, -499, -499, -499, -499, -499, -499, -499, -499, 503, + 512, 522, 524, 525, 141, 535, 547, 686, 30, 288, + 222, 547, -499, 227, 539, 722, 725, -499, 233, -499, + 235, 237, -499, 546, -499, 790, 451, -499, 451, -47, + 73, 543, -70, 548, -499, 91, -38, -499, 732, -499, + 733, -499, -499, 663, 335, 681, 557, 239, -499, 543, + 739, 741, 696, 702, 577, 254, -499, 26, 8, 695, + -499, -499, -499, -499, -499, -499, 697, -499, 752, -499, + -499, -499, -499, -499, -499, -499, -499, 567, 704, -499, + 759, 261, 747, 822, 831, 840, 640, 648, -499, -499, + 101, -499, 646, 547, 259, 589, -499, -499, 619, 263, + -499, 451, -499, -499, -499, 457, -499, -499, -499, 591, + 177, 34, -499, 451, 316, 595, 776, 454, 601, 600, + 603, 602, 612, 280, -499, -499, 624, 803, 804, 102, + -499, 673, 21, 533, 8, 8, 625, 224, 757, 765, + 282, 632, 639, 641, 644, 645, 647, 653, 654, 657, + 659, 660, 661, 662, 664, 688, 711, 712, 714, 715, + 718, 719, 728, 729, 737, 738, 746, 751, 758, 761, + 762, 763, 764, 766, 767, 769, 770, 771, 772, 773, + 774, -499, -499, 12, -499, -499, -499, 289, -499, 673, + 820, 666, 314, -499, -499, -499, 177, -499, 352, 775, + 321, 777, 24, 778, -499, -499, -499, -499, -499, 457, + -499, -499, -499, -499, -499, -499, -499, -499, -499, -499, + 784, 547, -499, 451, 451, -499, -499, 824, 830, 847, + 931, 949, 960, 963, 964, 965, 966, 967, 968, 969, + 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, + 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, + 990, 991, 992, 993, 994, 995, 996, 997, -499, -499, + 323, 498, 1001, -499, -499, 1002, -499, 1003, 1004, 1005, + 330, 451, 338, 815, 177, 823, 825, 826, 827, 828, + 829, 832, 833, 834, 835, 836, 837, 838, 839, 841, + 842, 843, 844, 845, 846, 848, 849, 850, 851, 852, + 853, 854, 855, 856, 857, 858, 859, 860, 861, 862, + 863, 864, 865, 866, 867, 868, -499, -499, 498, 821, + 869, 870, 343, -499, 177, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, -499, -499, -499, -499, -499, + -499, -499, -499, -499, -499, -499, -499, -499, 1013, -499, + 1014, 498, 1021, 344, 871, -499, 872, 498, 1028, 1053, + 875, 498, -499, 876, -499, -499, -499, 498, -499 }; /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM. @@ -1071,129 +1071,129 @@ static const yytype_int16 yypact[] = static const yytype_int16 yydefact[] = { 196, 0, 0, 0, 0, 0, 0, 0, 131, 0, - 0, 0, 0, 0, 0, 0, 196, 0, 410, 3, + 0, 0, 0, 0, 0, 0, 196, 0, 411, 3, 5, 10, 12, 13, 11, 6, 7, 9, 145, 144, - 0, 8, 14, 15, 16, 17, 408, 408, 408, 408, - 408, 0, 406, 406, 406, 406, 406, 189, 0, 0, + 0, 8, 14, 15, 16, 17, 409, 409, 409, 409, + 409, 0, 407, 407, 407, 407, 407, 189, 0, 0, 0, 0, 0, 0, 125, 129, 126, 127, 128, 130, 124, 196, 0, 0, 210, 211, 209, 215, 218, 0, 0, 0, 212, 0, 214, 0, 216, 0, 236, 237, - 238, 240, 239, 0, 195, 197, 0, 0, 0, 1, - 196, 2, 179, 181, 182, 0, 168, 150, 156, 0, - 0, 0, 0, 0, 0, 0, 122, 0, 0, 0, - 0, 0, 0, 0, 0, 174, 0, 0, 0, 0, - 0, 0, 123, 18, 23, 25, 24, 19, 20, 22, - 21, 26, 27, 28, 29, 224, 225, 219, 220, 0, - 221, 0, 213, 0, 257, 0, 0, 0, 149, 148, - 4, 180, 0, 146, 147, 167, 0, 0, 164, 0, - 30, 0, 31, 122, 411, 0, 0, 196, 405, 136, - 138, 137, 139, 0, 190, 0, 174, 133, 0, 118, - 404, 0, 0, 342, 346, 349, 350, 0, 0, 0, - 0, 0, 0, 0, 0, 347, 348, 0, 0, 0, - 0, 0, 0, 0, 344, 0, 196, 0, 258, 263, - 264, 278, 276, 279, 277, 280, 281, 273, 268, 267, - 266, 274, 275, 265, 272, 271, 357, 359, 0, 360, - 368, 0, 369, 0, 361, 358, 379, 0, 380, 0, - 356, 244, 246, 245, 242, 243, 249, 251, 250, 247, - 248, 254, 256, 255, 252, 253, 0, 0, 227, 226, - 232, 222, 223, 217, 0, 198, 241, 0, 0, 170, - 0, 0, 166, 407, 196, 0, 0, 0, 116, 0, - 0, 120, 0, 0, 0, 132, 173, 0, 0, 0, - 388, 387, 390, 389, 392, 391, 394, 393, 396, 395, - 398, 397, 0, 0, 308, 196, 0, 0, 0, 0, - 351, 352, 353, 354, 0, 355, 0, 0, 0, 0, - 0, 0, 0, 310, 309, 385, 382, 376, 366, 371, - 374, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 238, 241, 189, 239, 0, 195, 197, 0, 0, 0, + 1, 196, 2, 179, 181, 182, 0, 168, 150, 156, + 0, 0, 0, 0, 0, 0, 0, 122, 0, 0, + 0, 0, 0, 0, 0, 0, 174, 0, 0, 0, + 0, 0, 0, 123, 18, 23, 25, 24, 19, 20, + 22, 21, 26, 27, 28, 29, 224, 225, 219, 220, + 0, 221, 0, 213, 0, 258, 0, 0, 0, 0, + 149, 148, 4, 180, 0, 146, 147, 167, 0, 0, + 164, 0, 30, 0, 31, 122, 412, 0, 0, 196, + 406, 136, 138, 137, 139, 0, 190, 0, 174, 133, + 0, 118, 405, 0, 0, 343, 347, 350, 351, 0, + 0, 0, 0, 0, 0, 0, 0, 348, 349, 0, + 0, 0, 0, 0, 0, 0, 345, 0, 196, 0, + 259, 264, 265, 279, 277, 280, 278, 281, 282, 274, + 269, 268, 267, 275, 276, 266, 273, 272, 358, 360, + 0, 361, 369, 0, 370, 0, 362, 359, 380, 0, + 381, 0, 357, 245, 247, 246, 243, 244, 250, 252, + 251, 248, 249, 255, 257, 256, 253, 254, 0, 0, + 227, 226, 232, 222, 223, 217, 415, 0, 198, 242, + 0, 0, 170, 0, 0, 166, 408, 196, 0, 0, + 0, 116, 0, 0, 120, 0, 0, 0, 132, 173, + 0, 0, 0, 389, 388, 391, 390, 393, 392, 395, + 394, 397, 396, 399, 398, 0, 0, 309, 196, 0, + 0, 0, 0, 352, 353, 354, 355, 0, 356, 0, + 0, 0, 0, 0, 0, 0, 311, 310, 386, 383, + 377, 367, 372, 375, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 365, 0, 370, 0, - 373, 0, 381, 0, 384, 0, 233, 228, 0, 153, - 152, 0, 172, 155, 157, 162, 163, 0, 151, 33, - 0, 0, 0, 0, 36, 38, 39, 196, 0, 35, - 121, 0, 0, 119, 140, 135, 134, 0, 0, 0, - 303, 0, 196, 0, 0, 0, 0, 0, 333, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 270, - 269, 0, 259, 262, 326, 327, 0, 0, 196, 0, - 307, 317, 318, 321, 322, 0, 324, 316, 319, 320, - 312, 311, 313, 314, 315, 343, 345, 367, 0, 372, - 0, 375, 383, 386, 0, 0, 229, 199, 196, 169, - 183, 185, 194, 186, 0, 174, 0, 160, 161, 159, - 165, 42, 45, 46, 43, 44, 47, 48, 62, 49, - 51, 50, 65, 52, 53, 54, 55, 56, 57, 58, - 59, 60, 61, 0, 0, 0, 0, 0, 414, 0, - 0, 416, 0, 34, 0, 0, 117, 0, 0, 0, - 0, 403, 0, 399, 0, 0, 304, 0, 338, 0, - 0, 331, 0, 0, 0, 0, 0, 0, 342, 0, - 0, 291, 0, 293, 0, 378, 377, 0, 196, 325, - 0, 0, 306, 0, 0, 0, 234, 230, 0, 0, - 0, 203, 204, 205, 206, 202, 207, 0, 192, 0, - 187, 297, 295, 298, 296, 299, 300, 301, 171, 178, - 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 109, 110, 113, 106, 113, 0, 0, 0, 32, 37, - 425, 0, 260, 0, 402, 401, 143, 0, 141, 305, - 339, 0, 335, 0, 334, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 340, 329, 328, 0, - 0, 194, 184, 0, 0, 191, 0, 0, 176, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 366, + 0, 371, 0, 374, 0, 382, 0, 385, 0, 233, + 228, 0, 240, 0, 153, 152, 0, 172, 155, 157, + 162, 163, 0, 151, 33, 0, 0, 0, 0, 36, + 38, 39, 196, 0, 35, 121, 0, 0, 119, 140, + 135, 134, 0, 0, 0, 304, 0, 196, 0, 0, + 0, 0, 0, 334, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 271, 270, 0, 260, 263, 327, + 328, 0, 0, 196, 0, 308, 318, 319, 322, 323, + 0, 325, 317, 320, 321, 313, 312, 314, 315, 316, + 344, 346, 368, 0, 373, 0, 376, 384, 387, 0, + 0, 229, 0, 199, 196, 169, 183, 185, 194, 186, + 0, 174, 0, 160, 161, 159, 165, 42, 45, 46, + 43, 44, 47, 48, 62, 49, 51, 50, 65, 52, + 53, 54, 55, 56, 57, 58, 59, 60, 61, 0, + 0, 0, 0, 0, 415, 0, 0, 417, 0, 34, + 0, 0, 117, 0, 0, 0, 0, 404, 0, 400, + 0, 0, 305, 0, 339, 0, 0, 332, 0, 0, + 0, 0, 0, 0, 343, 0, 0, 292, 0, 294, + 0, 379, 378, 0, 196, 326, 0, 0, 307, 0, + 0, 0, 234, 230, 420, 0, 418, 0, 0, 0, + 203, 204, 205, 206, 202, 207, 0, 192, 0, 187, + 298, 296, 299, 297, 300, 301, 302, 171, 178, 158, + 0, 0, 0, 0, 0, 0, 0, 0, 109, 110, + 113, 106, 113, 0, 0, 0, 32, 37, 426, 0, + 261, 0, 403, 402, 143, 0, 141, 306, 340, 0, + 336, 0, 335, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 341, 330, 329, 0, 0, 0, + 414, 0, 194, 184, 0, 0, 191, 0, 0, 176, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 0, 108, 0, 107, 41, 40, 0, 115, 0, - 0, 0, 0, 400, 337, 332, 336, 323, 0, 0, - 0, 0, 0, 0, 362, 364, 363, 292, 294, 0, - 341, 330, 235, 231, 188, 200, 0, 0, 302, 0, - 0, 154, 64, 0, 0, 0, 0, 0, 0, 0, + 0, 111, 108, 0, 107, 41, 40, 0, 115, 0, + 0, 0, 0, 401, 338, 333, 337, 324, 0, 0, + 0, 0, 0, 0, 363, 365, 364, 293, 295, 0, + 342, 331, 235, 231, 421, 423, 422, 419, 188, 200, + 0, 0, 303, 0, 0, 154, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 419, 0, 417, 112, 114, 0, - 414, 0, 261, 382, 0, 289, 0, 0, 0, 0, - 0, 0, 177, 175, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 112, 114, + 0, 415, 0, 262, 383, 0, 290, 0, 0, 0, + 0, 0, 0, 177, 175, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 413, 0, 415, 423, - 414, 0, 0, 0, 0, 142, 201, 193, 63, 69, - 70, 67, 68, 71, 72, 73, 66, 93, 94, 91, - 92, 95, 96, 97, 90, 77, 78, 75, 76, 79, - 80, 81, 74, 101, 102, 99, 100, 103, 104, 105, - 98, 85, 86, 83, 84, 87, 88, 89, 82, 420, - 422, 421, 418, 424, 0, 290, 0, 414, 0, 0, - 283, 288, 0, 414, 0, 0, 0, 414, 286, 0, - 282, 284, 287, 414, 285 + 0, 0, 0, 0, 0, 0, 416, 424, 415, 0, + 0, 0, 0, 142, 201, 193, 63, 69, 70, 67, + 68, 71, 72, 73, 66, 93, 94, 91, 92, 95, + 96, 97, 90, 77, 78, 75, 76, 79, 80, 81, + 74, 101, 102, 99, 100, 103, 104, 105, 98, 85, + 86, 83, 84, 87, 88, 89, 82, 425, 0, 291, + 0, 415, 0, 0, 284, 289, 0, 415, 0, 0, + 0, 415, 287, 0, 283, 285, 288, 415, 286 }; /* YYPGOTO[NTERM-NUM]. */ static const yytype_int16 yypgoto[] = { - -642, -642, -642, 928, -642, 959, -642, 530, -642, 525, - -642, 437, 455, -642, -382, 994, 1001, 895, -642, -642, - 1002, -642, 780, 1009, 1011, -57, 1044, -15, 807, 922, - -44, -642, -642, 607, -642, -642, -642, -642, -642, -642, - -168, -642, -642, -642, -642, 526, -274, 23, 456, -642, - -642, 930, -642, -642, 1017, 1018, 1019, 1020, 1021, -154, - -642, 781, -178, -180, -642, -451, -445, -439, -431, -416, - -415, 457, -642, -642, -642, -642, -642, -642, 782, -642, - -642, 670, 413, -202, -642, -642, -642, 475, -642, -642, - -642, -642, 477, 783, 784, -125, -642, -642, -642, -642, - 891, -389, 492, -112, 405, 454, -642, -642, -641, -642, - 414, 274, -642 + -499, -499, -499, 998, -499, 1006, -499, 550, -499, 540, + -499, 474, 479, -499, -385, 1007, 1008, 901, -499, -499, + 1009, -499, 785, 1012, 1015, -57, 1058, -15, 808, 923, + -44, -499, -499, 606, -499, -499, -499, -499, -499, -499, + -168, -499, -499, -499, -499, 521, -233, 36, 449, -499, + -499, 932, -499, -499, 1022, 1023, 1024, 1025, 1026, -156, + -499, 745, -180, -182, -499, -451, -450, -449, -436, -425, + -423, 453, -499, -499, -499, -499, -499, -499, 780, -499, + -499, 674, 405, -204, -499, -499, -499, 480, -499, -499, + -499, -499, 483, 730, 731, -400, -499, -499, -499, -499, + 895, -394, 491, -113, 326, 294, -499, -499, -498, -499, + 408, 471, -499 }; /* YYDEFGOTO[NTERM-NUM]. */ static const yytype_int16 yydefgoto[] = { - 0, 17, 18, 19, 122, 20, 383, 384, 385, 498, - 582, 583, 675, 386, 279, 21, 22, 167, 23, 61, - 24, 176, 177, 25, 26, 27, 28, 29, 97, 153, - 98, 158, 373, 374, 469, 272, 378, 156, 372, 465, - 179, 711, 628, 95, 459, 460, 461, 462, 560, 30, - 84, 85, 463, 557, 31, 32, 33, 34, 35, 207, - 393, 208, 209, 210, 886, 211, 212, 213, 214, 215, - 216, 567, 568, 217, 218, 219, 220, 221, 309, 222, - 223, 224, 225, 226, 693, 227, 228, 229, 230, 231, - 232, 233, 234, 329, 330, 235, 236, 237, 238, 239, - 240, 512, 513, 181, 108, 100, 91, 105, 584, 588, - 755, 756, 389 + 0, 17, 18, 19, 123, 20, 388, 389, 390, 504, + 590, 591, 685, 391, 282, 21, 22, 169, 23, 61, + 24, 178, 179, 25, 26, 27, 28, 29, 98, 155, + 99, 160, 378, 379, 475, 275, 383, 158, 377, 471, + 181, 725, 639, 96, 465, 466, 467, 468, 569, 30, + 85, 86, 469, 566, 31, 32, 33, 34, 35, 209, + 398, 210, 211, 212, 890, 213, 214, 215, 216, 217, + 218, 576, 577, 219, 220, 221, 222, 223, 312, 224, + 225, 226, 227, 228, 703, 229, 230, 231, 232, 233, + 234, 235, 236, 332, 333, 237, 238, 239, 240, 241, + 242, 518, 519, 183, 109, 101, 92, 106, 372, 596, + 555, 556, 394 }; /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM. If @@ -1201,240 +1201,232 @@ static const yytype_int16 yydefgoto[] = number is the opposite. If YYTABLE_NINF, syntax error. */ static const yytype_int16 yytable[] = { - 286, 88, 269, 328, 129, 182, 504, 304, 285, 514, - 47, 96, 308, 561, 184, 185, 186, 325, 326, 562, - 325, 326, 323, 324, 445, 563, 178, 48, 332, 50, - -409, 528, 14, 564, 380, 266, 82, 1, 371, 2, - 3, 4, 5, 6, 7, 8, 9, 274, 565, 566, - 558, 154, 604, 10, 335, 11, 12, 13, 92, 41, - 93, 426, 94, 280, 106, 356, 241, 49, 242, 243, - 357, 1, 115, 2, 3, 4, 5, 6, 7, 8, - 9, 246, 333, 247, 248, 334, 136, 10, 47, 11, - 12, 13, 375, 376, 390, 77, 251, 391, 252, 253, - 144, -412, 429, 81, 559, 395, 467, 468, 268, 14, - 14, 99, 191, 192, 193, 194, 336, 337, 586, 819, - 116, 117, 304, 591, 161, 162, 163, 244, 405, 406, - 519, 83, 170, 171, 172, 401, 427, 86, 16, 195, - 196, 197, 249, 685, 14, 307, 336, 337, 284, 355, - 334, 609, 281, 430, 381, 447, 382, 254, 424, 425, - 89, 431, 432, 433, 434, 435, 436, 437, 438, 439, - 440, 441, 442, 443, 444, 561, 275, 455, 456, 873, - 358, 562, 520, 336, 337, 359, 99, 563, 277, 152, - 90, 331, 15, 458, 205, 564, 205, 327, 501, 470, - 327, 502, 446, 677, 605, 336, 337, 336, 337, 204, - 565, 566, 183, 184, 185, 186, 16, 379, 138, 139, - 336, 337, 336, 337, 245, 621, 15, 577, 118, 336, - 337, 523, 524, 96, 526, 360, 881, 530, 507, 250, - 361, 107, 888, 336, 337, 515, 892, 539, 340, 113, - 16, 114, 894, 1, 255, 2, 3, 4, 5, 6, - 7, 607, 9, 119, 403, 336, 337, -413, -413, 10, - 610, 11, 12, 13, 541, 579, 399, 580, 581, 137, - 673, 577, 362, 120, 187, 188, 578, 363, 375, 364, - 404, 140, 141, 189, 365, 190, 121, 569, -413, -413, - 350, 351, 352, 353, 354, 394, 78, 79, 80, 537, - 769, 191, 192, 193, 194, 869, 340, 870, 871, 183, - 184, 185, 186, 336, 337, 771, 14, 336, 337, 579, - 503, 580, 581, 341, 342, 343, 344, 135, 195, 196, - 197, 346, 602, 142, 603, 606, 36, 37, 38, 705, - 706, 51, 52, 183, 184, 185, 186, 53, 39, 40, - 198, 420, 143, 618, 145, 347, 348, 349, 350, 351, - 352, 353, 354, 42, 43, 44, 687, 146, 92, 199, - 93, 200, 94, 201, 615, 45, 46, 517, 202, 203, - 204, 187, 188, 205, 256, 206, 400, 147, 257, 258, - 189, 548, 190, 259, 260, 307, 516, 691, 15, 334, - 148, 149, 317, 540, 318, 319, 320, 151, 191, 192, - 193, 194, 352, 353, 354, 187, 188, 686, 535, 536, - 14, 408, 16, 409, 189, 410, 190, 155, 521, 682, - 522, 531, 410, 157, 532, 195, 196, 197, 109, 110, - 111, 112, 191, 192, 193, 194, 403, 630, 631, 632, - 633, 634, 159, 533, 635, 636, 534, 198, 183, 184, - 185, 186, 199, 160, 200, 590, 201, 164, 391, 195, - 196, 197, 689, 166, 637, 696, 199, 168, 200, 165, - 201, 101, 102, 103, 104, 202, 203, 204, 325, 763, - 205, 198, 206, 169, 183, 184, 185, 186, 340, 173, - 638, 639, 640, 641, 642, 174, 592, 643, 644, 334, - 199, 175, 200, 614, 201, 341, 342, 343, 344, 202, - 203, 204, 773, 346, 205, 596, 206, 645, 597, 180, - 187, 188, 54, 55, 56, 57, 58, 59, 261, 189, - 60, 190, 509, 510, 511, 772, 178, 347, 348, 349, - 350, 351, 352, 353, 354, 262, 263, 191, 192, 193, - 194, 646, 647, 648, 649, 650, 302, 303, 651, 652, - 598, 599, 270, 597, 334, 189, 617, 190, 264, 334, - 273, 338, 826, 339, 195, 196, 197, 678, 653, 681, - 391, 271, 391, 191, 192, 193, 194, 183, 184, 185, - 186, 654, 655, 656, 657, 658, 198, 701, 659, 660, - 334, 712, 758, 278, 713, 391, 62, 63, 282, 64, - 195, 196, 197, 762, 283, 199, 334, 200, 661, 201, - 310, 65, 66, 340, 202, 203, 204, 765, 287, 205, - 766, 206, 198, 816, 818, 825, 817, 817, 597, 288, - 341, 342, 343, 344, 345, 311, 827, 289, 346, 391, - 305, 199, 306, 200, 877, 201, 312, 878, 313, 302, - 202, 203, 204, 883, 314, 205, 884, 206, 189, 316, - 190, 321, 347, 348, 349, 350, 351, 352, 353, 354, - 366, 322, 367, 377, 387, 355, 191, 192, 193, 194, - 662, 663, 664, 665, 666, 369, 370, 667, 668, 388, - 392, 14, 397, 403, 550, -208, 551, 552, 553, 554, - 398, 555, 556, 195, 196, 197, 402, 669, 290, 291, - 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, - 411, 412, 67, 68, 69, 198, 70, 71, 413, 414, - 415, 72, 73, 74, 416, 417, 418, 419, 421, 75, - 76, 423, 428, 205, 199, 340, 200, 448, 201, 450, - 452, 453, 454, 202, 203, 204, 340, 457, 205, 466, - 206, 464, 341, 342, 343, 344, 500, 543, 505, 499, - 346, 506, 427, -413, -413, 343, 344, 508, 518, 525, - 527, -413, 336, 538, 542, 544, 545, 546, 547, 549, - 571, 572, 587, 573, 347, 348, 349, 350, 351, 352, - 353, 354, 574, 594, 595, -413, 348, 349, 350, 351, - 352, 353, 354, 471, 472, 473, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 600, 575, 492, 576, 585, 493, - 494, 593, 611, 495, 496, 497, 608, 612, 616, 613, - 536, 535, 620, 619, 623, 624, 625, 627, 626, 670, - 629, 671, 673, 672, 679, 680, 684, 688, 690, 692, - 709, 697, 702, 698, 699, 703, 700, 707, 710, 754, - 714, 760, 715, 716, 717, 718, 719, 720, 721, 722, - 723, 770, 724, 725, 726, 727, 774, 775, 728, 729, - 730, 731, 732, 733, 734, 735, 761, 736, 737, 738, - 739, 740, 741, 776, 742, 743, 744, 745, 746, 747, - 777, 778, 748, 779, 780, 749, 750, 751, 781, 782, - 752, 783, 753, 784, 785, 764, 786, 767, 787, 768, - 788, 789, 790, 791, 792, 793, 794, 795, 796, 797, - 798, 799, 800, 801, 802, 803, 804, 805, 806, 807, - 808, 809, 810, 811, 812, 813, 814, 815, 578, 820, - 334, 821, 822, 823, 824, 828, 829, 830, 879, 831, - 832, 833, 880, 882, 834, 835, 889, 890, 150, 674, - 123, 836, 837, 838, 839, 840, 841, 842, 843, 844, - 845, 846, 589, 847, 848, 849, 850, 851, 852, 676, - 853, 854, 855, 856, 601, 857, 858, 859, 860, 861, - 862, 863, 864, 865, 866, 124, 867, 868, 276, 875, - 87, 874, 125, 126, 396, 876, 887, 885, 891, 893, - 127, 368, 128, 570, 267, 622, 265, 704, 130, 131, - 132, 133, 134, 708, 529, 694, 757, 695, 315, 683, - 407, 872, 0, 759, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 422, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 449, 0, 0, 451 + 289, 89, 272, 331, 130, 184, 592, 307, 510, 520, + 288, 47, 311, 97, 450, 277, 186, 187, 188, 570, + 571, 572, 326, 327, 567, 328, 329, 146, 335, 328, + 329, 14, 534, 385, 573, 180, 269, 376, 612, 338, + 48, 47, 50, 431, -410, 574, 93, 575, 94, 83, + 95, 1, 156, 2, 3, 4, 5, 6, 7, 8, + 9, 243, -413, 244, 245, 336, 526, 10, 337, 11, + 12, 13, 100, 248, 434, 249, 250, 107, 568, 339, + 340, 117, 118, 41, 253, 116, 254, 255, 525, 283, + 77, 51, 52, 380, 381, 339, 340, 53, 81, 137, + 473, 474, 339, 340, 359, 714, 400, 715, 716, 360, + 271, 339, 340, 145, 193, 194, 195, 196, 432, 695, + 615, 594, 246, 307, 14, 435, 599, 339, 340, 410, + 411, 339, 340, 425, 251, 14, 406, 16, 163, 164, + 165, 197, 198, 199, 278, 256, 172, 173, 174, 337, + 82, 114, 618, 386, 284, 387, 452, 287, 49, 429, + 430, 613, 436, 437, 438, 439, 440, 441, 442, 443, + 444, 445, 446, 447, 448, 449, 586, 339, 340, 84, + 339, 340, 266, 339, 340, 310, 570, 571, 572, 119, + 87, 154, 451, 334, 464, 90, 207, 185, 186, 187, + 188, 573, 476, 280, 207, 330, 15, 91, 687, 330, + 206, 97, 574, 632, 575, 699, 586, 100, 706, 247, + 384, 371, 339, 340, 587, 108, 588, 589, 361, 683, + 16, 252, 343, 362, 529, 530, 114, 532, 363, 365, + 536, 513, 257, 364, 366, 339, 340, 120, 521, 258, + 545, -414, -414, 259, 260, 139, 140, 367, 261, 262, + 115, 404, 368, 138, 587, 395, 588, 589, 396, 189, + 190, 121, 320, 827, 321, 322, 323, 547, 191, 358, + 192, 617, -414, -414, 353, 354, 355, 356, 357, 36, + 37, 38, 380, 409, 141, 142, 193, 194, 195, 196, + 147, 39, 40, 578, 185, 186, 187, 188, 93, 122, + 94, 507, 95, 543, 508, 780, 42, 43, 44, 522, + 136, 399, 337, 197, 198, 199, 339, 340, 45, 46, + 877, 102, 103, 104, 105, 509, 782, 143, 185, 186, + 187, 188, 541, 542, 537, 200, 610, 538, 611, 614, + 460, 461, 641, 642, 643, 644, 645, 328, 774, 646, + 647, 413, 148, 414, 201, 415, 202, 626, 203, 110, + 111, 112, 113, 204, 205, 206, 189, 190, 207, 648, + 208, 405, 527, 885, 528, 191, 415, 192, 623, 892, + 310, 408, 523, 896, 201, 144, 202, 539, 203, 898, + 540, 719, 720, 193, 194, 195, 196, 557, 149, 598, + 189, 190, 396, 701, 600, 14, 150, 337, 546, 191, + 604, 192, 606, 605, 607, 605, 625, 337, 151, 337, + 197, 198, 199, 696, 78, 79, 80, 193, 194, 195, + 196, 630, 153, 343, 631, 692, 688, 157, 159, 396, + 691, 161, 200, 396, 185, 186, 187, 188, 162, 166, + 344, 345, 346, 347, 197, 198, 199, 711, 349, 726, + 337, 201, 727, 202, 167, 203, 769, 168, 171, 396, + 204, 205, 206, 170, 175, 207, 200, 208, 185, 186, + 187, 188, 350, 351, 352, 353, 354, 355, 356, 357, + 176, 773, 177, 697, 337, 201, 180, 202, 776, 203, + 826, 777, 182, 631, 204, 205, 206, 833, 263, 207, + 605, 208, 355, 356, 357, 835, 189, 190, 396, 622, + 881, 887, 264, 882, 888, 191, 265, 192, 515, 516, + 517, 267, 273, 274, 784, 276, 185, 186, 187, 188, + 281, 286, 285, 193, 194, 195, 196, 290, 291, 292, + 305, 306, 313, 308, 309, 314, 315, 783, 316, 191, + 319, 192, 317, 324, 325, 369, 370, 358, 371, 374, + 197, 198, 199, 375, 382, 392, 393, 193, 194, 195, + 196, 559, -208, 560, 561, 562, 563, 397, 564, 565, + 402, 834, 200, 403, 62, 63, 407, 64, 416, 417, + 418, 14, 419, 420, 197, 198, 199, 421, 305, 65, + 66, 201, 422, 202, 423, 203, 424, 191, 426, 192, + 204, 205, 206, 428, 433, 207, 200, 208, 207, 453, + 455, 457, 459, 458, 462, 193, 194, 195, 196, 463, + 472, 470, 506, 511, 512, 201, 505, 202, 514, 203, + 531, 432, 533, 524, 204, 205, 206, 339, 548, 207, + 544, 208, 197, 198, 199, 552, 554, 1, 553, 2, + 3, 4, 5, 6, 7, 8, 9, 550, 580, 581, + 341, 551, 342, 10, 200, 11, 12, 13, 582, 408, + 1, 558, 2, 3, 4, 5, 6, 7, 583, 9, + 584, 585, 408, 201, 595, 202, 10, 203, 11, 12, + 13, 593, 204, 205, 206, 601, 602, 207, 603, 208, + 67, 68, 69, 608, 70, 71, 619, 620, 616, 72, + 73, 74, 343, 621, 624, 542, 541, 75, 76, 627, + 14, 343, 628, 629, 634, 636, 635, 637, 638, 344, + 345, 346, 347, 348, 343, 640, 681, 349, 344, 345, + 346, 347, 682, 14, 683, 689, 349, 690, 694, 698, + 700, 344, 345, 346, 347, 702, 549, 707, 709, 349, + 708, 350, 351, 352, 353, 354, 355, 356, 357, 710, + 350, 351, 352, 353, 354, 355, 356, 357, 343, 712, + 713, 721, 723, 350, 351, 352, 353, 354, 355, 356, + 357, 724, 728, 771, 772, -414, -414, 346, 347, 729, + 785, 730, 15, -414, 731, 732, 786, 733, 649, 650, + 651, 652, 653, 734, 735, 654, 655, 736, 781, 737, + 738, 739, 740, 787, 741, 15, 16, -414, 351, 352, + 353, 354, 355, 356, 357, 656, 293, 294, 295, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 742, 16, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, + 497, 743, 744, 498, 745, 746, 499, 500, 747, 748, + 501, 502, 503, 657, 658, 659, 660, 661, 749, 750, + 662, 663, 665, 666, 667, 668, 669, 751, 752, 670, + 671, 673, 674, 675, 676, 677, 753, 788, 678, 679, + 664, 754, 54, 55, 56, 57, 58, 59, 755, 672, + 60, 756, 757, 758, 759, 789, 760, 761, 680, 762, + 763, 764, 765, 766, 767, 775, 790, 778, 779, 791, + 792, 793, 794, 795, 796, 797, 798, 799, 800, 801, + 802, 803, 804, 805, 806, 807, 808, 809, 810, 811, + 812, 813, 814, 815, 816, 817, 818, 819, 820, 821, + 822, 823, 824, 825, 828, 337, 829, 830, 831, 832, + 836, 878, 837, 838, 839, 840, 841, 883, 884, 842, + 843, 844, 845, 846, 847, 848, 849, 886, 850, 851, + 852, 853, 854, 855, 893, 856, 857, 858, 859, 860, + 861, 862, 863, 864, 865, 866, 867, 868, 869, 870, + 871, 872, 873, 874, 875, 876, 879, 894, 597, 891, + 880, 889, 895, 897, 684, 609, 279, 124, 125, 126, + 127, 686, 401, 128, 88, 373, 129, 270, 579, 633, + 268, 718, 427, 131, 132, 133, 134, 135, 768, 152, + 722, 412, 454, 535, 318, 456, 693, 770, 704, 0, + 0, 705, 717 }; static const yytype_int16 yycheck[] = { - 178, 16, 156, 205, 61, 117, 388, 187, 176, 398, - 3, 8, 190, 464, 4, 5, 6, 5, 6, 464, - 5, 6, 202, 203, 3, 464, 68, 4, 206, 6, - 0, 3, 80, 464, 3, 147, 13, 7, 78, 9, - 10, 11, 12, 13, 14, 15, 16, 57, 464, 464, - 3, 95, 85, 23, 57, 25, 26, 27, 20, 33, - 22, 77, 24, 34, 41, 185, 3, 78, 5, 6, - 190, 7, 49, 9, 10, 11, 12, 13, 14, 15, - 16, 3, 187, 5, 6, 190, 63, 23, 3, 25, - 26, 27, 270, 271, 187, 30, 3, 190, 5, 6, - 77, 64, 75, 3, 57, 283, 72, 73, 152, 80, - 80, 74, 102, 103, 104, 105, 149, 150, 500, 760, - 78, 79, 302, 505, 101, 102, 103, 64, 306, 307, - 57, 3, 109, 110, 111, 289, 152, 163, 186, 129, - 130, 131, 64, 85, 80, 89, 149, 150, 190, 188, - 190, 190, 167, 126, 123, 357, 125, 64, 336, 337, - 0, 341, 342, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 626, 186, 48, 49, 820, - 185, 626, 88, 149, 150, 190, 74, 626, 165, 186, - 189, 206, 162, 186, 184, 626, 184, 185, 187, 377, - 185, 190, 181, 585, 88, 149, 150, 149, 150, 181, - 626, 626, 3, 4, 5, 6, 186, 274, 167, 168, - 149, 150, 149, 150, 161, 187, 162, 75, 186, 149, - 150, 409, 410, 8, 412, 185, 877, 415, 392, 161, - 190, 74, 883, 149, 150, 399, 887, 427, 127, 188, - 186, 14, 893, 7, 161, 9, 10, 11, 12, 13, - 14, 190, 16, 3, 75, 149, 150, 146, 147, 23, - 190, 25, 26, 27, 428, 123, 67, 125, 126, 6, - 128, 75, 185, 3, 75, 76, 80, 190, 466, 185, - 305, 167, 168, 84, 190, 86, 3, 465, 177, 178, - 179, 180, 181, 182, 183, 282, 153, 154, 155, 421, - 699, 102, 103, 104, 105, 3, 127, 5, 6, 3, - 4, 5, 6, 149, 150, 707, 80, 149, 150, 123, - 387, 125, 126, 144, 145, 146, 147, 3, 129, 130, - 131, 152, 520, 3, 522, 525, 29, 30, 31, 623, - 624, 159, 160, 3, 4, 5, 6, 165, 41, 42, - 151, 187, 159, 543, 57, 176, 177, 178, 179, 180, - 181, 182, 183, 29, 30, 31, 187, 190, 20, 170, - 22, 172, 24, 174, 538, 41, 42, 402, 179, 180, - 181, 75, 76, 184, 42, 186, 187, 6, 46, 47, - 84, 458, 86, 51, 52, 89, 187, 609, 162, 190, - 187, 187, 117, 428, 119, 120, 121, 21, 102, 103, - 104, 105, 181, 182, 183, 75, 76, 605, 5, 6, - 80, 85, 186, 87, 84, 89, 86, 67, 85, 593, - 87, 187, 89, 69, 190, 129, 130, 131, 43, 44, - 45, 46, 102, 103, 104, 105, 75, 91, 92, 93, - 94, 95, 75, 187, 98, 99, 190, 151, 3, 4, - 5, 6, 170, 3, 172, 187, 174, 3, 190, 129, - 130, 131, 607, 186, 118, 610, 170, 76, 172, 64, - 174, 37, 38, 39, 40, 179, 180, 181, 5, 6, - 184, 151, 186, 3, 3, 4, 5, 6, 127, 3, - 91, 92, 93, 94, 95, 3, 187, 98, 99, 190, - 170, 3, 172, 538, 174, 144, 145, 146, 147, 179, - 180, 181, 710, 152, 184, 187, 186, 118, 190, 4, - 75, 76, 35, 36, 37, 38, 39, 40, 3, 84, - 43, 86, 81, 82, 83, 709, 68, 176, 177, 178, - 179, 180, 181, 182, 183, 3, 6, 102, 103, 104, - 105, 91, 92, 93, 94, 95, 75, 76, 98, 99, - 187, 187, 55, 190, 190, 84, 187, 86, 186, 190, - 76, 75, 770, 77, 129, 130, 131, 187, 118, 187, - 190, 70, 190, 102, 103, 104, 105, 3, 4, 5, - 6, 91, 92, 93, 94, 95, 151, 187, 98, 99, - 190, 187, 187, 3, 190, 190, 29, 30, 64, 32, - 129, 130, 131, 187, 176, 170, 190, 172, 118, 174, - 4, 44, 45, 127, 179, 180, 181, 187, 80, 184, - 190, 186, 151, 187, 187, 187, 190, 190, 190, 80, - 144, 145, 146, 147, 148, 4, 187, 186, 152, 190, - 186, 170, 186, 172, 187, 174, 4, 190, 4, 75, - 179, 180, 181, 187, 6, 184, 190, 186, 84, 186, - 86, 186, 176, 177, 178, 179, 180, 181, 182, 183, - 3, 186, 6, 71, 57, 188, 102, 103, 104, 105, - 91, 92, 93, 94, 95, 187, 187, 98, 99, 186, - 186, 80, 186, 75, 58, 59, 60, 61, 62, 63, - 186, 65, 66, 129, 130, 131, 186, 118, 132, 133, - 134, 135, 136, 137, 138, 139, 140, 141, 142, 143, - 4, 186, 155, 156, 157, 151, 159, 160, 186, 186, - 186, 164, 165, 166, 4, 4, 191, 187, 79, 172, - 173, 3, 186, 184, 170, 127, 172, 6, 174, 6, - 6, 5, 46, 179, 180, 181, 127, 187, 184, 190, - 186, 169, 144, 145, 146, 147, 186, 149, 186, 124, - 152, 3, 152, 144, 145, 146, 147, 190, 187, 78, - 4, 152, 149, 186, 126, 191, 191, 6, 6, 190, - 186, 186, 28, 186, 176, 177, 178, 179, 180, 181, - 182, 183, 186, 4, 3, 176, 177, 178, 179, 180, - 181, 182, 183, 90, 91, 92, 93, 94, 95, 96, - 97, 98, 99, 100, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 187, 186, 113, 186, 186, 116, - 117, 186, 4, 120, 121, 122, 190, 4, 187, 80, - 6, 5, 50, 53, 59, 59, 3, 54, 190, 126, - 6, 186, 128, 124, 186, 158, 187, 184, 4, 184, - 55, 187, 6, 187, 186, 6, 187, 186, 56, 3, - 190, 3, 190, 190, 190, 190, 190, 190, 190, 190, - 190, 64, 190, 190, 190, 190, 6, 6, 190, 190, - 190, 190, 190, 190, 190, 190, 158, 190, 190, 190, - 190, 190, 190, 6, 190, 190, 190, 190, 190, 190, - 6, 6, 190, 6, 6, 190, 190, 190, 6, 6, - 190, 6, 190, 6, 6, 190, 6, 190, 6, 190, + 180, 16, 158, 207, 61, 118, 504, 189, 393, 403, + 178, 3, 192, 8, 3, 57, 4, 5, 6, 470, + 470, 470, 204, 205, 3, 5, 6, 64, 208, 5, + 6, 80, 3, 3, 470, 68, 149, 78, 85, 57, + 4, 3, 6, 77, 0, 470, 20, 470, 22, 13, + 24, 7, 96, 9, 10, 11, 12, 13, 14, 15, + 16, 3, 64, 5, 6, 187, 88, 23, 190, 25, + 26, 27, 74, 3, 75, 5, 6, 41, 57, 149, + 150, 78, 79, 33, 3, 49, 5, 6, 57, 34, + 30, 159, 160, 273, 274, 149, 150, 165, 3, 63, + 72, 73, 149, 150, 185, 3, 286, 5, 6, 190, + 154, 149, 150, 77, 102, 103, 104, 105, 152, 85, + 190, 506, 64, 305, 80, 126, 511, 149, 150, 309, + 310, 149, 150, 187, 64, 80, 292, 186, 102, 103, + 104, 129, 130, 131, 186, 64, 110, 111, 112, 190, + 3, 188, 190, 123, 169, 125, 360, 190, 78, 339, + 340, 88, 344, 345, 346, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 75, 149, 150, 3, + 149, 150, 146, 149, 150, 89, 637, 637, 637, 186, + 163, 186, 181, 208, 186, 0, 184, 3, 4, 5, + 6, 637, 382, 167, 184, 185, 162, 189, 593, 185, + 181, 8, 637, 187, 637, 615, 75, 74, 618, 161, + 277, 80, 149, 150, 123, 74, 125, 126, 185, 128, + 186, 161, 127, 190, 414, 415, 188, 417, 185, 185, + 420, 397, 161, 190, 190, 149, 150, 3, 404, 42, + 432, 146, 147, 46, 47, 167, 168, 185, 51, 52, + 14, 67, 190, 6, 123, 187, 125, 126, 190, 75, + 76, 3, 117, 771, 119, 120, 121, 433, 84, 188, + 86, 190, 177, 178, 179, 180, 181, 182, 183, 29, + 30, 31, 472, 308, 167, 168, 102, 103, 104, 105, + 57, 41, 42, 471, 3, 4, 5, 6, 20, 3, + 22, 187, 24, 426, 190, 709, 29, 30, 31, 187, + 3, 285, 190, 129, 130, 131, 149, 150, 41, 42, + 828, 37, 38, 39, 40, 392, 721, 3, 3, 4, + 5, 6, 5, 6, 187, 151, 526, 190, 528, 531, + 48, 49, 91, 92, 93, 94, 95, 5, 6, 98, + 99, 85, 190, 87, 170, 89, 172, 549, 174, 43, + 44, 45, 46, 179, 180, 181, 75, 76, 184, 118, + 186, 187, 85, 881, 87, 84, 89, 86, 544, 887, + 89, 75, 407, 891, 170, 159, 172, 187, 174, 897, + 190, 634, 635, 102, 103, 104, 105, 464, 6, 187, + 75, 76, 190, 617, 187, 80, 187, 190, 433, 84, + 187, 86, 187, 190, 187, 190, 187, 190, 187, 190, + 129, 130, 131, 613, 153, 154, 155, 102, 103, 104, + 105, 187, 21, 127, 190, 601, 187, 67, 69, 190, + 187, 75, 151, 190, 3, 4, 5, 6, 3, 3, + 144, 145, 146, 147, 129, 130, 131, 187, 152, 187, + 190, 170, 190, 172, 64, 174, 187, 186, 3, 190, + 179, 180, 181, 76, 3, 184, 151, 186, 3, 4, + 5, 6, 176, 177, 178, 179, 180, 181, 182, 183, + 3, 187, 3, 187, 190, 170, 68, 172, 187, 174, + 187, 190, 4, 190, 179, 180, 181, 187, 3, 184, + 190, 186, 181, 182, 183, 187, 75, 76, 190, 544, + 187, 187, 3, 190, 190, 84, 6, 86, 81, 82, + 83, 186, 55, 70, 724, 76, 3, 4, 5, 6, + 3, 176, 64, 102, 103, 104, 105, 80, 80, 186, + 75, 76, 4, 186, 186, 4, 4, 723, 4, 84, + 186, 86, 6, 186, 186, 3, 6, 188, 80, 187, + 129, 130, 131, 187, 71, 57, 186, 102, 103, 104, + 105, 58, 59, 60, 61, 62, 63, 186, 65, 66, + 186, 781, 151, 186, 29, 30, 186, 32, 4, 186, + 186, 80, 186, 186, 129, 130, 131, 4, 75, 44, + 45, 170, 4, 172, 191, 174, 187, 84, 79, 86, + 179, 180, 181, 3, 186, 184, 151, 186, 184, 6, + 6, 6, 46, 5, 186, 102, 103, 104, 105, 187, + 190, 169, 186, 186, 3, 170, 124, 172, 190, 174, + 78, 152, 4, 187, 179, 180, 181, 149, 126, 184, + 186, 186, 129, 130, 131, 6, 3, 7, 6, 9, + 10, 11, 12, 13, 14, 15, 16, 191, 186, 186, + 75, 191, 77, 23, 151, 25, 26, 27, 186, 75, + 7, 190, 9, 10, 11, 12, 13, 14, 186, 16, + 186, 186, 75, 170, 28, 172, 23, 174, 25, 26, + 27, 186, 179, 180, 181, 186, 4, 184, 3, 186, + 155, 156, 157, 187, 159, 160, 4, 4, 190, 164, + 165, 166, 127, 80, 187, 6, 5, 172, 173, 53, + 80, 127, 50, 176, 59, 3, 59, 190, 54, 144, + 145, 146, 147, 148, 127, 6, 126, 152, 144, 145, + 146, 147, 124, 80, 128, 186, 152, 158, 187, 184, + 4, 144, 145, 146, 147, 184, 149, 187, 186, 152, + 187, 176, 177, 178, 179, 180, 181, 182, 183, 187, + 176, 177, 178, 179, 180, 181, 182, 183, 127, 6, + 6, 186, 55, 176, 177, 178, 179, 180, 181, 182, + 183, 56, 190, 3, 158, 144, 145, 146, 147, 190, + 6, 190, 162, 152, 190, 190, 6, 190, 91, 92, + 93, 94, 95, 190, 190, 98, 99, 190, 64, 190, + 190, 190, 190, 6, 190, 162, 186, 176, 177, 178, + 179, 180, 181, 182, 183, 118, 132, 133, 134, 135, + 136, 137, 138, 139, 140, 141, 142, 143, 190, 186, + 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 190, 190, 113, 190, 190, 116, 117, 190, 190, + 120, 121, 122, 91, 92, 93, 94, 95, 190, 190, + 98, 99, 91, 92, 93, 94, 95, 190, 190, 98, + 99, 91, 92, 93, 94, 95, 190, 6, 98, 99, + 118, 190, 35, 36, 37, 38, 39, 40, 190, 118, + 43, 190, 190, 190, 190, 6, 190, 190, 118, 190, + 190, 190, 190, 190, 190, 190, 6, 190, 190, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 176, 80, 3, - 190, 4, 4, 4, 4, 187, 187, 187, 4, 187, - 187, 187, 4, 6, 187, 187, 6, 4, 90, 582, - 61, 187, 187, 187, 187, 187, 187, 187, 187, 187, - 187, 187, 502, 187, 187, 187, 187, 187, 187, 584, - 187, 187, 187, 187, 519, 187, 187, 187, 187, 187, - 187, 187, 187, 187, 187, 61, 187, 187, 163, 187, - 16, 190, 61, 61, 284, 190, 187, 190, 187, 187, - 61, 264, 61, 466, 152, 549, 146, 621, 61, 61, - 61, 61, 61, 626, 414, 610, 673, 610, 197, 597, - 308, 817, -1, 679, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 334, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 359, -1, -1, 361 + 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, + 6, 6, 6, 6, 3, 190, 4, 4, 4, 4, + 187, 190, 187, 187, 187, 187, 187, 4, 4, 187, + 187, 187, 187, 187, 187, 187, 187, 6, 187, 187, + 187, 187, 187, 187, 6, 187, 187, 187, 187, 187, + 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, + 187, 187, 187, 187, 187, 187, 187, 4, 508, 187, + 190, 190, 187, 187, 590, 525, 165, 61, 61, 61, + 61, 592, 287, 61, 16, 267, 61, 154, 472, 558, + 148, 632, 337, 61, 61, 61, 61, 61, 683, 91, + 637, 311, 362, 419, 199, 364, 605, 689, 618, -1, + -1, 618, 631 }; /* YYSTOS[STATE-NUM] -- The symbol kind of the accessing symbol of @@ -1449,88 +1441,88 @@ static const yytype_int16 yystos[] = 239, 159, 160, 165, 35, 36, 37, 38, 39, 40, 43, 211, 29, 30, 32, 44, 45, 155, 156, 157, 159, 160, 164, 165, 166, 172, 173, 30, 153, 154, - 155, 3, 239, 3, 242, 243, 163, 218, 219, 0, - 189, 298, 20, 22, 24, 235, 8, 220, 222, 74, - 297, 297, 297, 297, 297, 299, 239, 74, 296, 296, - 296, 296, 296, 188, 14, 239, 78, 79, 186, 3, - 3, 3, 196, 197, 207, 208, 212, 215, 216, 217, - 246, 247, 248, 249, 250, 3, 239, 6, 167, 168, - 167, 168, 3, 159, 239, 57, 190, 6, 187, 187, - 195, 21, 186, 221, 222, 67, 229, 69, 223, 75, - 3, 239, 239, 239, 3, 64, 186, 209, 76, 3, - 239, 239, 239, 3, 3, 3, 213, 214, 68, 232, - 4, 295, 295, 3, 4, 5, 6, 75, 76, 84, - 86, 102, 103, 104, 105, 129, 130, 131, 151, 170, - 172, 174, 179, 180, 181, 184, 186, 251, 253, 254, - 255, 257, 258, 259, 260, 261, 262, 265, 266, 267, - 268, 269, 271, 272, 273, 274, 275, 277, 278, 279, - 280, 281, 282, 283, 284, 287, 288, 289, 290, 291, - 292, 3, 5, 6, 64, 161, 3, 5, 6, 64, - 161, 3, 5, 6, 64, 161, 42, 46, 47, 51, - 52, 3, 3, 6, 186, 243, 295, 221, 222, 251, - 55, 70, 227, 76, 57, 186, 209, 239, 3, 206, - 34, 219, 64, 176, 190, 232, 254, 80, 80, 186, - 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, - 142, 143, 75, 76, 255, 186, 186, 89, 254, 270, - 4, 4, 4, 4, 6, 292, 186, 117, 119, 120, - 121, 186, 186, 255, 255, 5, 6, 185, 275, 285, - 286, 219, 254, 187, 190, 57, 149, 150, 75, 77, - 127, 144, 145, 146, 147, 148, 152, 176, 177, 178, - 179, 180, 181, 182, 183, 188, 185, 190, 185, 190, - 185, 190, 185, 190, 185, 190, 3, 6, 220, 187, - 187, 78, 230, 224, 225, 254, 254, 71, 228, 217, - 3, 123, 125, 198, 199, 200, 205, 57, 186, 304, - 187, 190, 186, 252, 239, 254, 214, 186, 186, 67, - 187, 251, 186, 75, 219, 254, 254, 270, 85, 87, - 89, 4, 186, 186, 186, 186, 4, 4, 191, 187, - 187, 79, 253, 3, 254, 254, 77, 152, 186, 75, - 126, 255, 255, 255, 255, 255, 255, 255, 255, 255, - 255, 255, 255, 255, 255, 3, 181, 275, 6, 285, - 6, 286, 6, 5, 46, 48, 49, 187, 186, 236, - 237, 238, 239, 244, 169, 231, 190, 72, 73, 226, - 254, 90, 91, 92, 93, 94, 95, 96, 97, 98, - 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 113, 116, 117, 120, 121, 122, 201, 124, - 186, 187, 190, 217, 206, 186, 3, 251, 190, 81, - 82, 83, 293, 294, 293, 251, 187, 219, 187, 57, - 88, 85, 87, 254, 254, 78, 254, 4, 3, 273, - 254, 187, 190, 187, 190, 5, 6, 295, 186, 255, - 219, 251, 126, 149, 191, 191, 6, 6, 217, 190, - 58, 60, 61, 62, 63, 65, 66, 245, 3, 57, - 240, 257, 258, 259, 260, 261, 262, 263, 264, 232, - 225, 186, 186, 186, 186, 186, 186, 75, 80, 123, - 125, 126, 202, 203, 300, 186, 206, 28, 301, 199, - 187, 206, 187, 186, 4, 3, 187, 190, 187, 187, - 187, 201, 254, 254, 85, 88, 255, 190, 190, 190, - 190, 4, 4, 80, 219, 251, 187, 187, 255, 53, - 50, 187, 237, 59, 59, 3, 190, 54, 234, 6, - 91, 92, 93, 94, 95, 98, 99, 118, 91, 92, - 93, 94, 95, 98, 99, 118, 91, 92, 93, 94, - 95, 98, 99, 118, 91, 92, 93, 94, 95, 98, - 99, 118, 91, 92, 93, 94, 95, 98, 99, 118, - 126, 186, 124, 128, 203, 204, 204, 206, 187, 186, + 155, 3, 3, 239, 3, 242, 243, 163, 218, 219, + 0, 189, 298, 20, 22, 24, 235, 8, 220, 222, + 74, 297, 297, 297, 297, 297, 299, 239, 74, 296, + 296, 296, 296, 296, 188, 14, 239, 78, 79, 186, + 3, 3, 3, 196, 197, 207, 208, 212, 215, 216, + 217, 246, 247, 248, 249, 250, 3, 239, 6, 167, + 168, 167, 168, 3, 159, 239, 64, 57, 190, 6, + 187, 187, 195, 21, 186, 221, 222, 67, 229, 69, + 223, 75, 3, 239, 239, 239, 3, 64, 186, 209, + 76, 3, 239, 239, 239, 3, 3, 3, 213, 214, + 68, 232, 4, 295, 295, 3, 4, 5, 6, 75, + 76, 84, 86, 102, 103, 104, 105, 129, 130, 131, + 151, 170, 172, 174, 179, 180, 181, 184, 186, 251, + 253, 254, 255, 257, 258, 259, 260, 261, 262, 265, + 266, 267, 268, 269, 271, 272, 273, 274, 275, 277, + 278, 279, 280, 281, 282, 283, 284, 287, 288, 289, + 290, 291, 292, 3, 5, 6, 64, 161, 3, 5, + 6, 64, 161, 3, 5, 6, 64, 161, 42, 46, + 47, 51, 52, 3, 3, 6, 239, 186, 243, 295, + 221, 222, 251, 55, 70, 227, 76, 57, 186, 209, + 239, 3, 206, 34, 219, 64, 176, 190, 232, 254, + 80, 80, 186, 132, 133, 134, 135, 136, 137, 138, + 139, 140, 141, 142, 143, 75, 76, 255, 186, 186, + 89, 254, 270, 4, 4, 4, 4, 6, 292, 186, + 117, 119, 120, 121, 186, 186, 255, 255, 5, 6, + 185, 275, 285, 286, 219, 254, 187, 190, 57, 149, + 150, 75, 77, 127, 144, 145, 146, 147, 148, 152, + 176, 177, 178, 179, 180, 181, 182, 183, 188, 185, + 190, 185, 190, 185, 190, 185, 190, 185, 190, 3, + 6, 80, 300, 220, 187, 187, 78, 230, 224, 225, + 254, 254, 71, 228, 217, 3, 123, 125, 198, 199, + 200, 205, 57, 186, 304, 187, 190, 186, 252, 239, + 254, 214, 186, 186, 67, 187, 251, 186, 75, 219, + 254, 254, 270, 85, 87, 89, 4, 186, 186, 186, + 186, 4, 4, 191, 187, 187, 79, 253, 3, 254, + 254, 77, 152, 186, 75, 126, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 3, 181, 275, 6, 285, 6, 286, 6, 5, 46, + 48, 49, 186, 187, 186, 236, 237, 238, 239, 244, + 169, 231, 190, 72, 73, 226, 254, 90, 91, 92, + 93, 94, 95, 96, 97, 98, 99, 100, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 113, 116, + 117, 120, 121, 122, 201, 124, 186, 187, 190, 217, + 206, 186, 3, 251, 190, 81, 82, 83, 293, 294, + 293, 251, 187, 219, 187, 57, 88, 85, 87, 254, + 254, 78, 254, 4, 3, 273, 254, 187, 190, 187, + 190, 5, 6, 295, 186, 255, 219, 251, 126, 149, + 191, 191, 6, 6, 3, 302, 303, 217, 190, 58, + 60, 61, 62, 63, 65, 66, 245, 3, 57, 240, + 257, 258, 259, 260, 261, 262, 263, 264, 232, 225, + 186, 186, 186, 186, 186, 186, 75, 123, 125, 126, + 202, 203, 300, 186, 206, 28, 301, 199, 187, 206, + 187, 186, 4, 3, 187, 190, 187, 187, 187, 201, + 254, 254, 85, 88, 255, 190, 190, 190, 190, 4, + 4, 80, 219, 251, 187, 187, 255, 53, 50, 176, + 187, 190, 187, 237, 59, 59, 3, 190, 54, 234, + 6, 91, 92, 93, 94, 95, 98, 99, 118, 91, + 92, 93, 94, 95, 98, 99, 118, 91, 92, 93, + 94, 95, 98, 99, 118, 91, 92, 93, 94, 95, + 98, 99, 118, 91, 92, 93, 94, 95, 98, 99, + 118, 126, 124, 128, 203, 204, 204, 206, 187, 186, 158, 187, 251, 294, 187, 85, 254, 187, 184, 287, 4, 275, 184, 276, 279, 284, 287, 187, 187, 186, - 187, 187, 6, 6, 240, 238, 238, 186, 263, 55, - 56, 233, 187, 190, 190, 190, 190, 190, 190, 190, + 187, 187, 6, 6, 3, 5, 6, 303, 240, 238, + 238, 186, 263, 55, 56, 233, 187, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, - 190, 190, 190, 190, 3, 302, 303, 274, 187, 302, - 3, 158, 187, 6, 190, 187, 190, 190, 190, 293, - 64, 206, 251, 254, 6, 6, 6, 6, 6, 6, + 190, 190, 190, 190, 190, 190, 190, 190, 274, 187, + 302, 3, 158, 187, 6, 190, 187, 190, 190, 190, + 293, 64, 206, 251, 254, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, - 6, 6, 6, 6, 6, 176, 187, 190, 187, 300, - 3, 4, 4, 4, 4, 187, 254, 187, 187, 187, + 6, 6, 6, 6, 6, 6, 187, 300, 3, 4, + 4, 4, 4, 187, 254, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, - 187, 187, 187, 187, 187, 187, 187, 187, 187, 3, - 5, 6, 303, 300, 190, 187, 190, 187, 190, 4, - 4, 300, 6, 187, 190, 190, 256, 187, 300, 6, - 4, 187, 300, 187, 300 + 187, 187, 187, 187, 187, 187, 187, 300, 190, 187, + 190, 187, 190, 4, 4, 300, 6, 187, 190, 190, + 256, 187, 300, 6, 4, 187, 300, 187, 300 }; /* YYR1[RULE-NUM] -- Symbol kind of the left-hand side of rule RULE-NUM. */ @@ -1560,25 +1552,25 @@ static const yytype_int16 yyr1[] = 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 247, 247, 247, 248, - 249, 249, 249, 249, 249, 249, 249, 249, 249, 249, - 249, 249, 249, 249, 249, 249, 249, 250, 251, 251, - 252, 252, 253, 253, 254, 254, 254, 254, 254, 255, + 248, 249, 249, 249, 249, 249, 249, 249, 249, 249, + 249, 249, 249, 249, 249, 249, 249, 249, 250, 251, + 251, 252, 252, 253, 253, 254, 254, 254, 254, 254, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, - 255, 255, 256, 256, 257, 258, 258, 259, 259, 260, - 260, 261, 261, 262, 262, 263, 263, 263, 263, 263, - 263, 264, 264, 265, 265, 265, 265, 265, 265, 265, + 255, 255, 255, 256, 256, 257, 258, 258, 259, 259, + 260, 260, 261, 261, 262, 262, 263, 263, 263, 263, + 263, 263, 264, 264, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, - 265, 265, 265, 265, 265, 265, 266, 266, 267, 268, - 268, 269, 269, 269, 269, 270, 270, 271, 272, 272, - 272, 272, 273, 273, 273, 273, 274, 274, 274, 274, - 274, 274, 274, 274, 274, 274, 274, 274, 275, 275, - 275, 275, 276, 276, 276, 277, 278, 278, 279, 279, - 280, 281, 281, 282, 283, 283, 284, 285, 286, 287, - 287, 288, 289, 289, 290, 291, 291, 292, 292, 292, - 292, 292, 292, 292, 292, 292, 292, 292, 292, 293, - 293, 294, 294, 294, 295, 296, 296, 297, 297, 298, - 298, 299, 299, 300, 300, 301, 301, 302, 302, 303, - 303, 303, 303, 304, 304, 304 + 265, 265, 265, 265, 265, 265, 265, 266, 266, 267, + 268, 268, 269, 269, 269, 269, 270, 270, 271, 272, + 272, 272, 272, 273, 273, 273, 273, 274, 274, 274, + 274, 274, 274, 274, 274, 274, 274, 274, 274, 275, + 275, 275, 275, 276, 276, 276, 277, 278, 278, 279, + 279, 280, 281, 281, 282, 283, 283, 284, 285, 286, + 287, 287, 288, 289, 289, 290, 291, 291, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 293, 293, 294, 294, 294, 295, 296, 296, 297, 297, + 298, 298, 299, 299, 300, 300, 301, 301, 302, 302, + 303, 303, 303, 303, 304, 304, 304 }; /* YYR2[RULE-NUM] -- Number of symbols on the right-hand side of rule RULE-NUM. */ @@ -1608,25 +1600,25 @@ static const yytype_int8 yyr2[] = 2, 2, 2, 3, 2, 2, 2, 4, 2, 3, 3, 3, 4, 4, 3, 3, 4, 4, 5, 6, 7, 9, 4, 5, 7, 9, 2, 2, 2, 2, - 2, 4, 4, 4, 4, 4, 4, 4, 4, 4, - 4, 4, 4, 4, 4, 4, 4, 3, 1, 3, - 3, 5, 3, 1, 1, 1, 1, 1, 1, 3, - 3, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 2, 0, 12, 14, 12, 12, 10, 7, - 9, 4, 6, 4, 6, 1, 1, 1, 1, 1, - 1, 1, 3, 3, 4, 5, 4, 3, 2, 2, - 2, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 6, 3, 4, 3, 3, 5, 5, - 6, 4, 6, 3, 5, 4, 5, 6, 4, 5, - 5, 6, 1, 3, 1, 3, 1, 1, 1, 1, - 1, 2, 2, 2, 2, 2, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 2, 2, 3, 1, 1, - 2, 2, 3, 2, 2, 3, 2, 3, 3, 1, - 1, 2, 2, 3, 2, 2, 3, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 2, 1, - 3, 2, 2, 1, 1, 2, 0, 3, 0, 1, - 0, 2, 0, 4, 0, 4, 0, 1, 3, 1, - 3, 3, 3, 6, 7, 3 + 5, 2, 4, 4, 4, 4, 4, 4, 4, 4, + 4, 4, 4, 4, 4, 4, 4, 4, 3, 1, + 3, 3, 5, 3, 1, 1, 1, 1, 1, 1, + 3, 3, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 2, 0, 12, 14, 12, 12, 10, + 7, 9, 4, 6, 4, 6, 1, 1, 1, 1, + 1, 1, 1, 3, 3, 4, 5, 4, 3, 2, + 2, 2, 3, 3, 3, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 6, 3, 4, 3, 3, 5, + 5, 6, 4, 6, 3, 5, 4, 5, 6, 4, + 5, 5, 6, 1, 3, 1, 3, 1, 1, 1, + 1, 1, 2, 2, 2, 2, 2, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 2, 2, 3, 1, + 1, 2, 2, 3, 2, 2, 3, 2, 3, 3, + 1, 1, 2, 2, 3, 2, 2, 3, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 1, 3, 2, 2, 1, 1, 2, 0, 3, 0, + 1, 0, 2, 0, 4, 0, 4, 0, 1, 3, + 1, 3, 3, 3, 6, 7, 3 }; @@ -2194,7 +2186,7 @@ yydestruct (const char *yymsg, { free(((*yyvaluep).str_value)); } -#line 2198 "parser.cpp" +#line 2190 "parser.cpp" break; case YYSYMBOL_STRING: /* STRING */ @@ -2202,7 +2194,7 @@ yydestruct (const char *yymsg, { free(((*yyvaluep).str_value)); } -#line 2206 "parser.cpp" +#line 2198 "parser.cpp" break; case YYSYMBOL_statement_list: /* statement_list */ @@ -2216,7 +2208,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).stmt_array)); } } -#line 2220 "parser.cpp" +#line 2212 "parser.cpp" break; case YYSYMBOL_table_element_array: /* table_element_array */ @@ -2230,7 +2222,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).table_element_array_t)); } } -#line 2234 "parser.cpp" +#line 2226 "parser.cpp" break; case YYSYMBOL_column_constraints: /* column_constraints */ @@ -2241,7 +2233,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).column_constraints_t)); } } -#line 2245 "parser.cpp" +#line 2237 "parser.cpp" break; case YYSYMBOL_default_expr: /* default_expr */ @@ -2249,7 +2241,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2253 "parser.cpp" +#line 2245 "parser.cpp" break; case YYSYMBOL_identifier_array: /* identifier_array */ @@ -2258,7 +2250,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy identifier array\n"); delete (((*yyvaluep).identifier_array_t)); } -#line 2262 "parser.cpp" +#line 2254 "parser.cpp" break; case YYSYMBOL_optional_identifier_array: /* optional_identifier_array */ @@ -2267,7 +2259,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy identifier array\n"); delete (((*yyvaluep).identifier_array_t)); } -#line 2271 "parser.cpp" +#line 2263 "parser.cpp" break; case YYSYMBOL_update_expr_array: /* update_expr_array */ @@ -2281,7 +2273,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).update_expr_array_t)); } } -#line 2285 "parser.cpp" +#line 2277 "parser.cpp" break; case YYSYMBOL_update_expr: /* update_expr */ @@ -2292,7 +2284,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).update_expr_t); } } -#line 2296 "parser.cpp" +#line 2288 "parser.cpp" break; case YYSYMBOL_select_statement: /* select_statement */ @@ -2302,7 +2294,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).select_stmt); } } -#line 2306 "parser.cpp" +#line 2298 "parser.cpp" break; case YYSYMBOL_select_with_paren: /* select_with_paren */ @@ -2312,7 +2304,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).select_stmt); } } -#line 2316 "parser.cpp" +#line 2308 "parser.cpp" break; case YYSYMBOL_select_without_paren: /* select_without_paren */ @@ -2322,7 +2314,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).select_stmt); } } -#line 2326 "parser.cpp" +#line 2318 "parser.cpp" break; case YYSYMBOL_select_clause_with_modifier: /* select_clause_with_modifier */ @@ -2332,7 +2324,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).select_stmt); } } -#line 2336 "parser.cpp" +#line 2328 "parser.cpp" break; case YYSYMBOL_select_clause_without_modifier_paren: /* select_clause_without_modifier_paren */ @@ -2342,7 +2334,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).select_stmt); } } -#line 2346 "parser.cpp" +#line 2338 "parser.cpp" break; case YYSYMBOL_select_clause_without_modifier: /* select_clause_without_modifier */ @@ -2352,7 +2344,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).select_stmt); } } -#line 2356 "parser.cpp" +#line 2348 "parser.cpp" break; case YYSYMBOL_order_by_clause: /* order_by_clause */ @@ -2366,7 +2358,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).order_by_expr_list_t)); } } -#line 2370 "parser.cpp" +#line 2362 "parser.cpp" break; case YYSYMBOL_order_by_expr_list: /* order_by_expr_list */ @@ -2380,7 +2372,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).order_by_expr_list_t)); } } -#line 2384 "parser.cpp" +#line 2376 "parser.cpp" break; case YYSYMBOL_order_by_expr: /* order_by_expr */ @@ -2390,7 +2382,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).order_by_expr_t)->expr_; delete ((*yyvaluep).order_by_expr_t); } -#line 2394 "parser.cpp" +#line 2386 "parser.cpp" break; case YYSYMBOL_limit_expr: /* limit_expr */ @@ -2398,7 +2390,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2402 "parser.cpp" +#line 2394 "parser.cpp" break; case YYSYMBOL_offset_expr: /* offset_expr */ @@ -2406,7 +2398,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2410 "parser.cpp" +#line 2402 "parser.cpp" break; case YYSYMBOL_from_clause: /* from_clause */ @@ -2415,7 +2407,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2419 "parser.cpp" +#line 2411 "parser.cpp" break; case YYSYMBOL_search_clause: /* search_clause */ @@ -2423,7 +2415,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2427 "parser.cpp" +#line 2419 "parser.cpp" break; case YYSYMBOL_where_clause: /* where_clause */ @@ -2431,7 +2423,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2435 "parser.cpp" +#line 2427 "parser.cpp" break; case YYSYMBOL_having_clause: /* having_clause */ @@ -2439,7 +2431,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2443 "parser.cpp" +#line 2435 "parser.cpp" break; case YYSYMBOL_group_by_clause: /* group_by_clause */ @@ -2453,7 +2445,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_t)); } } -#line 2457 "parser.cpp" +#line 2449 "parser.cpp" break; case YYSYMBOL_table_reference: /* table_reference */ @@ -2462,7 +2454,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2466 "parser.cpp" +#line 2458 "parser.cpp" break; case YYSYMBOL_table_reference_unit: /* table_reference_unit */ @@ -2471,7 +2463,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2475 "parser.cpp" +#line 2467 "parser.cpp" break; case YYSYMBOL_table_reference_name: /* table_reference_name */ @@ -2480,7 +2472,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2484 "parser.cpp" +#line 2476 "parser.cpp" break; case YYSYMBOL_table_name: /* table_name */ @@ -2493,7 +2485,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).table_name_t)); } } -#line 2497 "parser.cpp" +#line 2489 "parser.cpp" break; case YYSYMBOL_table_alias: /* table_alias */ @@ -2502,7 +2494,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy table alias\n"); delete (((*yyvaluep).table_alias_t)); } -#line 2506 "parser.cpp" +#line 2498 "parser.cpp" break; case YYSYMBOL_with_clause: /* with_clause */ @@ -2516,7 +2508,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_expr_list_t)); } } -#line 2520 "parser.cpp" +#line 2512 "parser.cpp" break; case YYSYMBOL_with_expr_list: /* with_expr_list */ @@ -2530,7 +2522,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_expr_list_t)); } } -#line 2534 "parser.cpp" +#line 2526 "parser.cpp" break; case YYSYMBOL_with_expr: /* with_expr */ @@ -2540,7 +2532,7 @@ yydestruct (const char *yymsg, delete ((*yyvaluep).with_expr_t)->select_; delete ((*yyvaluep).with_expr_t); } -#line 2544 "parser.cpp" +#line 2536 "parser.cpp" break; case YYSYMBOL_join_clause: /* join_clause */ @@ -2549,7 +2541,7 @@ yydestruct (const char *yymsg, fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2553 "parser.cpp" +#line 2545 "parser.cpp" break; case YYSYMBOL_expr_array: /* expr_array */ @@ -2563,7 +2555,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_t)); } } -#line 2567 "parser.cpp" +#line 2559 "parser.cpp" break; case YYSYMBOL_expr_array_list: /* expr_array_list */ @@ -2580,7 +2572,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_list_t)); } } -#line 2584 "parser.cpp" +#line 2576 "parser.cpp" break; case YYSYMBOL_expr_alias: /* expr_alias */ @@ -2588,7 +2580,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2592 "parser.cpp" +#line 2584 "parser.cpp" break; case YYSYMBOL_expr: /* expr */ @@ -2596,7 +2588,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2600 "parser.cpp" +#line 2592 "parser.cpp" break; case YYSYMBOL_operand: /* operand */ @@ -2604,7 +2596,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2608 "parser.cpp" +#line 2600 "parser.cpp" break; case YYSYMBOL_extra_match_tensor_option: /* extra_match_tensor_option */ @@ -2612,7 +2604,7 @@ yydestruct (const char *yymsg, { free(((*yyvaluep).str_value)); } -#line 2616 "parser.cpp" +#line 2608 "parser.cpp" break; case YYSYMBOL_match_tensor_expr: /* match_tensor_expr */ @@ -2620,7 +2612,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2624 "parser.cpp" +#line 2616 "parser.cpp" break; case YYSYMBOL_match_vector_expr: /* match_vector_expr */ @@ -2628,7 +2620,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2632 "parser.cpp" +#line 2624 "parser.cpp" break; case YYSYMBOL_match_sparse_expr: /* match_sparse_expr */ @@ -2636,7 +2628,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2640 "parser.cpp" +#line 2632 "parser.cpp" break; case YYSYMBOL_match_text_expr: /* match_text_expr */ @@ -2644,7 +2636,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2648 "parser.cpp" +#line 2640 "parser.cpp" break; case YYSYMBOL_query_expr: /* query_expr */ @@ -2652,7 +2644,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2656 "parser.cpp" +#line 2648 "parser.cpp" break; case YYSYMBOL_fusion_expr: /* fusion_expr */ @@ -2660,7 +2652,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2664 "parser.cpp" +#line 2656 "parser.cpp" break; case YYSYMBOL_sub_search: /* sub_search */ @@ -2668,7 +2660,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2672 "parser.cpp" +#line 2664 "parser.cpp" break; case YYSYMBOL_sub_search_array: /* sub_search_array */ @@ -2682,7 +2674,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_t)); } } -#line 2686 "parser.cpp" +#line 2678 "parser.cpp" break; case YYSYMBOL_function_expr: /* function_expr */ @@ -2690,7 +2682,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2694 "parser.cpp" +#line 2686 "parser.cpp" break; case YYSYMBOL_conjunction_expr: /* conjunction_expr */ @@ -2698,7 +2690,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2702 "parser.cpp" +#line 2694 "parser.cpp" break; case YYSYMBOL_between_expr: /* between_expr */ @@ -2706,7 +2698,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2710 "parser.cpp" +#line 2702 "parser.cpp" break; case YYSYMBOL_in_expr: /* in_expr */ @@ -2714,7 +2706,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2718 "parser.cpp" +#line 2710 "parser.cpp" break; case YYSYMBOL_case_expr: /* case_expr */ @@ -2722,7 +2714,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2726 "parser.cpp" +#line 2718 "parser.cpp" break; case YYSYMBOL_case_check_array: /* case_check_array */ @@ -2735,7 +2727,7 @@ yydestruct (const char *yymsg, } } } -#line 2739 "parser.cpp" +#line 2731 "parser.cpp" break; case YYSYMBOL_cast_expr: /* cast_expr */ @@ -2743,7 +2735,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2747 "parser.cpp" +#line 2739 "parser.cpp" break; case YYSYMBOL_subquery_expr: /* subquery_expr */ @@ -2751,7 +2743,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2755 "parser.cpp" +#line 2747 "parser.cpp" break; case YYSYMBOL_column_expr: /* column_expr */ @@ -2759,7 +2751,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).expr_t)); } -#line 2763 "parser.cpp" +#line 2755 "parser.cpp" break; case YYSYMBOL_constant_expr: /* constant_expr */ @@ -2767,7 +2759,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2771 "parser.cpp" +#line 2763 "parser.cpp" break; case YYSYMBOL_common_array_expr: /* common_array_expr */ @@ -2775,7 +2767,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2779 "parser.cpp" +#line 2771 "parser.cpp" break; case YYSYMBOL_common_sparse_array_expr: /* common_sparse_array_expr */ @@ -2783,7 +2775,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2787 "parser.cpp" +#line 2779 "parser.cpp" break; case YYSYMBOL_subarray_array_expr: /* subarray_array_expr */ @@ -2791,7 +2783,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2795 "parser.cpp" +#line 2787 "parser.cpp" break; case YYSYMBOL_unclosed_subarray_array_expr: /* unclosed_subarray_array_expr */ @@ -2799,7 +2791,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2803 "parser.cpp" +#line 2795 "parser.cpp" break; case YYSYMBOL_sparse_array_expr: /* sparse_array_expr */ @@ -2807,7 +2799,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2811 "parser.cpp" +#line 2803 "parser.cpp" break; case YYSYMBOL_long_sparse_array_expr: /* long_sparse_array_expr */ @@ -2815,7 +2807,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2819 "parser.cpp" +#line 2811 "parser.cpp" break; case YYSYMBOL_unclosed_long_sparse_array_expr: /* unclosed_long_sparse_array_expr */ @@ -2823,7 +2815,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2827 "parser.cpp" +#line 2819 "parser.cpp" break; case YYSYMBOL_double_sparse_array_expr: /* double_sparse_array_expr */ @@ -2831,7 +2823,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2835 "parser.cpp" +#line 2827 "parser.cpp" break; case YYSYMBOL_unclosed_double_sparse_array_expr: /* unclosed_double_sparse_array_expr */ @@ -2839,7 +2831,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2843 "parser.cpp" +#line 2835 "parser.cpp" break; case YYSYMBOL_empty_array_expr: /* empty_array_expr */ @@ -2847,7 +2839,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2851 "parser.cpp" +#line 2843 "parser.cpp" break; case YYSYMBOL_int_sparse_ele: /* int_sparse_ele */ @@ -2855,7 +2847,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).int_sparse_ele_t)); } -#line 2859 "parser.cpp" +#line 2851 "parser.cpp" break; case YYSYMBOL_float_sparse_ele: /* float_sparse_ele */ @@ -2863,7 +2855,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).float_sparse_ele_t)); } -#line 2867 "parser.cpp" +#line 2859 "parser.cpp" break; case YYSYMBOL_array_expr: /* array_expr */ @@ -2871,7 +2863,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2875 "parser.cpp" +#line 2867 "parser.cpp" break; case YYSYMBOL_long_array_expr: /* long_array_expr */ @@ -2879,7 +2871,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2883 "parser.cpp" +#line 2875 "parser.cpp" break; case YYSYMBOL_unclosed_long_array_expr: /* unclosed_long_array_expr */ @@ -2887,7 +2879,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2891 "parser.cpp" +#line 2883 "parser.cpp" break; case YYSYMBOL_double_array_expr: /* double_array_expr */ @@ -2895,7 +2887,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2899 "parser.cpp" +#line 2891 "parser.cpp" break; case YYSYMBOL_unclosed_double_array_expr: /* unclosed_double_array_expr */ @@ -2903,7 +2895,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2907 "parser.cpp" +#line 2899 "parser.cpp" break; case YYSYMBOL_interval_expr: /* interval_expr */ @@ -2911,7 +2903,7 @@ yydestruct (const char *yymsg, { delete (((*yyvaluep).const_expr_t)); } -#line 2915 "parser.cpp" +#line 2907 "parser.cpp" break; case YYSYMBOL_file_path: /* file_path */ @@ -2919,7 +2911,7 @@ yydestruct (const char *yymsg, { free(((*yyvaluep).str_value)); } -#line 2923 "parser.cpp" +#line 2915 "parser.cpp" break; case YYSYMBOL_if_not_exists_info: /* if_not_exists_info */ @@ -2930,7 +2922,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).if_not_exists_info_t)); } } -#line 2934 "parser.cpp" +#line 2926 "parser.cpp" break; case YYSYMBOL_with_index_param_list: /* with_index_param_list */ @@ -2944,7 +2936,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_index_param_list_t)); } } -#line 2948 "parser.cpp" +#line 2940 "parser.cpp" break; case YYSYMBOL_optional_table_properties_list: /* optional_table_properties_list */ @@ -2958,7 +2950,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_index_param_list_t)); } } -#line 2962 "parser.cpp" +#line 2954 "parser.cpp" break; case YYSYMBOL_index_info_list: /* index_info_list */ @@ -2972,7 +2964,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).index_info_list_t)); } } -#line 2976 "parser.cpp" +#line 2968 "parser.cpp" break; default: @@ -3080,7 +3072,7 @@ YYLTYPE yylloc = yyloc_default; yylloc.string_length = 0; } -#line 3084 "parser.cpp" +#line 3076 "parser.cpp" yylsp[0] = yylloc; goto yysetstate; @@ -3295,7 +3287,7 @@ YYLTYPE yylloc = yyloc_default; { result->statements_ptr_ = (yyvsp[-1].stmt_array); } -#line 3299 "parser.cpp" +#line 3291 "parser.cpp" break; case 3: /* statement_list: statement */ @@ -3306,7 +3298,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.stmt_array) = new std::vector(); (yyval.stmt_array)->push_back((yyvsp[0].base_stmt)); } -#line 3310 "parser.cpp" +#line 3302 "parser.cpp" break; case 4: /* statement_list: statement_list ';' statement */ @@ -3317,157 +3309,157 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-2].stmt_array)->push_back((yyvsp[0].base_stmt)); (yyval.stmt_array) = (yyvsp[-2].stmt_array); } -#line 3321 "parser.cpp" +#line 3313 "parser.cpp" break; case 5: /* statement: create_statement */ #line 509 "parser.y" { (yyval.base_stmt) = (yyvsp[0].create_stmt); } -#line 3327 "parser.cpp" +#line 3319 "parser.cpp" break; case 6: /* statement: drop_statement */ #line 510 "parser.y" { (yyval.base_stmt) = (yyvsp[0].drop_stmt); } -#line 3333 "parser.cpp" +#line 3325 "parser.cpp" break; case 7: /* statement: copy_statement */ #line 511 "parser.y" { (yyval.base_stmt) = (yyvsp[0].copy_stmt); } -#line 3339 "parser.cpp" +#line 3331 "parser.cpp" break; case 8: /* statement: show_statement */ #line 512 "parser.y" { (yyval.base_stmt) = (yyvsp[0].show_stmt); } -#line 3345 "parser.cpp" +#line 3337 "parser.cpp" break; case 9: /* statement: select_statement */ #line 513 "parser.y" { (yyval.base_stmt) = (yyvsp[0].select_stmt); } -#line 3351 "parser.cpp" +#line 3343 "parser.cpp" break; case 10: /* statement: delete_statement */ #line 514 "parser.y" { (yyval.base_stmt) = (yyvsp[0].delete_stmt); } -#line 3357 "parser.cpp" +#line 3349 "parser.cpp" break; case 11: /* statement: update_statement */ #line 515 "parser.y" { (yyval.base_stmt) = (yyvsp[0].update_stmt); } -#line 3363 "parser.cpp" +#line 3355 "parser.cpp" break; case 12: /* statement: insert_statement */ #line 516 "parser.y" { (yyval.base_stmt) = (yyvsp[0].insert_stmt); } -#line 3369 "parser.cpp" +#line 3361 "parser.cpp" break; case 13: /* statement: explain_statement */ #line 517 "parser.y" { (yyval.base_stmt) = (yyvsp[0].explain_stmt); } -#line 3375 "parser.cpp" +#line 3367 "parser.cpp" break; case 14: /* statement: flush_statement */ #line 518 "parser.y" { (yyval.base_stmt) = (yyvsp[0].flush_stmt); } -#line 3381 "parser.cpp" +#line 3373 "parser.cpp" break; case 15: /* statement: optimize_statement */ #line 519 "parser.y" { (yyval.base_stmt) = (yyvsp[0].optimize_stmt); } -#line 3387 "parser.cpp" +#line 3379 "parser.cpp" break; case 16: /* statement: command_statement */ #line 520 "parser.y" { (yyval.base_stmt) = (yyvsp[0].command_stmt); } -#line 3393 "parser.cpp" +#line 3385 "parser.cpp" break; case 17: /* statement: compact_statement */ #line 521 "parser.y" { (yyval.base_stmt) = (yyvsp[0].compact_stmt); } -#line 3399 "parser.cpp" +#line 3391 "parser.cpp" break; case 18: /* explainable_statement: create_statement */ #line 523 "parser.y" { (yyval.base_stmt) = (yyvsp[0].create_stmt); } -#line 3405 "parser.cpp" +#line 3397 "parser.cpp" break; case 19: /* explainable_statement: drop_statement */ #line 524 "parser.y" { (yyval.base_stmt) = (yyvsp[0].drop_stmt); } -#line 3411 "parser.cpp" +#line 3403 "parser.cpp" break; case 20: /* explainable_statement: copy_statement */ #line 525 "parser.y" { (yyval.base_stmt) = (yyvsp[0].copy_stmt); } -#line 3417 "parser.cpp" +#line 3409 "parser.cpp" break; case 21: /* explainable_statement: show_statement */ #line 526 "parser.y" { (yyval.base_stmt) = (yyvsp[0].show_stmt); } -#line 3423 "parser.cpp" +#line 3415 "parser.cpp" break; case 22: /* explainable_statement: select_statement */ #line 527 "parser.y" { (yyval.base_stmt) = (yyvsp[0].select_stmt); } -#line 3429 "parser.cpp" +#line 3421 "parser.cpp" break; case 23: /* explainable_statement: delete_statement */ #line 528 "parser.y" { (yyval.base_stmt) = (yyvsp[0].delete_stmt); } -#line 3435 "parser.cpp" +#line 3427 "parser.cpp" break; case 24: /* explainable_statement: update_statement */ #line 529 "parser.y" { (yyval.base_stmt) = (yyvsp[0].update_stmt); } -#line 3441 "parser.cpp" +#line 3433 "parser.cpp" break; case 25: /* explainable_statement: insert_statement */ #line 530 "parser.y" { (yyval.base_stmt) = (yyvsp[0].insert_stmt); } -#line 3447 "parser.cpp" +#line 3439 "parser.cpp" break; case 26: /* explainable_statement: flush_statement */ #line 531 "parser.y" { (yyval.base_stmt) = (yyvsp[0].flush_stmt); } -#line 3453 "parser.cpp" +#line 3445 "parser.cpp" break; case 27: /* explainable_statement: optimize_statement */ #line 532 "parser.y" { (yyval.base_stmt) = (yyvsp[0].optimize_stmt); } -#line 3459 "parser.cpp" +#line 3451 "parser.cpp" break; case 28: /* explainable_statement: command_statement */ #line 533 "parser.y" { (yyval.base_stmt) = (yyvsp[0].command_stmt); } -#line 3465 "parser.cpp" +#line 3457 "parser.cpp" break; case 29: /* explainable_statement: compact_statement */ #line 534 "parser.y" { (yyval.base_stmt) = (yyvsp[0].compact_stmt); } -#line 3471 "parser.cpp" +#line 3463 "parser.cpp" break; case 30: /* create_statement: CREATE DATABASE if_not_exists IDENTIFIER */ @@ -3487,7 +3479,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.create_stmt)->create_info_ = create_schema_info; (yyval.create_stmt)->create_info_->conflict_type_ = (yyvsp[-1].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; } -#line 3491 "parser.cpp" +#line 3483 "parser.cpp" break; case 31: /* create_statement: CREATE COLLECTION if_not_exists table_name */ @@ -3505,7 +3497,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.create_stmt)->create_info_->conflict_type_ = (yyvsp[-1].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; delete (yyvsp[0].table_name_t); } -#line 3509 "parser.cpp" +#line 3501 "parser.cpp" break; case 32: /* create_statement: CREATE TABLE if_not_exists table_name '(' table_element_array ')' optional_table_properties_list */ @@ -3538,7 +3530,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.create_stmt)->create_info_ = create_table_info; (yyval.create_stmt)->create_info_->conflict_type_ = (yyvsp[-5].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; } -#line 3542 "parser.cpp" +#line 3534 "parser.cpp" break; case 33: /* create_statement: CREATE TABLE if_not_exists table_name AS select_statement */ @@ -3558,7 +3550,7 @@ YYLTYPE yylloc = yyloc_default; create_table_info->select_ = (yyvsp[0].select_stmt); (yyval.create_stmt)->create_info_ = create_table_info; } -#line 3562 "parser.cpp" +#line 3554 "parser.cpp" break; case 34: /* create_statement: CREATE VIEW if_not_exists table_name optional_identifier_array AS select_statement */ @@ -3579,7 +3571,7 @@ YYLTYPE yylloc = yyloc_default; create_view_info->conflict_type_ = (yyvsp[-4].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; (yyval.create_stmt)->create_info_ = create_view_info; } -#line 3583 "parser.cpp" +#line 3575 "parser.cpp" break; case 35: /* create_statement: CREATE INDEX if_not_exists_info ON table_name index_info_list */ @@ -3612,7 +3604,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.create_stmt) = new infinity::CreateStatement(); (yyval.create_stmt)->create_info_ = create_index_info; } -#line 3616 "parser.cpp" +#line 3608 "parser.cpp" break; case 36: /* table_element_array: table_element */ @@ -3621,7 +3613,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_element_array_t) = new std::vector(); (yyval.table_element_array_t)->push_back((yyvsp[0].table_element_t)); } -#line 3625 "parser.cpp" +#line 3617 "parser.cpp" break; case 37: /* table_element_array: table_element_array ',' table_element */ @@ -3630,7 +3622,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-2].table_element_array_t)->push_back((yyvsp[0].table_element_t)); (yyval.table_element_array_t) = (yyvsp[-2].table_element_array_t); } -#line 3634 "parser.cpp" +#line 3626 "parser.cpp" break; case 38: /* table_element: table_column */ @@ -3638,7 +3630,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.table_element_t) = (yyvsp[0].table_column_t); } -#line 3642 "parser.cpp" +#line 3634 "parser.cpp" break; case 39: /* table_element: table_constraint */ @@ -3646,7 +3638,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.table_element_t) = (yyvsp[0].table_constraint_t); } -#line 3650 "parser.cpp" +#line 3642 "parser.cpp" break; case 40: /* table_column: IDENTIFIER column_type with_index_param_list default_expr */ @@ -3701,7 +3693,7 @@ YYLTYPE yylloc = yyloc_default; } */ } -#line 3705 "parser.cpp" +#line 3697 "parser.cpp" break; case 41: /* table_column: IDENTIFIER column_type column_constraints default_expr */ @@ -3740,391 +3732,391 @@ YYLTYPE yylloc = yyloc_default; } */ } -#line 3744 "parser.cpp" +#line 3736 "parser.cpp" break; case 42: /* column_type: BOOLEAN */ #line 772 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kBoolean, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3750 "parser.cpp" +#line 3742 "parser.cpp" break; case 43: /* column_type: TINYINT */ #line 773 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTinyInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3756 "parser.cpp" +#line 3748 "parser.cpp" break; case 44: /* column_type: SMALLINT */ #line 774 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSmallInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3762 "parser.cpp" +#line 3754 "parser.cpp" break; case 45: /* column_type: INTEGER */ #line 775 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kInteger, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3768 "parser.cpp" +#line 3760 "parser.cpp" break; case 46: /* column_type: INT */ #line 776 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kInteger, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3774 "parser.cpp" +#line 3766 "parser.cpp" break; case 47: /* column_type: BIGINT */ #line 777 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kBigInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3780 "parser.cpp" +#line 3772 "parser.cpp" break; case 48: /* column_type: HUGEINT */ #line 778 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kHugeInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3786 "parser.cpp" +#line 3778 "parser.cpp" break; case 49: /* column_type: FLOAT */ #line 779 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kFloat, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3792 "parser.cpp" +#line 3784 "parser.cpp" break; case 50: /* column_type: REAL */ #line 780 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kFloat, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3798 "parser.cpp" +#line 3790 "parser.cpp" break; case 51: /* column_type: DOUBLE */ #line 781 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDouble, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3804 "parser.cpp" +#line 3796 "parser.cpp" break; case 52: /* column_type: DATE */ #line 782 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDate, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3810 "parser.cpp" +#line 3802 "parser.cpp" break; case 53: /* column_type: TIME */ #line 783 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTime, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3816 "parser.cpp" +#line 3808 "parser.cpp" break; case 54: /* column_type: DATETIME */ #line 784 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDateTime, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3822 "parser.cpp" +#line 3814 "parser.cpp" break; case 55: /* column_type: TIMESTAMP */ #line 785 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTimestamp, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3828 "parser.cpp" +#line 3820 "parser.cpp" break; case 56: /* column_type: UUID */ #line 786 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kUuid, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3834 "parser.cpp" +#line 3826 "parser.cpp" break; case 57: /* column_type: POINT */ #line 787 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kPoint, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3840 "parser.cpp" +#line 3832 "parser.cpp" break; case 58: /* column_type: LINE */ #line 788 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kLine, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3846 "parser.cpp" +#line 3838 "parser.cpp" break; case 59: /* column_type: LSEG */ #line 789 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kLineSeg, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3852 "parser.cpp" +#line 3844 "parser.cpp" break; case 60: /* column_type: BOX */ #line 790 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kBox, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3858 "parser.cpp" +#line 3850 "parser.cpp" break; case 61: /* column_type: CIRCLE */ #line 793 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kCircle, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3864 "parser.cpp" +#line 3856 "parser.cpp" break; case 62: /* column_type: VARCHAR */ #line 795 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kVarchar, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3870 "parser.cpp" +#line 3862 "parser.cpp" break; case 63: /* column_type: DECIMAL '(' LONG_VALUE ',' LONG_VALUE ')' */ #line 796 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDecimal, 0, (yyvsp[-3].long_value), (yyvsp[-1].long_value), infinity::EmbeddingDataType::kElemInvalid}; } -#line 3876 "parser.cpp" +#line 3868 "parser.cpp" break; case 64: /* column_type: DECIMAL '(' LONG_VALUE ')' */ #line 797 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDecimal, 0, (yyvsp[-1].long_value), 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3882 "parser.cpp" +#line 3874 "parser.cpp" break; case 65: /* column_type: DECIMAL */ #line 798 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDecimal, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3888 "parser.cpp" +#line 3880 "parser.cpp" break; case 66: /* column_type: EMBEDDING '(' BIT ',' LONG_VALUE ')' */ #line 801 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 3894 "parser.cpp" +#line 3886 "parser.cpp" break; case 67: /* column_type: EMBEDDING '(' TINYINT ',' LONG_VALUE ')' */ #line 802 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 3900 "parser.cpp" +#line 3892 "parser.cpp" break; case 68: /* column_type: EMBEDDING '(' SMALLINT ',' LONG_VALUE ')' */ #line 803 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 3906 "parser.cpp" +#line 3898 "parser.cpp" break; case 69: /* column_type: EMBEDDING '(' INTEGER ',' LONG_VALUE ')' */ #line 804 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3912 "parser.cpp" +#line 3904 "parser.cpp" break; case 70: /* column_type: EMBEDDING '(' INT ',' LONG_VALUE ')' */ #line 805 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3918 "parser.cpp" +#line 3910 "parser.cpp" break; case 71: /* column_type: EMBEDDING '(' BIGINT ',' LONG_VALUE ')' */ #line 806 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 3924 "parser.cpp" +#line 3916 "parser.cpp" break; case 72: /* column_type: EMBEDDING '(' FLOAT ',' LONG_VALUE ')' */ #line 807 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 3930 "parser.cpp" +#line 3922 "parser.cpp" break; case 73: /* column_type: EMBEDDING '(' DOUBLE ',' LONG_VALUE ')' */ #line 808 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 3936 "parser.cpp" +#line 3928 "parser.cpp" break; case 74: /* column_type: TENSOR '(' BIT ',' LONG_VALUE ')' */ #line 809 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 3942 "parser.cpp" +#line 3934 "parser.cpp" break; case 75: /* column_type: TENSOR '(' TINYINT ',' LONG_VALUE ')' */ #line 810 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 3948 "parser.cpp" +#line 3940 "parser.cpp" break; case 76: /* column_type: TENSOR '(' SMALLINT ',' LONG_VALUE ')' */ #line 811 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 3954 "parser.cpp" +#line 3946 "parser.cpp" break; case 77: /* column_type: TENSOR '(' INTEGER ',' LONG_VALUE ')' */ #line 812 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3960 "parser.cpp" +#line 3952 "parser.cpp" break; case 78: /* column_type: TENSOR '(' INT ',' LONG_VALUE ')' */ #line 813 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3966 "parser.cpp" +#line 3958 "parser.cpp" break; case 79: /* column_type: TENSOR '(' BIGINT ',' LONG_VALUE ')' */ #line 814 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 3972 "parser.cpp" +#line 3964 "parser.cpp" break; case 80: /* column_type: TENSOR '(' FLOAT ',' LONG_VALUE ')' */ #line 815 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 3978 "parser.cpp" +#line 3970 "parser.cpp" break; case 81: /* column_type: TENSOR '(' DOUBLE ',' LONG_VALUE ')' */ #line 816 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 3984 "parser.cpp" +#line 3976 "parser.cpp" break; case 82: /* column_type: TENSORARRAY '(' BIT ',' LONG_VALUE ')' */ #line 817 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 3990 "parser.cpp" +#line 3982 "parser.cpp" break; case 83: /* column_type: TENSORARRAY '(' TINYINT ',' LONG_VALUE ')' */ #line 818 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 3996 "parser.cpp" +#line 3988 "parser.cpp" break; case 84: /* column_type: TENSORARRAY '(' SMALLINT ',' LONG_VALUE ')' */ #line 819 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 4002 "parser.cpp" +#line 3994 "parser.cpp" break; case 85: /* column_type: TENSORARRAY '(' INTEGER ',' LONG_VALUE ')' */ #line 820 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 4008 "parser.cpp" +#line 4000 "parser.cpp" break; case 86: /* column_type: TENSORARRAY '(' INT ',' LONG_VALUE ')' */ #line 821 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 4014 "parser.cpp" +#line 4006 "parser.cpp" break; case 87: /* column_type: TENSORARRAY '(' BIGINT ',' LONG_VALUE ')' */ #line 822 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 4020 "parser.cpp" +#line 4012 "parser.cpp" break; case 88: /* column_type: TENSORARRAY '(' FLOAT ',' LONG_VALUE ')' */ #line 823 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 4026 "parser.cpp" +#line 4018 "parser.cpp" break; case 89: /* column_type: TENSORARRAY '(' DOUBLE ',' LONG_VALUE ')' */ #line 824 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensorArray, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 4032 "parser.cpp" +#line 4024 "parser.cpp" break; case 90: /* column_type: VECTOR '(' BIT ',' LONG_VALUE ')' */ #line 825 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 4038 "parser.cpp" +#line 4030 "parser.cpp" break; case 91: /* column_type: VECTOR '(' TINYINT ',' LONG_VALUE ')' */ #line 826 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 4044 "parser.cpp" +#line 4036 "parser.cpp" break; case 92: /* column_type: VECTOR '(' SMALLINT ',' LONG_VALUE ')' */ #line 827 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 4050 "parser.cpp" +#line 4042 "parser.cpp" break; case 93: /* column_type: VECTOR '(' INTEGER ',' LONG_VALUE ')' */ #line 828 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 4056 "parser.cpp" +#line 4048 "parser.cpp" break; case 94: /* column_type: VECTOR '(' INT ',' LONG_VALUE ')' */ #line 829 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 4062 "parser.cpp" +#line 4054 "parser.cpp" break; case 95: /* column_type: VECTOR '(' BIGINT ',' LONG_VALUE ')' */ #line 830 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 4068 "parser.cpp" +#line 4060 "parser.cpp" break; case 96: /* column_type: VECTOR '(' FLOAT ',' LONG_VALUE ')' */ #line 831 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 4074 "parser.cpp" +#line 4066 "parser.cpp" break; case 97: /* column_type: VECTOR '(' DOUBLE ',' LONG_VALUE ')' */ #line 832 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 4080 "parser.cpp" +#line 4072 "parser.cpp" break; case 98: /* column_type: SPARSE '(' BIT ',' LONG_VALUE ')' */ #line 833 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 4086 "parser.cpp" +#line 4078 "parser.cpp" break; case 99: /* column_type: SPARSE '(' TINYINT ',' LONG_VALUE ')' */ #line 834 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 4092 "parser.cpp" +#line 4084 "parser.cpp" break; case 100: /* column_type: SPARSE '(' SMALLINT ',' LONG_VALUE ')' */ #line 835 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 4098 "parser.cpp" +#line 4090 "parser.cpp" break; case 101: /* column_type: SPARSE '(' INTEGER ',' LONG_VALUE ')' */ #line 836 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 4104 "parser.cpp" +#line 4096 "parser.cpp" break; case 102: /* column_type: SPARSE '(' INT ',' LONG_VALUE ')' */ #line 837 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 4110 "parser.cpp" +#line 4102 "parser.cpp" break; case 103: /* column_type: SPARSE '(' BIGINT ',' LONG_VALUE ')' */ #line 838 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 4116 "parser.cpp" +#line 4108 "parser.cpp" break; case 104: /* column_type: SPARSE '(' FLOAT ',' LONG_VALUE ')' */ #line 839 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 4122 "parser.cpp" +#line 4114 "parser.cpp" break; case 105: /* column_type: SPARSE '(' DOUBLE ',' LONG_VALUE ')' */ #line 840 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSparse, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 4128 "parser.cpp" +#line 4120 "parser.cpp" break; case 106: /* column_constraints: column_constraint */ @@ -4133,7 +4125,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.column_constraints_t) = new std::set(); (yyval.column_constraints_t)->insert((yyvsp[0].column_constraint_t)); } -#line 4137 "parser.cpp" +#line 4129 "parser.cpp" break; case 107: /* column_constraints: column_constraints column_constraint */ @@ -4147,7 +4139,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-1].column_constraints_t)->insert((yyvsp[0].column_constraint_t)); (yyval.column_constraints_t) = (yyvsp[-1].column_constraints_t); } -#line 4151 "parser.cpp" +#line 4143 "parser.cpp" break; case 108: /* column_constraint: PRIMARY KEY */ @@ -4155,7 +4147,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.column_constraint_t) = infinity::ConstraintType::kPrimaryKey; } -#line 4159 "parser.cpp" +#line 4151 "parser.cpp" break; case 109: /* column_constraint: UNIQUE */ @@ -4163,7 +4155,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.column_constraint_t) = infinity::ConstraintType::kUnique; } -#line 4167 "parser.cpp" +#line 4159 "parser.cpp" break; case 110: /* column_constraint: NULLABLE */ @@ -4171,7 +4163,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.column_constraint_t) = infinity::ConstraintType::kNull; } -#line 4175 "parser.cpp" +#line 4167 "parser.cpp" break; case 111: /* column_constraint: NOT NULLABLE */ @@ -4179,7 +4171,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.column_constraint_t) = infinity::ConstraintType::kNotNull; } -#line 4183 "parser.cpp" +#line 4175 "parser.cpp" break; case 112: /* default_expr: DEFAULT constant_expr */ @@ -4187,7 +4179,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 4191 "parser.cpp" +#line 4183 "parser.cpp" break; case 113: /* default_expr: %empty */ @@ -4195,7 +4187,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.const_expr_t) = nullptr; } -#line 4199 "parser.cpp" +#line 4191 "parser.cpp" break; case 114: /* table_constraint: PRIMARY KEY '(' identifier_array ')' */ @@ -4205,7 +4197,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_constraint_t)->names_ptr_ = (yyvsp[-1].identifier_array_t); (yyval.table_constraint_t)->constraint_ = infinity::ConstraintType::kPrimaryKey; } -#line 4209 "parser.cpp" +#line 4201 "parser.cpp" break; case 115: /* table_constraint: UNIQUE '(' identifier_array ')' */ @@ -4215,7 +4207,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_constraint_t)->names_ptr_ = (yyvsp[-1].identifier_array_t); (yyval.table_constraint_t)->constraint_ = infinity::ConstraintType::kUnique; } -#line 4219 "parser.cpp" +#line 4211 "parser.cpp" break; case 116: /* identifier_array: IDENTIFIER */ @@ -4226,7 +4218,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.identifier_array_t)->emplace_back((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 4230 "parser.cpp" +#line 4222 "parser.cpp" break; case 117: /* identifier_array: identifier_array ',' IDENTIFIER */ @@ -4237,7 +4229,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].str_value)); (yyval.identifier_array_t) = (yyvsp[-2].identifier_array_t); } -#line 4241 "parser.cpp" +#line 4233 "parser.cpp" break; case 118: /* delete_statement: DELETE FROM table_name where_clause */ @@ -4254,7 +4246,7 @@ YYLTYPE yylloc = yyloc_default; delete (yyvsp[-1].table_name_t); (yyval.delete_stmt)->where_expr_ = (yyvsp[0].expr_t); } -#line 4258 "parser.cpp" +#line 4250 "parser.cpp" break; case 119: /* insert_statement: INSERT INTO table_name optional_identifier_array VALUES expr_array_list */ @@ -4293,7 +4285,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.insert_stmt)->columns_ = (yyvsp[-2].identifier_array_t); (yyval.insert_stmt)->values_ = (yyvsp[0].expr_array_list_t); } -#line 4297 "parser.cpp" +#line 4289 "parser.cpp" break; case 120: /* insert_statement: INSERT INTO table_name optional_identifier_array select_without_paren */ @@ -4310,7 +4302,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.insert_stmt)->columns_ = (yyvsp[-1].identifier_array_t); (yyval.insert_stmt)->select_ = (yyvsp[0].select_stmt); } -#line 4314 "parser.cpp" +#line 4306 "parser.cpp" break; case 121: /* optional_identifier_array: '(' identifier_array ')' */ @@ -4318,7 +4310,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.identifier_array_t) = (yyvsp[-1].identifier_array_t); } -#line 4322 "parser.cpp" +#line 4314 "parser.cpp" break; case 122: /* optional_identifier_array: %empty */ @@ -4326,7 +4318,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.identifier_array_t) = nullptr; } -#line 4330 "parser.cpp" +#line 4322 "parser.cpp" break; case 123: /* explain_statement: EXPLAIN explain_type explainable_statement */ @@ -4336,7 +4328,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.explain_stmt)->type_ = (yyvsp[-1].explain_type_t); (yyval.explain_stmt)->statement_ = (yyvsp[0].base_stmt); } -#line 4340 "parser.cpp" +#line 4332 "parser.cpp" break; case 124: /* explain_type: ANALYZE */ @@ -4344,7 +4336,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kAnalyze; } -#line 4348 "parser.cpp" +#line 4340 "parser.cpp" break; case 125: /* explain_type: AST */ @@ -4352,7 +4344,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kAst; } -#line 4356 "parser.cpp" +#line 4348 "parser.cpp" break; case 126: /* explain_type: RAW */ @@ -4360,7 +4352,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kUnOpt; } -#line 4364 "parser.cpp" +#line 4356 "parser.cpp" break; case 127: /* explain_type: LOGICAL */ @@ -4368,7 +4360,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kOpt; } -#line 4372 "parser.cpp" +#line 4364 "parser.cpp" break; case 128: /* explain_type: PHYSICAL */ @@ -4376,7 +4368,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kPhysical; } -#line 4380 "parser.cpp" +#line 4372 "parser.cpp" break; case 129: /* explain_type: PIPELINE */ @@ -4384,7 +4376,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kPipeline; } -#line 4388 "parser.cpp" +#line 4380 "parser.cpp" break; case 130: /* explain_type: FRAGMENT */ @@ -4392,7 +4384,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kFragment; } -#line 4396 "parser.cpp" +#line 4388 "parser.cpp" break; case 131: /* explain_type: %empty */ @@ -4400,7 +4392,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.explain_type_t) = infinity::ExplainType::kPhysical; } -#line 4404 "parser.cpp" +#line 4396 "parser.cpp" break; case 132: /* update_statement: UPDATE table_name SET update_expr_array where_clause */ @@ -4417,7 +4409,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.update_stmt)->where_expr_ = (yyvsp[0].expr_t); (yyval.update_stmt)->update_expr_array_ = (yyvsp[-1].update_expr_array_t); } -#line 4421 "parser.cpp" +#line 4413 "parser.cpp" break; case 133: /* update_expr_array: update_expr */ @@ -4426,7 +4418,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.update_expr_array_t) = new std::vector(); (yyval.update_expr_array_t)->emplace_back((yyvsp[0].update_expr_t)); } -#line 4430 "parser.cpp" +#line 4422 "parser.cpp" break; case 134: /* update_expr_array: update_expr_array ',' update_expr */ @@ -4435,7 +4427,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-2].update_expr_array_t)->emplace_back((yyvsp[0].update_expr_t)); (yyval.update_expr_array_t) = (yyvsp[-2].update_expr_array_t); } -#line 4439 "parser.cpp" +#line 4431 "parser.cpp" break; case 135: /* update_expr: IDENTIFIER '=' expr */ @@ -4447,7 +4439,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-2].str_value)); (yyval.update_expr_t)->value = (yyvsp[0].expr_t); } -#line 4451 "parser.cpp" +#line 4443 "parser.cpp" break; case 136: /* drop_statement: DROP DATABASE if_exists IDENTIFIER */ @@ -4463,7 +4455,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.drop_stmt)->drop_info_ = drop_schema_info; (yyval.drop_stmt)->drop_info_->conflict_type_ = (yyvsp[-1].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; } -#line 4467 "parser.cpp" +#line 4459 "parser.cpp" break; case 137: /* drop_statement: DROP COLLECTION if_exists table_name */ @@ -4481,7 +4473,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.drop_stmt)->drop_info_->conflict_type_ = (yyvsp[-1].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; delete (yyvsp[0].table_name_t); } -#line 4485 "parser.cpp" +#line 4477 "parser.cpp" break; case 138: /* drop_statement: DROP TABLE if_exists table_name */ @@ -4499,7 +4491,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.drop_stmt)->drop_info_->conflict_type_ = (yyvsp[-1].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; delete (yyvsp[0].table_name_t); } -#line 4503 "parser.cpp" +#line 4495 "parser.cpp" break; case 139: /* drop_statement: DROP VIEW if_exists table_name */ @@ -4517,7 +4509,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.drop_stmt)->drop_info_->conflict_type_ = (yyvsp[-1].bool_value) ? infinity::ConflictType::kIgnore : infinity::ConflictType::kError; delete (yyvsp[0].table_name_t); } -#line 4521 "parser.cpp" +#line 4513 "parser.cpp" break; case 140: /* drop_statement: DROP INDEX if_exists IDENTIFIER ON table_name */ @@ -4540,7 +4532,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].table_name_t)->table_name_ptr_); delete (yyvsp[0].table_name_t); } -#line 4544 "parser.cpp" +#line 4536 "parser.cpp" break; case 141: /* copy_statement: COPY table_name TO file_path WITH '(' copy_option_list ')' */ @@ -4586,7 +4578,7 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].copy_option_array); } -#line 4590 "parser.cpp" +#line 4582 "parser.cpp" break; case 142: /* copy_statement: COPY table_name '(' expr_array ')' TO file_path WITH '(' copy_option_list ')' */ @@ -4634,7 +4626,7 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].copy_option_array); } -#line 4638 "parser.cpp" +#line 4630 "parser.cpp" break; case 143: /* copy_statement: COPY table_name FROM file_path WITH '(' copy_option_list ')' */ @@ -4680,7 +4672,7 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].copy_option_array); } -#line 4684 "parser.cpp" +#line 4676 "parser.cpp" break; case 144: /* select_statement: select_without_paren */ @@ -4688,7 +4680,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.select_stmt) = (yyvsp[0].select_stmt); } -#line 4692 "parser.cpp" +#line 4684 "parser.cpp" break; case 145: /* select_statement: select_with_paren */ @@ -4696,7 +4688,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.select_stmt) = (yyvsp[0].select_stmt); } -#line 4700 "parser.cpp" +#line 4692 "parser.cpp" break; case 146: /* select_statement: select_statement set_operator select_clause_without_modifier_paren */ @@ -4710,7 +4702,7 @@ YYLTYPE yylloc = yyloc_default; node->nested_select_ = (yyvsp[0].select_stmt); (yyval.select_stmt) = (yyvsp[-2].select_stmt); } -#line 4714 "parser.cpp" +#line 4706 "parser.cpp" break; case 147: /* select_statement: select_statement set_operator select_clause_without_modifier */ @@ -4724,7 +4716,7 @@ YYLTYPE yylloc = yyloc_default; node->nested_select_ = (yyvsp[0].select_stmt); (yyval.select_stmt) = (yyvsp[-2].select_stmt); } -#line 4728 "parser.cpp" +#line 4720 "parser.cpp" break; case 148: /* select_with_paren: '(' select_without_paren ')' */ @@ -4732,7 +4724,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4736 "parser.cpp" +#line 4728 "parser.cpp" break; case 149: /* select_with_paren: '(' select_with_paren ')' */ @@ -4740,7 +4732,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4744 "parser.cpp" +#line 4736 "parser.cpp" break; case 150: /* select_without_paren: with_clause select_clause_with_modifier */ @@ -4749,7 +4741,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[0].select_stmt)->with_exprs_ = (yyvsp[-1].with_expr_list_t); (yyval.select_stmt) = (yyvsp[0].select_stmt); } -#line 4753 "parser.cpp" +#line 4745 "parser.cpp" break; case 151: /* select_clause_with_modifier: select_clause_without_modifier order_by_clause limit_expr offset_expr */ @@ -4775,7 +4767,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-3].select_stmt)->offset_expr_ = (yyvsp[0].expr_t); (yyval.select_stmt) = (yyvsp[-3].select_stmt); } -#line 4779 "parser.cpp" +#line 4771 "parser.cpp" break; case 152: /* select_clause_without_modifier_paren: '(' select_clause_without_modifier ')' */ @@ -4783,7 +4775,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4787 "parser.cpp" +#line 4779 "parser.cpp" break; case 153: /* select_clause_without_modifier_paren: '(' select_clause_without_modifier_paren ')' */ @@ -4791,7 +4783,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4795 "parser.cpp" +#line 4787 "parser.cpp" break; case 154: /* select_clause_without_modifier: SELECT distinct expr_array from_clause search_clause where_clause group_by_clause having_clause */ @@ -4811,7 +4803,7 @@ YYLTYPE yylloc = yyloc_default; YYERROR; } } -#line 4815 "parser.cpp" +#line 4807 "parser.cpp" break; case 155: /* order_by_clause: ORDER BY order_by_expr_list */ @@ -4819,7 +4811,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.order_by_expr_list_t) = (yyvsp[0].order_by_expr_list_t); } -#line 4823 "parser.cpp" +#line 4815 "parser.cpp" break; case 156: /* order_by_clause: %empty */ @@ -4827,7 +4819,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.order_by_expr_list_t) = nullptr; } -#line 4831 "parser.cpp" +#line 4823 "parser.cpp" break; case 157: /* order_by_expr_list: order_by_expr */ @@ -4836,7 +4828,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.order_by_expr_list_t) = new std::vector(); (yyval.order_by_expr_list_t)->emplace_back((yyvsp[0].order_by_expr_t)); } -#line 4840 "parser.cpp" +#line 4832 "parser.cpp" break; case 158: /* order_by_expr_list: order_by_expr_list ',' order_by_expr */ @@ -4845,7 +4837,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-2].order_by_expr_list_t)->emplace_back((yyvsp[0].order_by_expr_t)); (yyval.order_by_expr_list_t) = (yyvsp[-2].order_by_expr_list_t); } -#line 4849 "parser.cpp" +#line 4841 "parser.cpp" break; case 159: /* order_by_expr: expr order_by_type */ @@ -4855,7 +4847,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.order_by_expr_t)->expr_ = (yyvsp[-1].expr_t); (yyval.order_by_expr_t)->type_ = (yyvsp[0].order_by_type_t); } -#line 4859 "parser.cpp" +#line 4851 "parser.cpp" break; case 160: /* order_by_type: ASC */ @@ -4863,7 +4855,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.order_by_type_t) = infinity::kAsc; } -#line 4867 "parser.cpp" +#line 4859 "parser.cpp" break; case 161: /* order_by_type: DESC */ @@ -4871,7 +4863,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.order_by_type_t) = infinity::kDesc; } -#line 4875 "parser.cpp" +#line 4867 "parser.cpp" break; case 162: /* order_by_type: %empty */ @@ -4879,7 +4871,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.order_by_type_t) = infinity::kAsc; } -#line 4883 "parser.cpp" +#line 4875 "parser.cpp" break; case 163: /* limit_expr: LIMIT expr */ @@ -4887,13 +4879,13 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4891 "parser.cpp" +#line 4883 "parser.cpp" break; case 164: /* limit_expr: %empty */ #line 1393 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4897 "parser.cpp" +#line 4889 "parser.cpp" break; case 165: /* offset_expr: OFFSET expr */ @@ -4901,13 +4893,13 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4905 "parser.cpp" +#line 4897 "parser.cpp" break; case 166: /* offset_expr: %empty */ #line 1399 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4911 "parser.cpp" +#line 4903 "parser.cpp" break; case 167: /* distinct: DISTINCT */ @@ -4915,7 +4907,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.bool_value) = true; } -#line 4919 "parser.cpp" +#line 4911 "parser.cpp" break; case 168: /* distinct: %empty */ @@ -4923,7 +4915,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.bool_value) = false; } -#line 4927 "parser.cpp" +#line 4919 "parser.cpp" break; case 169: /* from_clause: FROM table_reference */ @@ -4931,7 +4923,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.table_reference_t) = (yyvsp[0].table_reference_t); } -#line 4935 "parser.cpp" +#line 4927 "parser.cpp" break; case 170: /* from_clause: %empty */ @@ -4939,7 +4931,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.table_reference_t) = nullptr; } -#line 4943 "parser.cpp" +#line 4935 "parser.cpp" break; case 171: /* search_clause: SEARCH sub_search_array */ @@ -4949,7 +4941,7 @@ YYLTYPE yylloc = yyloc_default; search_expr->SetExprs((yyvsp[0].expr_array_t)); (yyval.expr_t) = search_expr; } -#line 4953 "parser.cpp" +#line 4945 "parser.cpp" break; case 172: /* search_clause: %empty */ @@ -4957,7 +4949,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = nullptr; } -#line 4961 "parser.cpp" +#line 4953 "parser.cpp" break; case 173: /* where_clause: WHERE expr */ @@ -4965,7 +4957,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4969 "parser.cpp" +#line 4961 "parser.cpp" break; case 174: /* where_clause: %empty */ @@ -4973,7 +4965,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = nullptr; } -#line 4977 "parser.cpp" +#line 4969 "parser.cpp" break; case 175: /* having_clause: HAVING expr */ @@ -4981,7 +4973,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4985 "parser.cpp" +#line 4977 "parser.cpp" break; case 176: /* having_clause: %empty */ @@ -4989,7 +4981,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_t) = nullptr; } -#line 4993 "parser.cpp" +#line 4985 "parser.cpp" break; case 177: /* group_by_clause: GROUP BY expr_array */ @@ -4997,7 +4989,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_array_t) = (yyvsp[0].expr_array_t); } -#line 5001 "parser.cpp" +#line 4993 "parser.cpp" break; case 178: /* group_by_clause: %empty */ @@ -5005,7 +4997,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.expr_array_t) = nullptr; } -#line 5009 "parser.cpp" +#line 5001 "parser.cpp" break; case 179: /* set_operator: UNION */ @@ -5013,7 +5005,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.set_operator_t) = infinity::SetOperatorType::kUnion; } -#line 5017 "parser.cpp" +#line 5009 "parser.cpp" break; case 180: /* set_operator: UNION ALL */ @@ -5021,7 +5013,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.set_operator_t) = infinity::SetOperatorType::kUnionAll; } -#line 5025 "parser.cpp" +#line 5017 "parser.cpp" break; case 181: /* set_operator: INTERSECT */ @@ -5029,7 +5021,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.set_operator_t) = infinity::SetOperatorType::kIntersect; } -#line 5033 "parser.cpp" +#line 5025 "parser.cpp" break; case 182: /* set_operator: EXCEPT */ @@ -5037,7 +5029,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.set_operator_t) = infinity::SetOperatorType::kExcept; } -#line 5041 "parser.cpp" +#line 5033 "parser.cpp" break; case 183: /* table_reference: table_reference_unit */ @@ -5045,7 +5037,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.table_reference_t) = (yyvsp[0].table_reference_t); } -#line 5049 "parser.cpp" +#line 5041 "parser.cpp" break; case 184: /* table_reference: table_reference ',' table_reference_unit */ @@ -5063,7 +5055,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_reference_t) = cross_product_ref; } -#line 5067 "parser.cpp" +#line 5059 "parser.cpp" break; case 187: /* table_reference_name: table_name table_alias */ @@ -5081,7 +5073,7 @@ YYLTYPE yylloc = yyloc_default; table_ref->alias_ = (yyvsp[0].table_alias_t); (yyval.table_reference_t) = table_ref; } -#line 5085 "parser.cpp" +#line 5077 "parser.cpp" break; case 188: /* table_reference_name: '(' select_statement ')' table_alias */ @@ -5092,7 +5084,7 @@ YYLTYPE yylloc = yyloc_default; subquery_reference->alias_ = (yyvsp[0].table_alias_t); (yyval.table_reference_t) = subquery_reference; } -#line 5096 "parser.cpp" +#line 5088 "parser.cpp" break; case 189: /* table_name: IDENTIFIER */ @@ -5102,7 +5094,7 @@ YYLTYPE yylloc = yyloc_default; ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.table_name_t)->table_name_ptr_ = (yyvsp[0].str_value); } -#line 5106 "parser.cpp" +#line 5098 "parser.cpp" break; case 190: /* table_name: IDENTIFIER '.' IDENTIFIER */ @@ -5114,7 +5106,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_name_t)->schema_name_ptr_ = (yyvsp[-2].str_value); (yyval.table_name_t)->table_name_ptr_ = (yyvsp[0].str_value); } -#line 5118 "parser.cpp" +#line 5110 "parser.cpp" break; case 191: /* table_alias: AS IDENTIFIER */ @@ -5124,7 +5116,7 @@ YYLTYPE yylloc = yyloc_default; ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.table_alias_t)->alias_ = (yyvsp[0].str_value); } -#line 5128 "parser.cpp" +#line 5120 "parser.cpp" break; case 192: /* table_alias: IDENTIFIER */ @@ -5134,7 +5126,7 @@ YYLTYPE yylloc = yyloc_default; ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.table_alias_t)->alias_ = (yyvsp[0].str_value); } -#line 5138 "parser.cpp" +#line 5130 "parser.cpp" break; case 193: /* table_alias: AS IDENTIFIER '(' identifier_array ')' */ @@ -5145,7 +5137,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_alias_t)->alias_ = (yyvsp[-3].str_value); (yyval.table_alias_t)->column_alias_array_ = (yyvsp[-1].identifier_array_t); } -#line 5149 "parser.cpp" +#line 5141 "parser.cpp" break; case 194: /* table_alias: %empty */ @@ -5153,7 +5145,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.table_alias_t) = nullptr; } -#line 5157 "parser.cpp" +#line 5149 "parser.cpp" break; case 195: /* with_clause: WITH with_expr_list */ @@ -5161,7 +5153,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.with_expr_list_t) = (yyvsp[0].with_expr_list_t); } -#line 5165 "parser.cpp" +#line 5157 "parser.cpp" break; case 196: /* with_clause: %empty */ @@ -5169,7 +5161,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.with_expr_list_t) = nullptr; } -#line 5173 "parser.cpp" +#line 5165 "parser.cpp" break; case 197: /* with_expr_list: with_expr */ @@ -5178,7 +5170,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.with_expr_list_t) = new std::vector(); (yyval.with_expr_list_t)->emplace_back((yyvsp[0].with_expr_t)); } -#line 5182 "parser.cpp" +#line 5174 "parser.cpp" break; case 198: /* with_expr_list: with_expr_list ',' with_expr */ @@ -5187,7 +5179,7 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-2].with_expr_list_t)->emplace_back((yyvsp[0].with_expr_t)); (yyval.with_expr_list_t) = (yyvsp[-2].with_expr_list_t); } -#line 5191 "parser.cpp" +#line 5183 "parser.cpp" break; case 199: /* with_expr: IDENTIFIER AS '(' select_clause_with_modifier ')' */ @@ -5199,7 +5191,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-4].str_value)); (yyval.with_expr_t)->select_ = (yyvsp[-1].select_stmt); } -#line 5203 "parser.cpp" +#line 5195 "parser.cpp" break; case 200: /* join_clause: table_reference_unit NATURAL JOIN table_reference_name */ @@ -5211,7 +5203,7 @@ YYLTYPE yylloc = yyloc_default; join_reference->join_type_ = infinity::JoinType::kNatural; (yyval.table_reference_t) = join_reference; } -#line 5215 "parser.cpp" +#line 5207 "parser.cpp" break; case 201: /* join_clause: table_reference_unit join_type JOIN table_reference_name ON expr */ @@ -5224,7 +5216,7 @@ YYLTYPE yylloc = yyloc_default; join_reference->condition_ = (yyvsp[0].expr_t); (yyval.table_reference_t) = join_reference; } -#line 5228 "parser.cpp" +#line 5220 "parser.cpp" break; case 202: /* join_type: INNER */ @@ -5232,7 +5224,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.join_type_t) = infinity::JoinType::kInner; } -#line 5236 "parser.cpp" +#line 5228 "parser.cpp" break; case 203: /* join_type: LEFT */ @@ -5240,7 +5232,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.join_type_t) = infinity::JoinType::kLeft; } -#line 5244 "parser.cpp" +#line 5236 "parser.cpp" break; case 204: /* join_type: RIGHT */ @@ -5248,7 +5240,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.join_type_t) = infinity::JoinType::kRight; } -#line 5252 "parser.cpp" +#line 5244 "parser.cpp" break; case 205: /* join_type: OUTER */ @@ -5256,7 +5248,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.join_type_t) = infinity::JoinType::kFull; } -#line 5260 "parser.cpp" +#line 5252 "parser.cpp" break; case 206: /* join_type: FULL */ @@ -5264,7 +5256,7 @@ YYLTYPE yylloc = yyloc_default; { (yyval.join_type_t) = infinity::JoinType::kFull; } -#line 5268 "parser.cpp" +#line 5260 "parser.cpp" break; case 207: /* join_type: CROSS */ @@ -5272,14 +5264,14 @@ YYLTYPE yylloc = yyloc_default; { (yyval.join_type_t) = infinity::JoinType::kCross; } -#line 5276 "parser.cpp" +#line 5268 "parser.cpp" break; case 208: /* join_type: %empty */ #line 1608 "parser.y" { } -#line 5283 "parser.cpp" +#line 5275 "parser.cpp" break; case 209: /* show_statement: SHOW DATABASES */ @@ -5288,7 +5280,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kDatabases; } -#line 5292 "parser.cpp" +#line 5284 "parser.cpp" break; case 210: /* show_statement: SHOW TABLES */ @@ -5297,7 +5289,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kTables; } -#line 5301 "parser.cpp" +#line 5293 "parser.cpp" break; case 211: /* show_statement: SHOW VIEWS */ @@ -5306,7 +5298,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kViews; } -#line 5310 "parser.cpp" +#line 5302 "parser.cpp" break; case 212: /* show_statement: SHOW CONFIGS */ @@ -5315,7 +5307,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kConfigs; } -#line 5319 "parser.cpp" +#line 5311 "parser.cpp" break; case 213: /* show_statement: SHOW CONFIG IDENTIFIER */ @@ -5327,7 +5319,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->var_name_ = std::string((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 5331 "parser.cpp" +#line 5323 "parser.cpp" break; case 214: /* show_statement: SHOW PROFILES */ @@ -5336,7 +5328,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kProfiles; } -#line 5340 "parser.cpp" +#line 5332 "parser.cpp" break; case 215: /* show_statement: SHOW BUFFER */ @@ -5345,7 +5337,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kBuffer; } -#line 5349 "parser.cpp" +#line 5341 "parser.cpp" break; case 216: /* show_statement: SHOW QUERIES */ @@ -5354,7 +5346,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kQueries; } -#line 5358 "parser.cpp" +#line 5350 "parser.cpp" break; case 217: /* show_statement: SHOW QUERY SESSION LONG_VALUE */ @@ -5364,7 +5356,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kQuery; (yyval.show_stmt)->session_id_ = (yyvsp[0].long_value); } -#line 5368 "parser.cpp" +#line 5360 "parser.cpp" break; case 218: /* show_statement: SHOW TRANSACTIONS */ @@ -5373,7 +5365,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kTransactions; } -#line 5377 "parser.cpp" +#line 5369 "parser.cpp" break; case 219: /* show_statement: SHOW TRANSACTION LONG_VALUE */ @@ -5383,7 +5375,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kTransaction; (yyval.show_stmt)->txn_id_ = (yyvsp[0].long_value); } -#line 5387 "parser.cpp" +#line 5379 "parser.cpp" break; case 220: /* show_statement: SHOW SESSION VARIABLES */ @@ -5392,7 +5384,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kSessionVariables; } -#line 5396 "parser.cpp" +#line 5388 "parser.cpp" break; case 221: /* show_statement: SHOW GLOBAL VARIABLES */ @@ -5401,7 +5393,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kGlobalVariables; } -#line 5405 "parser.cpp" +#line 5397 "parser.cpp" break; case 222: /* show_statement: SHOW SESSION VARIABLE IDENTIFIER */ @@ -5412,7 +5404,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->var_name_ = std::string((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 5416 "parser.cpp" +#line 5408 "parser.cpp" break; case 223: /* show_statement: SHOW GLOBAL VARIABLE IDENTIFIER */ @@ -5423,7 +5415,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->var_name_ = std::string((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 5427 "parser.cpp" +#line 5419 "parser.cpp" break; case 224: /* show_statement: SHOW DATABASE IDENTIFIER */ @@ -5434,7 +5426,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->schema_name_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 5438 "parser.cpp" +#line 5430 "parser.cpp" break; case 225: /* show_statement: SHOW TABLE table_name */ @@ -5450,7 +5442,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].table_name_t)->table_name_ptr_); delete (yyvsp[0].table_name_t); } -#line 5454 "parser.cpp" +#line 5446 "parser.cpp" break; case 226: /* show_statement: SHOW TABLE table_name COLUMNS */ @@ -5466,7 +5458,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].table_name_t)->table_name_ptr_); delete (yyvsp[-1].table_name_t); } -#line 5470 "parser.cpp" +#line 5462 "parser.cpp" break; case 227: /* show_statement: SHOW TABLE table_name SEGMENTS */ @@ -5482,7 +5474,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].table_name_t)->table_name_ptr_); delete (yyvsp[-1].table_name_t); } -#line 5486 "parser.cpp" +#line 5478 "parser.cpp" break; case 228: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE */ @@ -5499,7 +5491,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->segment_id_ = (yyvsp[0].long_value); delete (yyvsp[-2].table_name_t); } -#line 5503 "parser.cpp" +#line 5495 "parser.cpp" break; case 229: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCKS */ @@ -5516,7 +5508,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->segment_id_ = (yyvsp[-1].long_value); delete (yyvsp[-3].table_name_t); } -#line 5520 "parser.cpp" +#line 5512 "parser.cpp" break; case 230: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCK LONG_VALUE */ @@ -5534,7 +5526,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->block_id_ = (yyvsp[0].long_value); delete (yyvsp[-4].table_name_t); } -#line 5538 "parser.cpp" +#line 5530 "parser.cpp" break; case 231: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCK LONG_VALUE COLUMN LONG_VALUE */ @@ -5553,7 +5545,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->column_id_ = (yyvsp[0].long_value); delete (yyvsp[-6].table_name_t); } -#line 5557 "parser.cpp" +#line 5549 "parser.cpp" break; case 232: /* show_statement: SHOW TABLE table_name INDEXES */ @@ -5569,7 +5561,7 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].table_name_t)->table_name_ptr_); delete (yyvsp[-1].table_name_t); } -#line 5573 "parser.cpp" +#line 5565 "parser.cpp" break; case 233: /* show_statement: SHOW TABLE table_name INDEX IDENTIFIER */ @@ -5588,7 +5580,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->index_name_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 5592 "parser.cpp" +#line 5584 "parser.cpp" break; case 234: /* show_statement: SHOW TABLE table_name INDEX IDENTIFIER SEGMENT LONG_VALUE */ @@ -5609,7 +5601,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->segment_id_ = (yyvsp[0].long_value); } -#line 5613 "parser.cpp" +#line 5605 "parser.cpp" break; case 235: /* show_statement: SHOW TABLE table_name INDEX IDENTIFIER SEGMENT LONG_VALUE CHUNK LONG_VALUE */ @@ -5631,7 +5623,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->segment_id_ = (yyvsp[-2].long_value); (yyval.show_stmt)->chunk_id_ = (yyvsp[0].long_value); } -#line 5635 "parser.cpp" +#line 5627 "parser.cpp" break; case 236: /* flush_statement: FLUSH DATA */ @@ -5640,7 +5632,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.flush_stmt) = new infinity::FlushStatement(); (yyval.flush_stmt)->type_ = infinity::FlushType::kData; } -#line 5644 "parser.cpp" +#line 5636 "parser.cpp" break; case 237: /* flush_statement: FLUSH LOG */ @@ -5649,7 +5641,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.flush_stmt) = new infinity::FlushStatement(); (yyval.flush_stmt)->type_ = infinity::FlushType::kLog; } -#line 5653 "parser.cpp" +#line 5645 "parser.cpp" break; case 238: /* flush_statement: FLUSH BUFFER */ @@ -5658,7 +5650,7 @@ YYLTYPE yylloc = yyloc_default; (yyval.flush_stmt) = new infinity::FlushStatement(); (yyval.flush_stmt)->type_ = infinity::FlushType::kBuffer; } -#line 5662 "parser.cpp" +#line 5654 "parser.cpp" break; case 239: /* optimize_statement: OPTIMIZE table_name */ @@ -5673,54 +5665,78 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].table_name_t)->table_name_ptr_); delete (yyvsp[0].table_name_t); } -#line 5677 "parser.cpp" +#line 5669 "parser.cpp" + break; + + case 240: /* optimize_statement: OPTIMIZE IDENTIFIER ON table_name with_index_param_list */ +#line 1862 "parser.y" + { + (yyval.optimize_stmt) = new infinity::OptimizeStatement(); + if((yyvsp[-1].table_name_t)->schema_name_ptr_ != nullptr) { + (yyval.optimize_stmt)->schema_name_ = (yyvsp[-1].table_name_t)->schema_name_ptr_; + free((yyvsp[-1].table_name_t)->schema_name_ptr_); + } + (yyval.optimize_stmt)->table_name_ = (yyvsp[-1].table_name_t)->table_name_ptr_; + free((yyvsp[-1].table_name_t)->table_name_ptr_); + delete (yyvsp[-1].table_name_t); + + (yyval.optimize_stmt)->index_name_ = (yyvsp[-3].str_value); + free((yyvsp[-3].str_value)); + + for (auto *&index_param : *(yyvsp[0].with_index_param_list_t)) { + (yyval.optimize_stmt)->opt_params_.emplace_back(std::unique_ptr(index_param)); + index_param = nullptr; + } + delete (yyvsp[0].with_index_param_list_t); +} +#line 5693 "parser.cpp" break; - case 240: /* command_statement: USE IDENTIFIER */ -#line 1865 "parser.y" + case 241: /* command_statement: USE IDENTIFIER */ +#line 1885 "parser.y" { (yyval.command_stmt) = new infinity::CommandStatement(); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.command_stmt)->command_info_ = std::make_shared((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 5688 "parser.cpp" +#line 5704 "parser.cpp" break; - case 241: /* command_statement: EXPORT PROFILE LONG_VALUE file_path */ -#line 1871 "parser.y" + case 242: /* command_statement: EXPORT PROFILE LONG_VALUE file_path */ +#line 1891 "parser.y" { (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared((yyvsp[0].str_value), infinity::ExportType::kProfileRecord, (yyvsp[-1].long_value)); free((yyvsp[0].str_value)); } -#line 5698 "parser.cpp" +#line 5714 "parser.cpp" break; - case 242: /* command_statement: SET SESSION IDENTIFIER ON */ -#line 1876 "parser.y" + case 243: /* command_statement: SET SESSION IDENTIFIER ON */ +#line 1896 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kSession, infinity::SetVarType::kBool, (yyvsp[-1].str_value), true); free((yyvsp[-1].str_value)); } -#line 5709 "parser.cpp" +#line 5725 "parser.cpp" break; - case 243: /* command_statement: SET SESSION IDENTIFIER OFF */ -#line 1882 "parser.y" + case 244: /* command_statement: SET SESSION IDENTIFIER OFF */ +#line 1902 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kSession, infinity::SetVarType::kBool, (yyvsp[-1].str_value), false); free((yyvsp[-1].str_value)); } -#line 5720 "parser.cpp" +#line 5736 "parser.cpp" break; - case 244: /* command_statement: SET SESSION IDENTIFIER IDENTIFIER */ -#line 1888 "parser.y" + case 245: /* command_statement: SET SESSION IDENTIFIER IDENTIFIER */ +#line 1908 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -5729,55 +5745,55 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); free((yyvsp[0].str_value)); } -#line 5733 "parser.cpp" +#line 5749 "parser.cpp" break; - case 245: /* command_statement: SET SESSION IDENTIFIER LONG_VALUE */ -#line 1896 "parser.y" + case 246: /* command_statement: SET SESSION IDENTIFIER LONG_VALUE */ +#line 1916 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kSession, infinity::SetVarType::kInteger, (yyvsp[-1].str_value), (yyvsp[0].long_value)); free((yyvsp[-1].str_value)); } -#line 5744 "parser.cpp" +#line 5760 "parser.cpp" break; - case 246: /* command_statement: SET SESSION IDENTIFIER DOUBLE_VALUE */ -#line 1902 "parser.y" + case 247: /* command_statement: SET SESSION IDENTIFIER DOUBLE_VALUE */ +#line 1922 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kSession, infinity::SetVarType::kDouble, (yyvsp[-1].str_value), (yyvsp[0].double_value)); free((yyvsp[-1].str_value)); } -#line 5755 "parser.cpp" +#line 5771 "parser.cpp" break; - case 247: /* command_statement: SET GLOBAL IDENTIFIER ON */ -#line 1908 "parser.y" + case 248: /* command_statement: SET GLOBAL IDENTIFIER ON */ +#line 1928 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kGlobal, infinity::SetVarType::kBool, (yyvsp[-1].str_value), true); free((yyvsp[-1].str_value)); } -#line 5766 "parser.cpp" +#line 5782 "parser.cpp" break; - case 248: /* command_statement: SET GLOBAL IDENTIFIER OFF */ -#line 1914 "parser.y" + case 249: /* command_statement: SET GLOBAL IDENTIFIER OFF */ +#line 1934 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kGlobal, infinity::SetVarType::kBool, (yyvsp[-1].str_value), false); free((yyvsp[-1].str_value)); } -#line 5777 "parser.cpp" +#line 5793 "parser.cpp" break; - case 249: /* command_statement: SET GLOBAL IDENTIFIER IDENTIFIER */ -#line 1920 "parser.y" + case 250: /* command_statement: SET GLOBAL IDENTIFIER IDENTIFIER */ +#line 1940 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -5786,55 +5802,55 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); free((yyvsp[0].str_value)); } -#line 5790 "parser.cpp" +#line 5806 "parser.cpp" break; - case 250: /* command_statement: SET GLOBAL IDENTIFIER LONG_VALUE */ -#line 1928 "parser.y" + case 251: /* command_statement: SET GLOBAL IDENTIFIER LONG_VALUE */ +#line 1948 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kGlobal, infinity::SetVarType::kInteger, (yyvsp[-1].str_value), (yyvsp[0].long_value)); free((yyvsp[-1].str_value)); } -#line 5801 "parser.cpp" +#line 5817 "parser.cpp" break; - case 251: /* command_statement: SET GLOBAL IDENTIFIER DOUBLE_VALUE */ -#line 1934 "parser.y" + case 252: /* command_statement: SET GLOBAL IDENTIFIER DOUBLE_VALUE */ +#line 1954 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kGlobal, infinity::SetVarType::kDouble, (yyvsp[-1].str_value), (yyvsp[0].double_value)); free((yyvsp[-1].str_value)); } -#line 5812 "parser.cpp" +#line 5828 "parser.cpp" break; - case 252: /* command_statement: SET CONFIG IDENTIFIER ON */ -#line 1940 "parser.y" + case 253: /* command_statement: SET CONFIG IDENTIFIER ON */ +#line 1960 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kConfig, infinity::SetVarType::kBool, (yyvsp[-1].str_value), true); free((yyvsp[-1].str_value)); } -#line 5823 "parser.cpp" +#line 5839 "parser.cpp" break; - case 253: /* command_statement: SET CONFIG IDENTIFIER OFF */ -#line 1946 "parser.y" + case 254: /* command_statement: SET CONFIG IDENTIFIER OFF */ +#line 1966 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kConfig, infinity::SetVarType::kBool, (yyvsp[-1].str_value), false); free((yyvsp[-1].str_value)); } -#line 5834 "parser.cpp" +#line 5850 "parser.cpp" break; - case 254: /* command_statement: SET CONFIG IDENTIFIER IDENTIFIER */ -#line 1952 "parser.y" + case 255: /* command_statement: SET CONFIG IDENTIFIER IDENTIFIER */ +#line 1972 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -5843,33 +5859,33 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); free((yyvsp[0].str_value)); } -#line 5847 "parser.cpp" +#line 5863 "parser.cpp" break; - case 255: /* command_statement: SET CONFIG IDENTIFIER LONG_VALUE */ -#line 1960 "parser.y" + case 256: /* command_statement: SET CONFIG IDENTIFIER LONG_VALUE */ +#line 1980 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kConfig, infinity::SetVarType::kInteger, (yyvsp[-1].str_value), (yyvsp[0].long_value)); free((yyvsp[-1].str_value)); } -#line 5858 "parser.cpp" +#line 5874 "parser.cpp" break; - case 256: /* command_statement: SET CONFIG IDENTIFIER DOUBLE_VALUE */ -#line 1966 "parser.y" + case 257: /* command_statement: SET CONFIG IDENTIFIER DOUBLE_VALUE */ +#line 1986 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_shared(infinity::SetScope::kConfig, infinity::SetVarType::kDouble, (yyvsp[-1].str_value), (yyvsp[0].double_value)); free((yyvsp[-1].str_value)); } -#line 5869 "parser.cpp" +#line 5885 "parser.cpp" break; - case 257: /* compact_statement: COMPACT TABLE table_name */ -#line 1973 "parser.y" + case 258: /* compact_statement: COMPACT TABLE table_name */ +#line 1993 "parser.y" { std::string schema_name; if ((yyvsp[0].table_name_t)->schema_name_ptr_ != nullptr) { @@ -5882,38 +5898,38 @@ YYLTYPE yylloc = yyloc_default; (yyval.compact_stmt) = new infinity::ManualCompactStatement(std::move(schema_name), std::move(table_name)); delete (yyvsp[0].table_name_t); } -#line 5886 "parser.cpp" +#line 5902 "parser.cpp" break; - case 258: /* expr_array: expr_alias */ -#line 1990 "parser.y" + case 259: /* expr_array: expr_alias */ +#line 2010 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 5895 "parser.cpp" +#line 5911 "parser.cpp" break; - case 259: /* expr_array: expr_array ',' expr_alias */ -#line 1994 "parser.y" + case 260: /* expr_array: expr_array ',' expr_alias */ +#line 2014 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 5904 "parser.cpp" +#line 5920 "parser.cpp" break; - case 260: /* expr_array_list: '(' expr_array ')' */ -#line 1999 "parser.y" + case 261: /* expr_array_list: '(' expr_array ')' */ +#line 2019 "parser.y" { (yyval.expr_array_list_t) = new std::vector*>(); (yyval.expr_array_list_t)->push_back((yyvsp[-1].expr_array_t)); } -#line 5913 "parser.cpp" +#line 5929 "parser.cpp" break; - case 261: /* expr_array_list: expr_array_list ',' '(' expr_array ')' */ -#line 2003 "parser.y" + case 262: /* expr_array_list: expr_array_list ',' '(' expr_array ')' */ +#line 2023 "parser.y" { if(!(yyvsp[-4].expr_array_list_t)->empty() && (yyvsp[-4].expr_array_list_t)->back()->size() != (yyvsp[-1].expr_array_t)->size()) { yyerror(&yyloc, scanner, result, "The expr_array in list shall have the same size."); @@ -5929,73 +5945,73 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-4].expr_array_list_t)->push_back((yyvsp[-1].expr_array_t)); (yyval.expr_array_list_t) = (yyvsp[-4].expr_array_list_t); } -#line 5933 "parser.cpp" +#line 5949 "parser.cpp" break; - case 262: /* expr_alias: expr AS IDENTIFIER */ -#line 2030 "parser.y" + case 263: /* expr_alias: expr AS IDENTIFIER */ +#line 2050 "parser.y" { (yyval.expr_t) = (yyvsp[-2].expr_t); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.expr_t)->alias_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 5944 "parser.cpp" +#line 5960 "parser.cpp" break; - case 263: /* expr_alias: expr */ -#line 2036 "parser.y" + case 264: /* expr_alias: expr */ +#line 2056 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 5952 "parser.cpp" +#line 5968 "parser.cpp" break; - case 269: /* operand: '(' expr ')' */ -#line 2046 "parser.y" + case 270: /* operand: '(' expr ')' */ +#line 2066 "parser.y" { (yyval.expr_t) = (yyvsp[-1].expr_t); } -#line 5960 "parser.cpp" +#line 5976 "parser.cpp" break; - case 270: /* operand: '(' select_without_paren ')' */ -#line 2049 "parser.y" + case 271: /* operand: '(' select_without_paren ')' */ +#line 2069 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kScalar; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 5971 "parser.cpp" +#line 5987 "parser.cpp" break; - case 271: /* operand: constant_expr */ -#line 2055 "parser.y" + case 272: /* operand: constant_expr */ +#line 2075 "parser.y" { (yyval.expr_t) = (yyvsp[0].const_expr_t); } -#line 5979 "parser.cpp" +#line 5995 "parser.cpp" break; - case 282: /* extra_match_tensor_option: ',' STRING */ -#line 2069 "parser.y" + case 283: /* extra_match_tensor_option: ',' STRING */ +#line 2089 "parser.y" { (yyval.str_value) = (yyvsp[0].str_value); } -#line 5987 "parser.cpp" +#line 6003 "parser.cpp" break; - case 283: /* extra_match_tensor_option: %empty */ -#line 2072 "parser.y" + case 284: /* extra_match_tensor_option: %empty */ +#line 2092 "parser.y" { (yyval.str_value) = nullptr; } -#line 5995 "parser.cpp" +#line 6011 "parser.cpp" break; - case 284: /* match_tensor_expr: MATCH TENSOR '(' column_expr ',' common_array_expr ',' STRING ',' STRING extra_match_tensor_option ')' */ -#line 2078 "parser.y" + case 285: /* match_tensor_expr: MATCH TENSOR '(' column_expr ',' common_array_expr ',' STRING ',' STRING extra_match_tensor_option ')' */ +#line 2098 "parser.y" { auto match_tensor_expr = std::make_unique(); // search column @@ -6012,11 +6028,11 @@ YYLTYPE yylloc = yyloc_default; } (yyval.expr_t) = match_tensor_expr.release(); } -#line 6016 "parser.cpp" +#line 6032 "parser.cpp" break; - case 285: /* match_vector_expr: MATCH VECTOR '(' expr ',' array_expr ',' STRING ',' STRING ',' LONG_VALUE ')' with_index_param_list */ -#line 2097 "parser.y" + case 286: /* match_vector_expr: MATCH VECTOR '(' expr ',' array_expr ',' STRING ',' STRING ',' LONG_VALUE ')' with_index_param_list */ +#line 2117 "parser.y" { infinity::KnnExpr* match_vector_expr = new infinity::KnnExpr(); (yyval.expr_t) = match_vector_expr; @@ -6058,11 +6074,11 @@ YYLTYPE yylloc = yyloc_default; Return1: ; } -#line 6062 "parser.cpp" +#line 6078 "parser.cpp" break; - case 286: /* match_vector_expr: MATCH VECTOR '(' expr ',' array_expr ',' STRING ',' STRING ')' with_index_param_list */ -#line 2139 "parser.y" + case 287: /* match_vector_expr: MATCH VECTOR '(' expr ',' array_expr ',' STRING ',' STRING ')' with_index_param_list */ +#line 2159 "parser.y" { infinity::KnnExpr* match_vector_expr = new infinity::KnnExpr(); (yyval.expr_t) = match_vector_expr; @@ -6105,11 +6121,11 @@ YYLTYPE yylloc = yyloc_default; Return2: ; } -#line 6109 "parser.cpp" +#line 6125 "parser.cpp" break; - case 287: /* match_sparse_expr: MATCH SPARSE '(' expr ',' common_sparse_array_expr ',' STRING ',' LONG_VALUE ')' with_index_param_list */ -#line 2185 "parser.y" + case 288: /* match_sparse_expr: MATCH SPARSE '(' expr ',' common_sparse_array_expr ',' STRING ',' LONG_VALUE ')' with_index_param_list */ +#line 2205 "parser.y" { auto match_sparse_expr = new infinity::MatchSparseExpr(); (yyval.expr_t) = match_sparse_expr; @@ -6127,11 +6143,11 @@ YYLTYPE yylloc = yyloc_default; // topn and options match_sparse_expr->SetOptParams((yyvsp[-2].long_value), (yyvsp[0].with_index_param_list_t)); } -#line 6131 "parser.cpp" +#line 6147 "parser.cpp" break; - case 288: /* match_sparse_expr: MATCH SPARSE '(' expr ',' common_sparse_array_expr ',' STRING ')' with_index_param_list */ -#line 2203 "parser.y" + case 289: /* match_sparse_expr: MATCH SPARSE '(' expr ',' common_sparse_array_expr ',' STRING ')' with_index_param_list */ +#line 2223 "parser.y" { auto match_sparse_expr = new infinity::MatchSparseExpr(); (yyval.expr_t) = match_sparse_expr; @@ -6149,11 +6165,11 @@ YYLTYPE yylloc = yyloc_default; // topn and options match_sparse_expr->SetOptParams(infinity::DEFAULT_MATCH_SPARSE_TOP_N, (yyvsp[0].with_index_param_list_t)); } -#line 6153 "parser.cpp" +#line 6169 "parser.cpp" break; - case 289: /* match_text_expr: MATCH TEXT '(' STRING ',' STRING ')' */ -#line 2221 "parser.y" + case 290: /* match_text_expr: MATCH TEXT '(' STRING ',' STRING ')' */ +#line 2241 "parser.y" { infinity::MatchExpr* match_text_expr = new infinity::MatchExpr(); match_text_expr->fields_ = std::string((yyvsp[-3].str_value)); @@ -6162,11 +6178,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = match_text_expr; } -#line 6166 "parser.cpp" +#line 6182 "parser.cpp" break; - case 290: /* match_text_expr: MATCH TEXT '(' STRING ',' STRING ',' STRING ')' */ -#line 2229 "parser.y" + case 291: /* match_text_expr: MATCH TEXT '(' STRING ',' STRING ',' STRING ')' */ +#line 2249 "parser.y" { infinity::MatchExpr* match_text_expr = new infinity::MatchExpr(); match_text_expr->fields_ = std::string((yyvsp[-5].str_value)); @@ -6177,22 +6193,22 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = match_text_expr; } -#line 6181 "parser.cpp" +#line 6197 "parser.cpp" break; - case 291: /* query_expr: QUERY '(' STRING ')' */ -#line 2240 "parser.y" + case 292: /* query_expr: QUERY '(' STRING ')' */ +#line 2260 "parser.y" { infinity::MatchExpr* match_text_expr = new infinity::MatchExpr(); match_text_expr->matching_text_ = std::string((yyvsp[-1].str_value)); free((yyvsp[-1].str_value)); (yyval.expr_t) = match_text_expr; } -#line 6192 "parser.cpp" +#line 6208 "parser.cpp" break; - case 292: /* query_expr: QUERY '(' STRING ',' STRING ')' */ -#line 2246 "parser.y" + case 293: /* query_expr: QUERY '(' STRING ',' STRING ')' */ +#line 2266 "parser.y" { infinity::MatchExpr* match_text_expr = new infinity::MatchExpr(); match_text_expr->matching_text_ = std::string((yyvsp[-3].str_value)); @@ -6201,22 +6217,22 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = match_text_expr; } -#line 6205 "parser.cpp" +#line 6221 "parser.cpp" break; - case 293: /* fusion_expr: FUSION '(' STRING ')' */ -#line 2255 "parser.y" + case 294: /* fusion_expr: FUSION '(' STRING ')' */ +#line 2275 "parser.y" { infinity::FusionExpr* fusion_expr = new infinity::FusionExpr(); fusion_expr->method_ = std::string((yyvsp[-1].str_value)); free((yyvsp[-1].str_value)); (yyval.expr_t) = fusion_expr; } -#line 6216 "parser.cpp" +#line 6232 "parser.cpp" break; - case 294: /* fusion_expr: FUSION '(' STRING ',' STRING ')' */ -#line 2261 "parser.y" + case 295: /* fusion_expr: FUSION '(' STRING ',' STRING ')' */ +#line 2281 "parser.y" { auto fusion_expr = std::make_unique(); fusion_expr->method_ = std::string((yyvsp[-3].str_value)); @@ -6228,77 +6244,77 @@ YYLTYPE yylloc = yyloc_default; fusion_expr->JobAfterParser(); (yyval.expr_t) = fusion_expr.release(); } -#line 6232 "parser.cpp" +#line 6248 "parser.cpp" break; - case 295: /* sub_search: match_vector_expr */ -#line 2273 "parser.y" + case 296: /* sub_search: match_vector_expr */ +#line 2293 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 6240 "parser.cpp" +#line 6256 "parser.cpp" break; - case 296: /* sub_search: match_text_expr */ -#line 2276 "parser.y" + case 297: /* sub_search: match_text_expr */ +#line 2296 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 6248 "parser.cpp" +#line 6264 "parser.cpp" break; - case 297: /* sub_search: match_tensor_expr */ -#line 2279 "parser.y" + case 298: /* sub_search: match_tensor_expr */ +#line 2299 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 6256 "parser.cpp" +#line 6272 "parser.cpp" break; - case 298: /* sub_search: match_sparse_expr */ -#line 2282 "parser.y" + case 299: /* sub_search: match_sparse_expr */ +#line 2302 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 6264 "parser.cpp" +#line 6280 "parser.cpp" break; - case 299: /* sub_search: query_expr */ -#line 2285 "parser.y" + case 300: /* sub_search: query_expr */ +#line 2305 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 6272 "parser.cpp" +#line 6288 "parser.cpp" break; - case 300: /* sub_search: fusion_expr */ -#line 2288 "parser.y" + case 301: /* sub_search: fusion_expr */ +#line 2308 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 6280 "parser.cpp" +#line 6296 "parser.cpp" break; - case 301: /* sub_search_array: sub_search */ -#line 2292 "parser.y" + case 302: /* sub_search_array: sub_search */ +#line 2312 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 6289 "parser.cpp" +#line 6305 "parser.cpp" break; - case 302: /* sub_search_array: sub_search_array ',' sub_search */ -#line 2296 "parser.y" + case 303: /* sub_search_array: sub_search_array ',' sub_search */ +#line 2316 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 6298 "parser.cpp" +#line 6314 "parser.cpp" break; - case 303: /* function_expr: IDENTIFIER '(' ')' */ -#line 2301 "parser.y" + case 304: /* function_expr: IDENTIFIER '(' ')' */ +#line 2321 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-2].str_value)); @@ -6307,11 +6323,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_ = nullptr; (yyval.expr_t) = func_expr; } -#line 6311 "parser.cpp" +#line 6327 "parser.cpp" break; - case 304: /* function_expr: IDENTIFIER '(' expr_array ')' */ -#line 2309 "parser.y" + case 305: /* function_expr: IDENTIFIER '(' expr_array ')' */ +#line 2329 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-3].str_value)); @@ -6320,11 +6336,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_ = (yyvsp[-1].expr_array_t); (yyval.expr_t) = func_expr; } -#line 6324 "parser.cpp" +#line 6340 "parser.cpp" break; - case 305: /* function_expr: IDENTIFIER '(' DISTINCT expr_array ')' */ -#line 2317 "parser.y" + case 306: /* function_expr: IDENTIFIER '(' DISTINCT expr_array ')' */ +#line 2337 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-4].str_value)); @@ -6334,11 +6350,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->distinct_ = true; (yyval.expr_t) = func_expr; } -#line 6338 "parser.cpp" +#line 6354 "parser.cpp" break; - case 306: /* function_expr: operand IS NOT NULLABLE */ -#line 2326 "parser.y" + case 307: /* function_expr: operand IS NOT NULLABLE */ +#line 2346 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "is_not_null"; @@ -6346,11 +6362,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[-3].expr_t)); (yyval.expr_t) = func_expr; } -#line 6350 "parser.cpp" +#line 6366 "parser.cpp" break; - case 307: /* function_expr: operand IS NULLABLE */ -#line 2333 "parser.y" + case 308: /* function_expr: operand IS NULLABLE */ +#line 2353 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "is_null"; @@ -6358,11 +6374,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[-2].expr_t)); (yyval.expr_t) = func_expr; } -#line 6362 "parser.cpp" +#line 6378 "parser.cpp" break; - case 308: /* function_expr: NOT operand */ -#line 2340 "parser.y" + case 309: /* function_expr: NOT operand */ +#line 2360 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "not"; @@ -6370,11 +6386,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6374 "parser.cpp" +#line 6390 "parser.cpp" break; - case 309: /* function_expr: '-' operand */ -#line 2347 "parser.y" + case 310: /* function_expr: '-' operand */ +#line 2367 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "-"; @@ -6382,11 +6398,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6386 "parser.cpp" +#line 6402 "parser.cpp" break; - case 310: /* function_expr: '+' operand */ -#line 2354 "parser.y" + case 311: /* function_expr: '+' operand */ +#line 2374 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "+"; @@ -6394,11 +6410,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6398 "parser.cpp" +#line 6414 "parser.cpp" break; - case 311: /* function_expr: operand '-' operand */ -#line 2361 "parser.y" + case 312: /* function_expr: operand '-' operand */ +#line 2381 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "-"; @@ -6407,11 +6423,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6411 "parser.cpp" +#line 6427 "parser.cpp" break; - case 312: /* function_expr: operand '+' operand */ -#line 2369 "parser.y" + case 313: /* function_expr: operand '+' operand */ +#line 2389 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "+"; @@ -6420,11 +6436,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6424 "parser.cpp" +#line 6440 "parser.cpp" break; - case 313: /* function_expr: operand '*' operand */ -#line 2377 "parser.y" + case 314: /* function_expr: operand '*' operand */ +#line 2397 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "*"; @@ -6433,11 +6449,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6437 "parser.cpp" +#line 6453 "parser.cpp" break; - case 314: /* function_expr: operand '/' operand */ -#line 2385 "parser.y" + case 315: /* function_expr: operand '/' operand */ +#line 2405 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "/"; @@ -6446,11 +6462,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6450 "parser.cpp" +#line 6466 "parser.cpp" break; - case 315: /* function_expr: operand '%' operand */ -#line 2393 "parser.y" + case 316: /* function_expr: operand '%' operand */ +#line 2413 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "%"; @@ -6459,11 +6475,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6463 "parser.cpp" +#line 6479 "parser.cpp" break; - case 316: /* function_expr: operand '=' operand */ -#line 2401 "parser.y" + case 317: /* function_expr: operand '=' operand */ +#line 2421 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "="; @@ -6472,11 +6488,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6476 "parser.cpp" +#line 6492 "parser.cpp" break; - case 317: /* function_expr: operand EQUAL operand */ -#line 2409 "parser.y" + case 318: /* function_expr: operand EQUAL operand */ +#line 2429 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "="; @@ -6485,11 +6501,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6489 "parser.cpp" +#line 6505 "parser.cpp" break; - case 318: /* function_expr: operand NOT_EQ operand */ -#line 2417 "parser.y" + case 319: /* function_expr: operand NOT_EQ operand */ +#line 2437 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "<>"; @@ -6498,11 +6514,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6502 "parser.cpp" +#line 6518 "parser.cpp" break; - case 319: /* function_expr: operand '<' operand */ -#line 2425 "parser.y" + case 320: /* function_expr: operand '<' operand */ +#line 2445 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "<"; @@ -6511,11 +6527,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6515 "parser.cpp" +#line 6531 "parser.cpp" break; - case 320: /* function_expr: operand '>' operand */ -#line 2433 "parser.y" + case 321: /* function_expr: operand '>' operand */ +#line 2453 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = ">"; @@ -6524,11 +6540,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6528 "parser.cpp" +#line 6544 "parser.cpp" break; - case 321: /* function_expr: operand LESS_EQ operand */ -#line 2441 "parser.y" + case 322: /* function_expr: operand LESS_EQ operand */ +#line 2461 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "<="; @@ -6537,11 +6553,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6541 "parser.cpp" +#line 6557 "parser.cpp" break; - case 322: /* function_expr: operand GREATER_EQ operand */ -#line 2449 "parser.y" + case 323: /* function_expr: operand GREATER_EQ operand */ +#line 2469 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = ">="; @@ -6550,11 +6566,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6554 "parser.cpp" +#line 6570 "parser.cpp" break; - case 323: /* function_expr: EXTRACT '(' STRING FROM operand ')' */ -#line 2457 "parser.y" + case 324: /* function_expr: EXTRACT '(' STRING FROM operand ')' */ +#line 2477 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-3].str_value)); @@ -6585,11 +6601,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[-1].expr_t)); (yyval.expr_t) = func_expr; } -#line 6589 "parser.cpp" +#line 6605 "parser.cpp" break; - case 324: /* function_expr: operand LIKE operand */ -#line 2487 "parser.y" + case 325: /* function_expr: operand LIKE operand */ +#line 2507 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "like"; @@ -6598,11 +6614,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6602 "parser.cpp" +#line 6618 "parser.cpp" break; - case 325: /* function_expr: operand NOT LIKE operand */ -#line 2495 "parser.y" + case 326: /* function_expr: operand NOT LIKE operand */ +#line 2515 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "not_like"; @@ -6611,11 +6627,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6615 "parser.cpp" +#line 6631 "parser.cpp" break; - case 326: /* conjunction_expr: expr AND expr */ -#line 2504 "parser.y" + case 327: /* conjunction_expr: expr AND expr */ +#line 2524 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "and"; @@ -6624,11 +6640,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6628 "parser.cpp" +#line 6644 "parser.cpp" break; - case 327: /* conjunction_expr: expr OR expr */ -#line 2512 "parser.y" + case 328: /* conjunction_expr: expr OR expr */ +#line 2532 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "or"; @@ -6637,11 +6653,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6641 "parser.cpp" +#line 6657 "parser.cpp" break; - case 328: /* between_expr: operand BETWEEN operand AND operand */ -#line 2521 "parser.y" + case 329: /* between_expr: operand BETWEEN operand AND operand */ +#line 2541 "parser.y" { infinity::BetweenExpr* between_expr = new infinity::BetweenExpr(); between_expr->value_ = (yyvsp[-4].expr_t); @@ -6649,44 +6665,44 @@ YYLTYPE yylloc = yyloc_default; between_expr->upper_bound_ = (yyvsp[0].expr_t); (yyval.expr_t) = between_expr; } -#line 6653 "parser.cpp" +#line 6669 "parser.cpp" break; - case 329: /* in_expr: operand IN '(' expr_array ')' */ -#line 2529 "parser.y" + case 330: /* in_expr: operand IN '(' expr_array ')' */ +#line 2549 "parser.y" { infinity::InExpr* in_expr = new infinity::InExpr(true); in_expr->left_ = (yyvsp[-4].expr_t); in_expr->arguments_ = (yyvsp[-1].expr_array_t); (yyval.expr_t) = in_expr; } -#line 6664 "parser.cpp" +#line 6680 "parser.cpp" break; - case 330: /* in_expr: operand NOT IN '(' expr_array ')' */ -#line 2535 "parser.y" + case 331: /* in_expr: operand NOT IN '(' expr_array ')' */ +#line 2555 "parser.y" { infinity::InExpr* in_expr = new infinity::InExpr(false); in_expr->left_ = (yyvsp[-5].expr_t); in_expr->arguments_ = (yyvsp[-1].expr_array_t); (yyval.expr_t) = in_expr; } -#line 6675 "parser.cpp" +#line 6691 "parser.cpp" break; - case 331: /* case_expr: CASE expr case_check_array END */ -#line 2542 "parser.y" + case 332: /* case_expr: CASE expr case_check_array END */ +#line 2562 "parser.y" { infinity::CaseExpr* case_expr = new infinity::CaseExpr(); case_expr->expr_ = (yyvsp[-2].expr_t); case_expr->case_check_array_ = (yyvsp[-1].case_check_array_t); (yyval.expr_t) = case_expr; } -#line 6686 "parser.cpp" +#line 6702 "parser.cpp" break; - case 332: /* case_expr: CASE expr case_check_array ELSE expr END */ -#line 2548 "parser.y" + case 333: /* case_expr: CASE expr case_check_array ELSE expr END */ +#line 2568 "parser.y" { infinity::CaseExpr* case_expr = new infinity::CaseExpr(); case_expr->expr_ = (yyvsp[-4].expr_t); @@ -6694,32 +6710,32 @@ YYLTYPE yylloc = yyloc_default; case_expr->else_expr_ = (yyvsp[-1].expr_t); (yyval.expr_t) = case_expr; } -#line 6698 "parser.cpp" +#line 6714 "parser.cpp" break; - case 333: /* case_expr: CASE case_check_array END */ -#line 2555 "parser.y" + case 334: /* case_expr: CASE case_check_array END */ +#line 2575 "parser.y" { infinity::CaseExpr* case_expr = new infinity::CaseExpr(); case_expr->case_check_array_ = (yyvsp[-1].case_check_array_t); (yyval.expr_t) = case_expr; } -#line 6708 "parser.cpp" +#line 6724 "parser.cpp" break; - case 334: /* case_expr: CASE case_check_array ELSE expr END */ -#line 2560 "parser.y" + case 335: /* case_expr: CASE case_check_array ELSE expr END */ +#line 2580 "parser.y" { infinity::CaseExpr* case_expr = new infinity::CaseExpr(); case_expr->case_check_array_ = (yyvsp[-3].case_check_array_t); case_expr->else_expr_ = (yyvsp[-1].expr_t); (yyval.expr_t) = case_expr; } -#line 6719 "parser.cpp" +#line 6735 "parser.cpp" break; - case 335: /* case_check_array: WHEN expr THEN expr */ -#line 2567 "parser.y" + case 336: /* case_check_array: WHEN expr THEN expr */ +#line 2587 "parser.y" { (yyval.case_check_array_t) = new std::vector(); infinity::WhenThen* when_then_ptr = new infinity::WhenThen(); @@ -6727,11 +6743,11 @@ YYLTYPE yylloc = yyloc_default; when_then_ptr->then_ = (yyvsp[0].expr_t); (yyval.case_check_array_t)->emplace_back(when_then_ptr); } -#line 6731 "parser.cpp" +#line 6747 "parser.cpp" break; - case 336: /* case_check_array: case_check_array WHEN expr THEN expr */ -#line 2574 "parser.y" + case 337: /* case_check_array: case_check_array WHEN expr THEN expr */ +#line 2594 "parser.y" { infinity::WhenThen* when_then_ptr = new infinity::WhenThen(); when_then_ptr->when_ = (yyvsp[-2].expr_t); @@ -6739,11 +6755,11 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-4].case_check_array_t)->emplace_back(when_then_ptr); (yyval.case_check_array_t) = (yyvsp[-4].case_check_array_t); } -#line 6743 "parser.cpp" +#line 6759 "parser.cpp" break; - case 337: /* cast_expr: CAST '(' expr AS column_type ')' */ -#line 2582 "parser.y" + case 338: /* cast_expr: CAST '(' expr AS column_type ')' */ +#line 2602 "parser.y" { std::shared_ptr type_info_ptr{nullptr}; switch((yyvsp[-1].column_type_t).logical_type_) { @@ -6767,33 +6783,33 @@ YYLTYPE yylloc = yyloc_default; cast_expr->expr_ = (yyvsp[-3].expr_t); (yyval.expr_t) = cast_expr; } -#line 6771 "parser.cpp" +#line 6787 "parser.cpp" break; - case 338: /* subquery_expr: EXISTS '(' select_without_paren ')' */ -#line 2606 "parser.y" + case 339: /* subquery_expr: EXISTS '(' select_without_paren ')' */ +#line 2626 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kExists; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 6782 "parser.cpp" +#line 6798 "parser.cpp" break; - case 339: /* subquery_expr: NOT EXISTS '(' select_without_paren ')' */ -#line 2612 "parser.y" + case 340: /* subquery_expr: NOT EXISTS '(' select_without_paren ')' */ +#line 2632 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kNotExists; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 6793 "parser.cpp" +#line 6809 "parser.cpp" break; - case 340: /* subquery_expr: operand IN '(' select_without_paren ')' */ -#line 2618 "parser.y" + case 341: /* subquery_expr: operand IN '(' select_without_paren ')' */ +#line 2638 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kIn; @@ -6801,11 +6817,11 @@ YYLTYPE yylloc = yyloc_default; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 6805 "parser.cpp" +#line 6821 "parser.cpp" break; - case 341: /* subquery_expr: operand NOT IN '(' select_without_paren ')' */ -#line 2625 "parser.y" + case 342: /* subquery_expr: operand NOT IN '(' select_without_paren ')' */ +#line 2645 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kNotIn; @@ -6813,11 +6829,11 @@ YYLTYPE yylloc = yyloc_default; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 6817 "parser.cpp" +#line 6833 "parser.cpp" break; - case 342: /* column_expr: IDENTIFIER */ -#line 2633 "parser.y" + case 343: /* column_expr: IDENTIFIER */ +#line 2653 "parser.y" { infinity::ColumnExpr* column_expr = new infinity::ColumnExpr(); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -6825,11 +6841,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].str_value)); (yyval.expr_t) = column_expr; } -#line 6829 "parser.cpp" +#line 6845 "parser.cpp" break; - case 343: /* column_expr: column_expr '.' IDENTIFIER */ -#line 2640 "parser.y" + case 344: /* column_expr: column_expr '.' IDENTIFIER */ +#line 2660 "parser.y" { infinity::ColumnExpr* column_expr = (infinity::ColumnExpr*)(yyvsp[-2].expr_t); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -6837,21 +6853,21 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].str_value)); (yyval.expr_t) = column_expr; } -#line 6841 "parser.cpp" +#line 6857 "parser.cpp" break; - case 344: /* column_expr: '*' */ -#line 2647 "parser.y" + case 345: /* column_expr: '*' */ +#line 2667 "parser.y" { infinity::ColumnExpr* column_expr = new infinity::ColumnExpr(); column_expr->star_ = true; (yyval.expr_t) = column_expr; } -#line 6851 "parser.cpp" +#line 6867 "parser.cpp" break; - case 345: /* column_expr: column_expr '.' '*' */ -#line 2652 "parser.y" + case 346: /* column_expr: column_expr '.' '*' */ +#line 2672 "parser.y" { infinity::ColumnExpr* column_expr = (infinity::ColumnExpr*)(yyvsp[-2].expr_t); if(column_expr->star_) { @@ -6861,232 +6877,232 @@ YYLTYPE yylloc = yyloc_default; column_expr->star_ = true; (yyval.expr_t) = column_expr; } -#line 6865 "parser.cpp" +#line 6881 "parser.cpp" break; - case 346: /* constant_expr: STRING */ -#line 2662 "parser.y" + case 347: /* constant_expr: STRING */ +#line 2682 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kString); const_expr->str_value_ = (yyvsp[0].str_value); (yyval.const_expr_t) = const_expr; } -#line 6875 "parser.cpp" +#line 6891 "parser.cpp" break; - case 347: /* constant_expr: TRUE */ -#line 2667 "parser.y" + case 348: /* constant_expr: TRUE */ +#line 2687 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kBoolean); const_expr->bool_value_ = true; (yyval.const_expr_t) = const_expr; } -#line 6885 "parser.cpp" +#line 6901 "parser.cpp" break; - case 348: /* constant_expr: FALSE */ -#line 2672 "parser.y" + case 349: /* constant_expr: FALSE */ +#line 2692 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kBoolean); const_expr->bool_value_ = false; (yyval.const_expr_t) = const_expr; } -#line 6895 "parser.cpp" +#line 6911 "parser.cpp" break; - case 349: /* constant_expr: DOUBLE_VALUE */ -#line 2677 "parser.y" + case 350: /* constant_expr: DOUBLE_VALUE */ +#line 2697 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kDouble); const_expr->double_value_ = (yyvsp[0].double_value); (yyval.const_expr_t) = const_expr; } -#line 6905 "parser.cpp" +#line 6921 "parser.cpp" break; - case 350: /* constant_expr: LONG_VALUE */ -#line 2682 "parser.y" + case 351: /* constant_expr: LONG_VALUE */ +#line 2702 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInteger); const_expr->integer_value_ = (yyvsp[0].long_value); (yyval.const_expr_t) = const_expr; } -#line 6915 "parser.cpp" +#line 6931 "parser.cpp" break; - case 351: /* constant_expr: DATE STRING */ -#line 2687 "parser.y" + case 352: /* constant_expr: DATE STRING */ +#line 2707 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kDate); const_expr->date_value_ = (yyvsp[0].str_value); (yyval.const_expr_t) = const_expr; } -#line 6925 "parser.cpp" +#line 6941 "parser.cpp" break; - case 352: /* constant_expr: TIME STRING */ -#line 2692 "parser.y" + case 353: /* constant_expr: TIME STRING */ +#line 2712 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kTime); const_expr->date_value_ = (yyvsp[0].str_value); (yyval.const_expr_t) = const_expr; } -#line 6935 "parser.cpp" +#line 6951 "parser.cpp" break; - case 353: /* constant_expr: DATETIME STRING */ -#line 2697 "parser.y" + case 354: /* constant_expr: DATETIME STRING */ +#line 2717 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kDateTime); const_expr->date_value_ = (yyvsp[0].str_value); (yyval.const_expr_t) = const_expr; } -#line 6945 "parser.cpp" +#line 6961 "parser.cpp" break; - case 354: /* constant_expr: TIMESTAMP STRING */ -#line 2702 "parser.y" + case 355: /* constant_expr: TIMESTAMP STRING */ +#line 2722 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kTimestamp); const_expr->date_value_ = (yyvsp[0].str_value); (yyval.const_expr_t) = const_expr; } -#line 6955 "parser.cpp" +#line 6971 "parser.cpp" break; - case 355: /* constant_expr: INTERVAL interval_expr */ -#line 2707 "parser.y" + case 356: /* constant_expr: INTERVAL interval_expr */ +#line 2727 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6963 "parser.cpp" +#line 6979 "parser.cpp" break; - case 356: /* constant_expr: interval_expr */ -#line 2710 "parser.y" + case 357: /* constant_expr: interval_expr */ +#line 2730 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6971 "parser.cpp" +#line 6987 "parser.cpp" break; - case 357: /* constant_expr: common_array_expr */ -#line 2713 "parser.y" + case 358: /* constant_expr: common_array_expr */ +#line 2733 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6979 "parser.cpp" +#line 6995 "parser.cpp" break; - case 358: /* common_array_expr: array_expr */ -#line 2717 "parser.y" + case 359: /* common_array_expr: array_expr */ +#line 2737 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6987 "parser.cpp" +#line 7003 "parser.cpp" break; - case 359: /* common_array_expr: subarray_array_expr */ -#line 2720 "parser.y" + case 360: /* common_array_expr: subarray_array_expr */ +#line 2740 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6995 "parser.cpp" +#line 7011 "parser.cpp" break; - case 360: /* common_array_expr: sparse_array_expr */ -#line 2723 "parser.y" + case 361: /* common_array_expr: sparse_array_expr */ +#line 2743 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7003 "parser.cpp" +#line 7019 "parser.cpp" break; - case 361: /* common_array_expr: empty_array_expr */ -#line 2726 "parser.y" + case 362: /* common_array_expr: empty_array_expr */ +#line 2746 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7011 "parser.cpp" +#line 7027 "parser.cpp" break; - case 362: /* common_sparse_array_expr: sparse_array_expr */ -#line 2730 "parser.y" + case 363: /* common_sparse_array_expr: sparse_array_expr */ +#line 2750 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7019 "parser.cpp" +#line 7035 "parser.cpp" break; - case 363: /* common_sparse_array_expr: array_expr */ -#line 2733 "parser.y" + case 364: /* common_sparse_array_expr: array_expr */ +#line 2753 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7027 "parser.cpp" +#line 7043 "parser.cpp" break; - case 364: /* common_sparse_array_expr: empty_array_expr */ -#line 2736 "parser.y" + case 365: /* common_sparse_array_expr: empty_array_expr */ +#line 2756 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7035 "parser.cpp" +#line 7051 "parser.cpp" break; - case 365: /* subarray_array_expr: unclosed_subarray_array_expr ']' */ -#line 2740 "parser.y" + case 366: /* subarray_array_expr: unclosed_subarray_array_expr ']' */ +#line 2760 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 7043 "parser.cpp" +#line 7059 "parser.cpp" break; - case 366: /* unclosed_subarray_array_expr: '[' common_array_expr */ -#line 2744 "parser.y" + case 367: /* unclosed_subarray_array_expr: '[' common_array_expr */ +#line 2764 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kSubArrayArray); const_expr->sub_array_array_.emplace_back((yyvsp[0].const_expr_t)); (yyval.const_expr_t) = const_expr; } -#line 7053 "parser.cpp" +#line 7069 "parser.cpp" break; - case 367: /* unclosed_subarray_array_expr: unclosed_subarray_array_expr ',' common_array_expr */ -#line 2749 "parser.y" + case 368: /* unclosed_subarray_array_expr: unclosed_subarray_array_expr ',' common_array_expr */ +#line 2769 "parser.y" { (yyvsp[-2].const_expr_t)->sub_array_array_.emplace_back((yyvsp[0].const_expr_t)); (yyval.const_expr_t) = (yyvsp[-2].const_expr_t); } -#line 7062 "parser.cpp" +#line 7078 "parser.cpp" break; - case 368: /* sparse_array_expr: long_sparse_array_expr */ -#line 2754 "parser.y" + case 369: /* sparse_array_expr: long_sparse_array_expr */ +#line 2774 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7070 "parser.cpp" +#line 7086 "parser.cpp" break; - case 369: /* sparse_array_expr: double_sparse_array_expr */ -#line 2757 "parser.y" + case 370: /* sparse_array_expr: double_sparse_array_expr */ +#line 2777 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7078 "parser.cpp" +#line 7094 "parser.cpp" break; - case 370: /* long_sparse_array_expr: unclosed_long_sparse_array_expr ']' */ -#line 2761 "parser.y" + case 371: /* long_sparse_array_expr: unclosed_long_sparse_array_expr ']' */ +#line 2781 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 7086 "parser.cpp" +#line 7102 "parser.cpp" break; - case 371: /* unclosed_long_sparse_array_expr: '[' int_sparse_ele */ -#line 2765 "parser.y" + case 372: /* unclosed_long_sparse_array_expr: '[' int_sparse_ele */ +#line 2785 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kLongSparseArray); const_expr->long_sparse_array_.first.emplace_back((yyvsp[0].int_sparse_ele_t)->first); @@ -7094,30 +7110,30 @@ YYLTYPE yylloc = yyloc_default; delete (yyvsp[0].int_sparse_ele_t); (yyval.const_expr_t) = const_expr; } -#line 7098 "parser.cpp" +#line 7114 "parser.cpp" break; - case 372: /* unclosed_long_sparse_array_expr: unclosed_long_sparse_array_expr ',' int_sparse_ele */ -#line 2772 "parser.y" + case 373: /* unclosed_long_sparse_array_expr: unclosed_long_sparse_array_expr ',' int_sparse_ele */ +#line 2792 "parser.y" { (yyvsp[-2].const_expr_t)->long_sparse_array_.first.emplace_back((yyvsp[0].int_sparse_ele_t)->first); (yyvsp[-2].const_expr_t)->long_sparse_array_.second.emplace_back((yyvsp[0].int_sparse_ele_t)->second); delete (yyvsp[0].int_sparse_ele_t); (yyval.const_expr_t) = (yyvsp[-2].const_expr_t); } -#line 7109 "parser.cpp" +#line 7125 "parser.cpp" break; - case 373: /* double_sparse_array_expr: unclosed_double_sparse_array_expr ']' */ -#line 2779 "parser.y" + case 374: /* double_sparse_array_expr: unclosed_double_sparse_array_expr ']' */ +#line 2799 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 7117 "parser.cpp" +#line 7133 "parser.cpp" break; - case 374: /* unclosed_double_sparse_array_expr: '[' float_sparse_ele */ -#line 2783 "parser.y" + case 375: /* unclosed_double_sparse_array_expr: '[' float_sparse_ele */ +#line 2803 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kDoubleSparseArray); const_expr->double_sparse_array_.first.emplace_back((yyvsp[0].float_sparse_ele_t)->first); @@ -7125,266 +7141,266 @@ YYLTYPE yylloc = yyloc_default; delete (yyvsp[0].float_sparse_ele_t); (yyval.const_expr_t) = const_expr; } -#line 7129 "parser.cpp" +#line 7145 "parser.cpp" break; - case 375: /* unclosed_double_sparse_array_expr: unclosed_double_sparse_array_expr ',' float_sparse_ele */ -#line 2790 "parser.y" + case 376: /* unclosed_double_sparse_array_expr: unclosed_double_sparse_array_expr ',' float_sparse_ele */ +#line 2810 "parser.y" { (yyvsp[-2].const_expr_t)->double_sparse_array_.first.emplace_back((yyvsp[0].float_sparse_ele_t)->first); (yyvsp[-2].const_expr_t)->double_sparse_array_.second.emplace_back((yyvsp[0].float_sparse_ele_t)->second); delete (yyvsp[0].float_sparse_ele_t); (yyval.const_expr_t) = (yyvsp[-2].const_expr_t); } -#line 7140 "parser.cpp" +#line 7156 "parser.cpp" break; - case 376: /* empty_array_expr: '[' ']' */ -#line 2797 "parser.y" + case 377: /* empty_array_expr: '[' ']' */ +#line 2817 "parser.y" { (yyval.const_expr_t) = new infinity::ConstantExpr(infinity::LiteralType::kEmptyArray); } -#line 7148 "parser.cpp" +#line 7164 "parser.cpp" break; - case 377: /* int_sparse_ele: LONG_VALUE ':' LONG_VALUE */ -#line 2801 "parser.y" + case 378: /* int_sparse_ele: LONG_VALUE ':' LONG_VALUE */ +#line 2821 "parser.y" { (yyval.int_sparse_ele_t) = new std::pair{(yyvsp[-2].long_value), (yyvsp[0].long_value)}; } -#line 7156 "parser.cpp" +#line 7172 "parser.cpp" break; - case 378: /* float_sparse_ele: LONG_VALUE ':' DOUBLE_VALUE */ -#line 2805 "parser.y" + case 379: /* float_sparse_ele: LONG_VALUE ':' DOUBLE_VALUE */ +#line 2825 "parser.y" { (yyval.float_sparse_ele_t) = new std::pair{(yyvsp[-2].long_value), (yyvsp[0].double_value)}; } -#line 7164 "parser.cpp" +#line 7180 "parser.cpp" break; - case 379: /* array_expr: long_array_expr */ -#line 2809 "parser.y" + case 380: /* array_expr: long_array_expr */ +#line 2829 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7172 "parser.cpp" +#line 7188 "parser.cpp" break; - case 380: /* array_expr: double_array_expr */ -#line 2812 "parser.y" + case 381: /* array_expr: double_array_expr */ +#line 2832 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 7180 "parser.cpp" +#line 7196 "parser.cpp" break; - case 381: /* long_array_expr: unclosed_long_array_expr ']' */ -#line 2816 "parser.y" + case 382: /* long_array_expr: unclosed_long_array_expr ']' */ +#line 2836 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 7188 "parser.cpp" +#line 7204 "parser.cpp" break; - case 382: /* unclosed_long_array_expr: '[' LONG_VALUE */ -#line 2820 "parser.y" + case 383: /* unclosed_long_array_expr: '[' LONG_VALUE */ +#line 2840 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kIntegerArray); const_expr->long_array_.emplace_back((yyvsp[0].long_value)); (yyval.const_expr_t) = const_expr; } -#line 7198 "parser.cpp" +#line 7214 "parser.cpp" break; - case 383: /* unclosed_long_array_expr: unclosed_long_array_expr ',' LONG_VALUE */ -#line 2825 "parser.y" + case 384: /* unclosed_long_array_expr: unclosed_long_array_expr ',' LONG_VALUE */ +#line 2845 "parser.y" { (yyvsp[-2].const_expr_t)->long_array_.emplace_back((yyvsp[0].long_value)); (yyval.const_expr_t) = (yyvsp[-2].const_expr_t); } -#line 7207 "parser.cpp" +#line 7223 "parser.cpp" break; - case 384: /* double_array_expr: unclosed_double_array_expr ']' */ -#line 2830 "parser.y" + case 385: /* double_array_expr: unclosed_double_array_expr ']' */ +#line 2850 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 7215 "parser.cpp" +#line 7231 "parser.cpp" break; - case 385: /* unclosed_double_array_expr: '[' DOUBLE_VALUE */ -#line 2834 "parser.y" + case 386: /* unclosed_double_array_expr: '[' DOUBLE_VALUE */ +#line 2854 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kDoubleArray); const_expr->double_array_.emplace_back((yyvsp[0].double_value)); (yyval.const_expr_t) = const_expr; } -#line 7225 "parser.cpp" +#line 7241 "parser.cpp" break; - case 386: /* unclosed_double_array_expr: unclosed_double_array_expr ',' DOUBLE_VALUE */ -#line 2839 "parser.y" + case 387: /* unclosed_double_array_expr: unclosed_double_array_expr ',' DOUBLE_VALUE */ +#line 2859 "parser.y" { (yyvsp[-2].const_expr_t)->double_array_.emplace_back((yyvsp[0].double_value)); (yyval.const_expr_t) = (yyvsp[-2].const_expr_t); } -#line 7234 "parser.cpp" +#line 7250 "parser.cpp" break; - case 387: /* interval_expr: LONG_VALUE SECONDS */ -#line 2844 "parser.y" + case 388: /* interval_expr: LONG_VALUE SECONDS */ +#line 2864 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kSecond; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7245 "parser.cpp" +#line 7261 "parser.cpp" break; - case 388: /* interval_expr: LONG_VALUE SECOND */ -#line 2850 "parser.y" + case 389: /* interval_expr: LONG_VALUE SECOND */ +#line 2870 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kSecond; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7256 "parser.cpp" +#line 7272 "parser.cpp" break; - case 389: /* interval_expr: LONG_VALUE MINUTES */ -#line 2856 "parser.y" + case 390: /* interval_expr: LONG_VALUE MINUTES */ +#line 2876 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kMinute; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7267 "parser.cpp" +#line 7283 "parser.cpp" break; - case 390: /* interval_expr: LONG_VALUE MINUTE */ -#line 2862 "parser.y" + case 391: /* interval_expr: LONG_VALUE MINUTE */ +#line 2882 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kMinute; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7278 "parser.cpp" +#line 7294 "parser.cpp" break; - case 391: /* interval_expr: LONG_VALUE HOURS */ -#line 2868 "parser.y" + case 392: /* interval_expr: LONG_VALUE HOURS */ +#line 2888 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kHour; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7289 "parser.cpp" +#line 7305 "parser.cpp" break; - case 392: /* interval_expr: LONG_VALUE HOUR */ -#line 2874 "parser.y" + case 393: /* interval_expr: LONG_VALUE HOUR */ +#line 2894 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kHour; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7300 "parser.cpp" +#line 7316 "parser.cpp" break; - case 393: /* interval_expr: LONG_VALUE DAYS */ -#line 2880 "parser.y" + case 394: /* interval_expr: LONG_VALUE DAYS */ +#line 2900 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kDay; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7311 "parser.cpp" +#line 7327 "parser.cpp" break; - case 394: /* interval_expr: LONG_VALUE DAY */ -#line 2886 "parser.y" + case 395: /* interval_expr: LONG_VALUE DAY */ +#line 2906 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kDay; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7322 "parser.cpp" +#line 7338 "parser.cpp" break; - case 395: /* interval_expr: LONG_VALUE MONTHS */ -#line 2892 "parser.y" + case 396: /* interval_expr: LONG_VALUE MONTHS */ +#line 2912 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kMonth; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7333 "parser.cpp" +#line 7349 "parser.cpp" break; - case 396: /* interval_expr: LONG_VALUE MONTH */ -#line 2898 "parser.y" + case 397: /* interval_expr: LONG_VALUE MONTH */ +#line 2918 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kMonth; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7344 "parser.cpp" +#line 7360 "parser.cpp" break; - case 397: /* interval_expr: LONG_VALUE YEARS */ -#line 2904 "parser.y" + case 398: /* interval_expr: LONG_VALUE YEARS */ +#line 2924 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kYear; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7355 "parser.cpp" +#line 7371 "parser.cpp" break; - case 398: /* interval_expr: LONG_VALUE YEAR */ -#line 2910 "parser.y" + case 399: /* interval_expr: LONG_VALUE YEAR */ +#line 2930 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kInterval); const_expr->interval_type_ = infinity::TimeUnit::kYear; const_expr->integer_value_ = (yyvsp[-1].long_value); (yyval.const_expr_t) = const_expr; } -#line 7366 "parser.cpp" +#line 7382 "parser.cpp" break; - case 399: /* copy_option_list: copy_option */ -#line 2921 "parser.y" + case 400: /* copy_option_list: copy_option */ +#line 2941 "parser.y" { (yyval.copy_option_array) = new std::vector(); (yyval.copy_option_array)->push_back((yyvsp[0].copy_option_t)); } -#line 7375 "parser.cpp" +#line 7391 "parser.cpp" break; - case 400: /* copy_option_list: copy_option_list ',' copy_option */ -#line 2925 "parser.y" + case 401: /* copy_option_list: copy_option_list ',' copy_option */ +#line 2945 "parser.y" { (yyvsp[-2].copy_option_array)->push_back((yyvsp[0].copy_option_t)); (yyval.copy_option_array) = (yyvsp[-2].copy_option_array); } -#line 7384 "parser.cpp" +#line 7400 "parser.cpp" break; - case 401: /* copy_option: FORMAT IDENTIFIER */ -#line 2930 "parser.y" + case 402: /* copy_option: FORMAT IDENTIFIER */ +#line 2950 "parser.y" { (yyval.copy_option_t) = new infinity::CopyOption(); (yyval.copy_option_t)->option_type_ = infinity::CopyOptionType::kFormat; @@ -7413,11 +7429,11 @@ YYLTYPE yylloc = yyloc_default; YYERROR; } } -#line 7417 "parser.cpp" +#line 7433 "parser.cpp" break; - case 402: /* copy_option: DELIMITER STRING */ -#line 2958 "parser.y" + case 403: /* copy_option: DELIMITER STRING */ +#line 2978 "parser.y" { (yyval.copy_option_t) = new infinity::CopyOption(); (yyval.copy_option_t)->option_type_ = infinity::CopyOptionType::kDelimiter; @@ -7428,53 +7444,53 @@ YYLTYPE yylloc = yyloc_default; } free((yyvsp[0].str_value)); } -#line 7432 "parser.cpp" +#line 7448 "parser.cpp" break; - case 403: /* copy_option: HEADER */ -#line 2968 "parser.y" + case 404: /* copy_option: HEADER */ +#line 2988 "parser.y" { (yyval.copy_option_t) = new infinity::CopyOption(); (yyval.copy_option_t)->option_type_ = infinity::CopyOptionType::kHeader; (yyval.copy_option_t)->header_ = true; } -#line 7442 "parser.cpp" +#line 7458 "parser.cpp" break; - case 404: /* file_path: STRING */ -#line 2974 "parser.y" + case 405: /* file_path: STRING */ +#line 2994 "parser.y" { (yyval.str_value) = (yyvsp[0].str_value); } -#line 7450 "parser.cpp" +#line 7466 "parser.cpp" break; - case 405: /* if_exists: IF EXISTS */ -#line 2978 "parser.y" + case 406: /* if_exists: IF EXISTS */ +#line 2998 "parser.y" { (yyval.bool_value) = true; } -#line 7456 "parser.cpp" +#line 7472 "parser.cpp" break; - case 406: /* if_exists: %empty */ -#line 2979 "parser.y" + case 407: /* if_exists: %empty */ +#line 2999 "parser.y" { (yyval.bool_value) = false; } -#line 7462 "parser.cpp" +#line 7478 "parser.cpp" break; - case 407: /* if_not_exists: IF NOT EXISTS */ -#line 2981 "parser.y" + case 408: /* if_not_exists: IF NOT EXISTS */ +#line 3001 "parser.y" { (yyval.bool_value) = true; } -#line 7468 "parser.cpp" +#line 7484 "parser.cpp" break; - case 408: /* if_not_exists: %empty */ -#line 2982 "parser.y" + case 409: /* if_not_exists: %empty */ +#line 3002 "parser.y" { (yyval.bool_value) = false; } -#line 7474 "parser.cpp" +#line 7490 "parser.cpp" break; - case 411: /* if_not_exists_info: if_not_exists IDENTIFIER */ -#line 2997 "parser.y" + case 412: /* if_not_exists_info: if_not_exists IDENTIFIER */ +#line 3017 "parser.y" { (yyval.if_not_exists_info_t) = new infinity::IfNotExistsInfo(); (yyval.if_not_exists_info_t)->exists_ = true; @@ -7483,80 +7499,80 @@ YYLTYPE yylloc = yyloc_default; (yyval.if_not_exists_info_t)->info_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 7487 "parser.cpp" +#line 7503 "parser.cpp" break; - case 412: /* if_not_exists_info: %empty */ -#line 3005 "parser.y" + case 413: /* if_not_exists_info: %empty */ +#line 3025 "parser.y" { (yyval.if_not_exists_info_t) = new infinity::IfNotExistsInfo(); } -#line 7495 "parser.cpp" +#line 7511 "parser.cpp" break; - case 413: /* with_index_param_list: WITH '(' index_param_list ')' */ -#line 3009 "parser.y" + case 414: /* with_index_param_list: WITH '(' index_param_list ')' */ +#line 3029 "parser.y" { (yyval.with_index_param_list_t) = (yyvsp[-1].index_param_list_t); } -#line 7503 "parser.cpp" +#line 7519 "parser.cpp" break; - case 414: /* with_index_param_list: %empty */ -#line 3012 "parser.y" + case 415: /* with_index_param_list: %empty */ +#line 3032 "parser.y" { (yyval.with_index_param_list_t) = new std::vector(); } -#line 7511 "parser.cpp" +#line 7527 "parser.cpp" break; - case 415: /* optional_table_properties_list: PROPERTIES '(' index_param_list ')' */ -#line 3016 "parser.y" + case 416: /* optional_table_properties_list: PROPERTIES '(' index_param_list ')' */ +#line 3036 "parser.y" { (yyval.with_index_param_list_t) = (yyvsp[-1].index_param_list_t); } -#line 7519 "parser.cpp" +#line 7535 "parser.cpp" break; - case 416: /* optional_table_properties_list: %empty */ -#line 3019 "parser.y" + case 417: /* optional_table_properties_list: %empty */ +#line 3039 "parser.y" { (yyval.with_index_param_list_t) = nullptr; } -#line 7527 "parser.cpp" +#line 7543 "parser.cpp" break; - case 417: /* index_param_list: index_param */ -#line 3023 "parser.y" + case 418: /* index_param_list: index_param */ +#line 3043 "parser.y" { (yyval.index_param_list_t) = new std::vector(); (yyval.index_param_list_t)->push_back((yyvsp[0].index_param_t)); } -#line 7536 "parser.cpp" +#line 7552 "parser.cpp" break; - case 418: /* index_param_list: index_param_list ',' index_param */ -#line 3027 "parser.y" + case 419: /* index_param_list: index_param_list ',' index_param */ +#line 3047 "parser.y" { (yyvsp[-2].index_param_list_t)->push_back((yyvsp[0].index_param_t)); (yyval.index_param_list_t) = (yyvsp[-2].index_param_list_t); } -#line 7545 "parser.cpp" +#line 7561 "parser.cpp" break; - case 419: /* index_param: IDENTIFIER */ -#line 3032 "parser.y" + case 420: /* index_param: IDENTIFIER */ +#line 3052 "parser.y" { ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 7556 "parser.cpp" +#line 7572 "parser.cpp" break; - case 420: /* index_param: IDENTIFIER '=' IDENTIFIER */ -#line 3038 "parser.y" + case 421: /* index_param: IDENTIFIER '=' IDENTIFIER */ +#line 3058 "parser.y" { ParserHelper::ToLower((yyvsp[-2].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -7567,11 +7583,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.index_param_t)->param_value_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 7571 "parser.cpp" +#line 7587 "parser.cpp" break; - case 421: /* index_param: IDENTIFIER '=' LONG_VALUE */ -#line 3048 "parser.y" + case 422: /* index_param: IDENTIFIER '=' LONG_VALUE */ +#line 3068 "parser.y" { (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[-2].str_value); @@ -7579,11 +7595,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.index_param_t)->param_value_ = std::to_string((yyvsp[0].long_value)); } -#line 7583 "parser.cpp" +#line 7599 "parser.cpp" break; - case 422: /* index_param: IDENTIFIER '=' DOUBLE_VALUE */ -#line 3055 "parser.y" + case 423: /* index_param: IDENTIFIER '=' DOUBLE_VALUE */ +#line 3075 "parser.y" { (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[-2].str_value); @@ -7591,11 +7607,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.index_param_t)->param_value_ = std::to_string((yyvsp[0].double_value)); } -#line 7595 "parser.cpp" +#line 7611 "parser.cpp" break; - case 423: /* index_info_list: '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ -#line 3066 "parser.y" + case 424: /* index_info_list: '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ +#line 3086 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); infinity::IndexType index_type = infinity::IndexType::kInvalid; @@ -7648,11 +7664,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-4].identifier_array_t); } -#line 7652 "parser.cpp" +#line 7668 "parser.cpp" break; - case 424: /* index_info_list: index_info_list '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ -#line 3118 "parser.y" + case 425: /* index_info_list: index_info_list '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ +#line 3138 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); infinity::IndexType index_type = infinity::IndexType::kInvalid; @@ -7702,11 +7718,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-4].identifier_array_t); } -#line 7706 "parser.cpp" +#line 7722 "parser.cpp" break; - case 425: /* index_info_list: '(' identifier_array ')' */ -#line 3167 "parser.y" + case 426: /* index_info_list: '(' identifier_array ')' */ +#line 3187 "parser.y" { infinity::IndexType index_type = infinity::IndexType::kSecondary; size_t index_count = (yyvsp[-1].identifier_array_t)->size(); @@ -7720,11 +7736,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].identifier_array_t); } -#line 7724 "parser.cpp" +#line 7740 "parser.cpp" break; -#line 7728 "parser.cpp" +#line 7744 "parser.cpp" default: break; } @@ -7953,7 +7969,7 @@ YYLTYPE yylloc = yyloc_default; return yyresult; } -#line 3181 "parser.y" +#line 3201 "parser.y" void diff --git a/src/parser/parser.y b/src/parser/parser.y index 0d4734ca0d..e91073e3d5 100644 --- a/src/parser/parser.y +++ b/src/parser/parser.y @@ -1857,6 +1857,26 @@ optimize_statement: OPTIMIZE table_name { $$->table_name_ = $2->table_name_ptr_; free($2->table_name_ptr_); delete $2; +} +// 1 2 3 4 5 +| OPTIMIZE IDENTIFIER ON table_name with_index_param_list{ + $$ = new infinity::OptimizeStatement(); + if($4->schema_name_ptr_ != nullptr) { + $$->schema_name_ = $4->schema_name_ptr_; + free($4->schema_name_ptr_); + } + $$->table_name_ = $4->table_name_ptr_; + free($4->table_name_ptr_); + delete $4; + + $$->index_name_ = $2; + free($2); + + for (auto *&index_param : *$5) { + $$->opt_params_.emplace_back(std::unique_ptr(index_param)); + index_param = nullptr; + } + delete $5; }; /* diff --git a/src/parser/statement/optimize_statement.h b/src/parser/statement/optimize_statement.h index 5a303d9e46..d90b5ef0b8 100644 --- a/src/parser/statement/optimize_statement.h +++ b/src/parser/statement/optimize_statement.h @@ -33,6 +33,9 @@ class OptimizeStatement : public BaseStatement { std::string schema_name_{}; std::string table_name_{}; + + std::string index_name_{}; + std::vector> opt_params_{}; }; } // namespace infinity \ No newline at end of file diff --git a/src/planner/logical_planner.cpp b/src/planner/logical_planner.cpp index a28edf1368..53a3d08277 100644 --- a/src/planner/logical_planner.cpp +++ b/src/planner/logical_planner.cpp @@ -1615,8 +1615,11 @@ Status LogicalPlanner::BuildFlushBuffer(const FlushStatement *, SharedPtr &bind_context_ptr) { BindSchemaName(statement->schema_name_); - SharedPtr logical_optimize = - MakeShared(bind_context_ptr->GetNewLogicalNodeId(), query_context_ptr_->schema_name(), statement->table_name_); + SharedPtr logical_optimize = MakeShared(bind_context_ptr->GetNewLogicalNodeId(), + query_context_ptr_->schema_name(), + statement->table_name_, + std::move(statement->index_name_), + std::move(statement->opt_params_)); this->logical_plan_ = logical_optimize; return Status::OK(); } diff --git a/src/planner/node/logical_optimize.cppm b/src/planner/node/logical_optimize.cppm index f5b3a57896..73b456686e 100644 --- a/src/planner/node/logical_optimize.cppm +++ b/src/planner/node/logical_optimize.cppm @@ -23,13 +23,15 @@ import logical_node; import data_type; import internal_types; import optimize_statement; +import statement_common; namespace infinity { export class LogicalOptimize : public LogicalNode { public: - explicit LogicalOptimize(u64 node_id, String schema_name, String table_name) - : LogicalNode(node_id, LogicalNodeType::kOptimize), schema_name_(std::move(schema_name)), table_name_(std::move(table_name)) {} + explicit LogicalOptimize(u64 node_id, String schema_name, String table_name, String index_name, Vector> opt_params) + : LogicalNode(node_id, LogicalNodeType::kOptimize), schema_name_(std::move(schema_name)), table_name_(std::move(table_name)), + index_name_(std::move(index_name)), opt_params_(std::move(opt_params)) {} [[nodiscard]] Vector GetColumnBindings() const final; @@ -48,6 +50,10 @@ public: private: String schema_name_; String table_name_; + +public: + String index_name_; + Vector> opt_params_; }; } // namespace infinity diff --git a/src/scheduler/fragment_context.cpp b/src/scheduler/fragment_context.cpp index b7f40c375d..7fe7a9c489 100644 --- a/src/scheduler/fragment_context.cpp +++ b/src/scheduler/fragment_context.cpp @@ -128,7 +128,7 @@ UniquePtr MakeMatchSparseScanState(const PhysicalMatchSparseScan auto operator_state = MakeUnique(); auto *match_sparse_scan_source_state = static_cast(source_state); operator_state->match_sparse_scan_function_data_ = - MatchSparseScanFunctionData(physical_match_sparse_scan->GetBlockIndex(), match_sparse_scan_source_state->global_ids_); + MatchSparseScanFunctionData(match_sparse_scan_source_state->global_ids_, match_sparse_scan_source_state->segment_ids_); return operator_state; } @@ -978,8 +978,9 @@ void FragmentContext::MakeSourceState(i64 parallel_count) { } auto *match_sparse_scan_operator = static_cast(first_operator); Vector>> blocks_group = match_sparse_scan_operator->PlanBlockEntries(parallel_count); + Vector>> segment_group = match_sparse_scan_operator->PlanWithIndex(blocks_group, parallel_count); for (i64 task_id = 0; task_id < parallel_count; ++task_id) { - tasks_[task_id]->source_state_ = MakeUnique(std::move(blocks_group[task_id])); + tasks_[task_id]->source_state_ = MakeUnique(std::move(blocks_group[task_id]), segment_group[task_id]); } break; } diff --git a/src/storage/buffer/file_worker/bmp_index_file_worker.cpp b/src/storage/buffer/file_worker/bmp_index_file_worker.cpp index 3ca2b30143..43f89e752a 100644 --- a/src/storage/buffer/file_worker/bmp_index_file_worker.cpp +++ b/src/storage/buffer/file_worker/bmp_index_file_worker.cpp @@ -176,7 +176,8 @@ void BMPIndexFileWorker::ReadFromFileImpl() { if constexpr (std::is_same_v) { UnrecoverableError("Invalid index type."); } else { - index->Load(*file_handler_); + using IndexT = std::decay_t; + data_ = reinterpret_cast(new IndexT(IndexT::Load(*file_handler_))); } }, index); diff --git a/src/storage/common/block_index.cpp b/src/storage/common/block_index.cpp index 9049078cbe..b7b62c08e6 100644 --- a/src/storage/common/block_index.cpp +++ b/src/storage/common/block_index.cpp @@ -82,9 +82,9 @@ void IndexIndex::Insert(String index_name, SharedPtr index_snapsh index_snapshots_.emplace(std::move(index_name), index_snapshot); } -void IndexIndex::Insert(TableIndexEntry *table_index_entry, Txn *txn) { +SharedPtr IndexIndex::Insert(TableIndexEntry *table_index_entry, Txn *txn) { if (table_index_entry->CheckVisible(txn)) { - auto index_snapshot = MakeUnique(); + auto index_snapshot = MakeShared(); index_snapshot->table_index_entry_ = table_index_entry; SegmentIndexesGuard segment_index_guard = table_index_entry->GetSegmentIndexesGuard(); @@ -95,8 +95,10 @@ void IndexIndex::Insert(TableIndexEntry *table_index_entry, Txn *txn) { } String index_name = *table_index_entry->GetIndexName(); - Insert(std::move(index_name), std::move(index_snapshot)); + Insert(std::move(index_name), index_snapshot); + return index_snapshot; } + return nullptr; } void IndexIndex::Insert(TableIndexEntry *table_index_entry, SegmentIndexEntry *segment_index_entry) { diff --git a/src/storage/common/block_index.cppm b/src/storage/common/block_index.cppm index b2c96e6472..dcf89e582b 100644 --- a/src/storage/common/block_index.cppm +++ b/src/storage/common/block_index.cppm @@ -69,7 +69,7 @@ export struct IndexSnapshot { export struct IndexIndex { public: - void Insert(TableIndexEntry *table_index_entry, Txn *txn); + SharedPtr Insert(TableIndexEntry *table_index_entry, Txn *txn); void Insert(String index_name, SharedPtr index_snapshot); diff --git a/src/storage/knn_index/sparse/bmp_alg.cpp b/src/storage/knn_index/sparse/bmp_alg.cpp index 8d8eec5835..9303d35e44 100644 --- a/src/storage/knn_index/sparse/bmp_alg.cpp +++ b/src/storage/knn_index/sparse/bmp_alg.cpp @@ -16,12 +16,12 @@ module; #include #include +#include module bmp_alg; import infinity_exception; import third_party; -import knn_result_handler; import serialize; import segment_iter; @@ -50,7 +50,7 @@ void BMPIvt::Optimize(i32 topk, Vector> if ((i32)term_scores.size() < topk) { continue; } - std::partial_sort(term_scores.begin(), term_scores.begin() + topk - 1, term_scores.end(), std::greater<>()); + std::nth_element(term_scores.begin(), term_scores.begin() + topk - 1, term_scores.end(), std::greater<>()); posting.kth_score_ = term_scores[topk - 1]; } } @@ -207,6 +207,8 @@ template class BlockFwd; template void BMPAlg::AddDoc(const SparseVecRef &doc, BMPDocID doc_id) { + std::unique_lock lock(mtx_); + doc_ids_.push_back(doc_id); Optional> tail_fwd = block_fwd_.AddDoc(doc); if (!tail_fwd.has_value()) { @@ -218,85 +220,18 @@ void BMPAlg::AddDoc(const SparseVecRef -void BMPAlg::Optimize(i32 topk) { +void BMPAlg::Optimize(const BMPOptimizeOptions &options) { + std::unique_lock lock(mtx_); + SizeT term_num = bm_ivt_.term_num(); Vector> ivt_scores = block_fwd_.GetIvtScores(term_num); - bm_ivt_.Optimize(topk, std::move(ivt_scores)); + bm_ivt_.Optimize(options.topk_, std::move(ivt_scores)); } template Pair, Vector> -BMPAlg::SearchKnn(const SparseVecRef &query, i32 topk, f32 alpha, f32 beta) const { - SizeT block_size = block_fwd_.block_size(); - SparseVecEle keeped_query; - if (beta < 1.0) { - i32 terms_to_keep = std::ceil(query.nnz_ * beta); - Vector query_term_idxes(query.nnz_); - std::iota(query_term_idxes.begin(), query_term_idxes.end(), 0); - std::partial_sort(query_term_idxes.begin(), query_term_idxes.begin() + terms_to_keep, query_term_idxes.end(), [&](SizeT a, SizeT b) { - return query.data_[a] > query.data_[b]; - }); - query_term_idxes.resize(terms_to_keep); - std::sort(query_term_idxes.begin(), query_term_idxes.end(), [&](SizeT a, SizeT b) { return query.indices_[a] < query.indices_[b]; }); - - keeped_query.Init(query_term_idxes, query.data_, query.indices_); - } - const SparseVecRef &query_ref = - beta < 1.0 ? SparseVecRef(keeped_query.nnz_, keeped_query.indices_.get(), keeped_query.data_.get()) : query; - - const auto &postings = bm_ivt_.GetPostings(); - DataType threshold = 0.0; - SizeT block_num = block_fwd_.block_num(); - Vector upper_bounds(block_num, 0.0); - for (i32 i = 0; i < query_ref.nnz_; ++i) { - IdxType query_term = query_ref.indices_[i]; - DataType query_score = query_ref.data_[i]; - const auto &posting = postings[query_term]; - threshold = std::max(threshold, query_score * posting.kth(topk)); - posting.data_.Calculate(upper_bounds, query_score); - } - - Vector> block_scores; - for (SizeT block_id = 0; block_id < block_num; ++block_id) { - if (upper_bounds[block_id] >= threshold) { - block_scores.emplace_back(upper_bounds[block_id], block_id); - } - } - std::sort(block_scores.begin(), block_scores.end(), [](const auto &a, const auto &b) { return a.first > b.first; }); - - Vector result(topk); - Vector result_score(topk); - HeapResultHandler> result_handler(1 /*query_n*/, topk, result_score.data(), result.data()); - - SizeT block_scores_num = block_scores.size(); - for (SizeT i = 0; i < block_scores_num; ++i) { - if (i + 1 < block_scores_num) { - BMPBlockID next_block_id = block_scores[i + 1].second; - block_fwd_.Prefetch(next_block_id); - } - const auto &[ub_score, block_id] = block_scores[i]; - BMPDocID off = block_id * block_size; - Vector scores = block_fwd_.GetScores(block_id, query_ref); - for (SizeT block_off = 0; block_off < scores.size(); ++block_off) { - BMPDocID doc_id = off + block_off; - DataType score = scores[block_off]; - result_handler.AddResult(0 /*query_id*/, score, doc_id); - } - if (ub_score * alpha < result_handler.GetDistance0(0 /*query_id*/)) { - break; - } - } - Vector tail_scores = block_fwd_.GetScoresTail(query_ref); - for (SizeT i = 0; i < tail_scores.size(); ++i) { - BMPDocID doc_id = block_num * block_size + i; - DataType score = tail_scores[i]; - result_handler.AddResult(0 /*query_id*/, score, doc_id); - } - - result_handler.End(0 /*query_id*/); - Vector result_docid(topk); - std::transform(result.begin(), result.end(), result_docid.begin(), [&](BMPDocID doc_id) { return doc_ids_[doc_id]; }); - return {result_docid, result_score}; +BMPAlg::SearchKnn(const SparseVecRef &query, i32 topk, const BmpSearchOptions &options) const { + return SearchKnn(query, topk, options, nullptr); } template class BMPAlg; diff --git a/src/storage/knn_index/sparse/bmp_alg.cppm b/src/storage/knn_index/sparse/bmp_alg.cppm index 5edb38d391..258a56b66c 100644 --- a/src/storage/knn_index/sparse/bmp_alg.cppm +++ b/src/storage/knn_index/sparse/bmp_alg.cppm @@ -22,6 +22,7 @@ import file_system; import bm_posting; import bmp_util; import hnsw_common; +import knn_result_handler; namespace infinity { @@ -130,9 +131,13 @@ public: template , BMPDocID> Iterator> SizeT AddDocs(Iterator iter); - void Optimize(i32 topk); + void Optimize(const BMPOptimizeOptions &options); - Pair, Vector> SearchKnn(const SparseVecRef &query, i32 topk, f32 alpha, f32 beta) const; + Pair, Vector> SearchKnn(const SparseVecRef &query, i32 topk, const BmpSearchOptions &options) const; + + template Filter = NoneType> + Pair, Vector> + SearchKnn(const SparseVecRef &query, i32 topk, const BmpSearchOptions &options, const Filter &filter) const; void Save(FileHandler &file_handler) const; @@ -149,6 +154,8 @@ private: BMPIvt bm_ivt_; BlockFwd block_fwd_; Vector doc_ids_; + + mutable std::shared_mutex mtx_; }; template @@ -167,4 +174,100 @@ SizeT BMPAlg::AddDocs(Iterator iter) { return cnt; } +template +template Filter> +Pair, Vector> BMPAlg::SearchKnn(const SparseVecRef &query, + i32 topk, + const BmpSearchOptions &options, + const Filter &filter) const { + std::shared_lock lock(mtx_, std::defer_lock); + if (options.use_lock_) { + lock.lock(); + } + + SizeT block_size = block_fwd_.block_size(); + SparseVecEle keeped_query; + if (options.beta_ < 1.0) { + i32 terms_to_keep = std::ceil(query.nnz_ * options.beta_); + Vector query_term_idxes(query.nnz_); + std::iota(query_term_idxes.begin(), query_term_idxes.end(), 0); + std::partial_sort(query_term_idxes.begin(), query_term_idxes.begin() + terms_to_keep, query_term_idxes.end(), [&](SizeT a, SizeT b) { + return query.data_[a] > query.data_[b]; + }); + query_term_idxes.resize(terms_to_keep); + std::sort(query_term_idxes.begin(), query_term_idxes.end(), [&](SizeT a, SizeT b) { return query.indices_[a] < query.indices_[b]; }); + + keeped_query.Init(query_term_idxes, query.data_, query.indices_); + } + const SparseVecRef &query_ref = + options.beta_ < 1.0 ? SparseVecRef(keeped_query.nnz_, keeped_query.indices_.get(), keeped_query.data_.get()) : query; + + const auto &postings = bm_ivt_.GetPostings(); + DataType threshold = 0.0; + SizeT block_num = block_fwd_.block_num(); + Vector upper_bounds(block_num, 0.0); + for (i32 i = 0; i < query_ref.nnz_; ++i) { + IdxType query_term = query_ref.indices_[i]; + DataType query_score = query_ref.data_[i]; + const auto &posting = postings[query_term]; + threshold = std::max(threshold, query_score * posting.kth(topk)); + posting.data_.Calculate(upper_bounds, query_score); + } + + Vector> block_scores; + for (SizeT block_id = 0; block_id < block_num; ++block_id) { + if (upper_bounds[block_id] >= threshold) { + block_scores.emplace_back(upper_bounds[block_id], block_id); + } + } + std::sort(block_scores.begin(), block_scores.end(), [](const auto &a, const auto &b) { return a.first > b.first; }); + + Vector result(topk); + Vector result_score(topk); + HeapResultHandler> result_handler(1 /*query_n*/, topk, result_score.data(), result.data()); + + auto add_result = [&](DataType score, BMPDocID doc_id) { + if constexpr (std::is_same_v) { + result_handler.AddResult(0 /*query_id*/, score, doc_id); + } else { + if (filter(doc_ids_[doc_id])) { + result_handler.AddResult(0 /*query_id*/, score, doc_id); + } + } + }; + + SizeT block_scores_num = block_scores.size(); + for (SizeT i = 0; i < block_scores_num; ++i) { + if (i + 1 < block_scores_num) { + BMPBlockID next_block_id = block_scores[i + 1].second; + block_fwd_.Prefetch(next_block_id); + } + const auto &[ub_score, block_id] = block_scores[i]; + BMPDocID off = block_id * block_size; + Vector scores = block_fwd_.GetScores(block_id, query_ref); + for (SizeT block_off = 0; block_off < scores.size(); ++block_off) { + BMPDocID doc_id = off + block_off; + DataType score = scores[block_off]; + add_result(score, doc_id); + } + if (ub_score * options.alpha_ < result_handler.GetDistance0(0 /*query_id*/)) { + break; + } + } + + if (options.use_tail_) { + Vector tail_scores = block_fwd_.GetScoresTail(query_ref); + for (SizeT i = 0; i < tail_scores.size(); ++i) { + BMPDocID doc_id = block_num * block_size + i; + DataType score = tail_scores[i]; + add_result(score, doc_id); + } + } + + result_handler.End(0 /*query_id*/); + Vector result_docid(topk); + std::transform(result.begin(), result.end(), result_docid.begin(), [&](BMPDocID doc_id) { return doc_ids_[doc_id]; }); + return {result_docid, result_score}; +} + } // namespace infinity diff --git a/src/storage/knn_index/sparse/bmp_alg_serialize.cpp b/src/storage/knn_index/sparse/bmp_alg_serialize.cpp index 9ac30dfeae..73cb2b6314 100644 --- a/src/storage/knn_index/sparse/bmp_alg_serialize.cpp +++ b/src/storage/knn_index/sparse/bmp_alg_serialize.cpp @@ -201,6 +201,8 @@ BMPAlg BMPAlg: template SizeT BMPAlg::GetSizeInBytes() const { + std::shared_lock lock(mtx_); + SizeT size = 0; size += bm_ivt_.GetSizeInBytes(); size += block_fwd_.GetSizeInBytes(); @@ -211,6 +213,8 @@ SizeT BMPAlg::GetSizeInBytes() const { template void BMPAlg::WriteAdv(char *&p) const { + std::shared_lock lock(mtx_); + bm_ivt_.WriteAdv(p); block_fwd_.WriteAdv(p); SizeT doc_num = doc_ids_.size(); diff --git a/src/storage/knn_index/sparse/bmp_posting.cpp b/src/storage/knn_index/sparse/bmp_posting.cpp index fb06ef8a41..94bc3ddd2a 100644 --- a/src/storage/knn_index/sparse/bmp_posting.cpp +++ b/src/storage/knn_index/sparse/bmp_posting.cpp @@ -24,8 +24,8 @@ namespace infinity { template void BlockData::Calculate(Vector &upper_bounds, DataType query_score) const { - // MultiF32StoreI32(block_ids_.data(), max_scores_.data(), upper_bounds.data(), query_score, block_ids_.size()); - for (SizeT i = 0; i < block_ids_.size(); ++i) { + SizeT block_size = block_ids_.size(); + for (SizeT i = 0; i < block_size; ++i) { BMPBlockID block_id = block_ids_[i]; DataType score = max_scores_[i]; upper_bounds[block_id] += score * query_score; diff --git a/src/storage/knn_index/sparse/bmp_util.cpp b/src/storage/knn_index/sparse/bmp_util.cpp new file mode 100644 index 0000000000..bd6f6de9de --- /dev/null +++ b/src/storage/knn_index/sparse/bmp_util.cpp @@ -0,0 +1,81 @@ +// Copyright(C) 2023 InfiniFlow, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://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. + +module; + +#include +#include + +module bmp_util; + +import logger; +import third_party; + +namespace infinity { + +BmpSearchOptions BMPUtil::ParseBmpSearchOptions(const Vector> &opt_params) { + BmpSearchOptions options; + for (const auto &opt_param : opt_params) { + if (opt_param->param_name_ == "alpha") { + f32 alpha = std::stof(opt_param->param_value_); + if (alpha < 0.0 || alpha > 1.0) { + LOG_WARN("Invalid alpha value, should be in [0, 1]"); + continue; + } + options.alpha_ = alpha; + } else if (opt_param->param_name_ == "beta") { + f32 beta = std::stof(opt_param->param_value_); + if (beta < 0.0 || beta > 1.0) { + LOG_WARN("Invalid beta value, should be in [0, 1]"); + continue; + } + options.beta_ = beta; + } else if (opt_param->param_name_ == "use_tail") { + if (!IsEqual(opt_param->param_value_, "T") && !IsEqual(opt_param->param_value_, "F")) { + LOG_WARN("Invalid use_tail value, should be T or F"); + continue; + } + options.use_tail_ = IsEqual(opt_param->param_value_, "T"); + } else if (opt_param->param_name_ == "use_lock") { + if (!IsEqual(opt_param->param_value_, "T") && !IsEqual(opt_param->param_value_, "F")) { + LOG_WARN("Invalid use_lock value, should be T or F"); + continue; + } + options.use_lock_ = IsEqual(opt_param->param_value_, "T"); + } + } + return options; +}; + +Optional BMPUtil::ParseBMPOptimizeOptions(const Vector> &opt_params) { + BMPOptimizeOptions options; + for (const auto &opt_param : opt_params) { + if (opt_param->param_name_ == "topk") { + i32 topk = std::stoi(opt_param->param_value_); + if (topk <= 0) { + continue; + } + if (topk > 1000) { + LOG_WARN(fmt::format("topk value is large {}", topk)); + } + options.topk_ = topk; + } + } + if (options.topk_ == 0) { + return None; + } + return options; +} + +} // namespace infinity \ No newline at end of file diff --git a/src/storage/knn_index/sparse/bmp_util.cppm b/src/storage/knn_index/sparse/bmp_util.cppm index 76a61faadd..19f9eb1067 100644 --- a/src/storage/knn_index/sparse/bmp_util.cppm +++ b/src/storage/knn_index/sparse/bmp_util.cppm @@ -17,6 +17,7 @@ module; export module bmp_util; import stl; +import statement_common; namespace infinity { @@ -54,5 +55,22 @@ export String BMCompressTypeToString(BMPCompressType compress_type) { } } } +export struct BmpSearchOptions { + f32 alpha_ = 1.0; + f32 beta_ = 1.0; + bool use_tail_ = true; + bool use_lock_ = true; +}; + +export struct BMPOptimizeOptions { + i32 topk_ = 0; +}; + +export class BMPUtil { +public: + static BmpSearchOptions ParseBmpSearchOptions(const Vector> &opt_params); + + static Optional ParseBMPOptimizeOptions(const Vector> &opt_params); +}; } // namespace infinity \ No newline at end of file diff --git a/src/storage/meta/entry/segment_index_entry.cpp b/src/storage/meta/entry/segment_index_entry.cpp index f82274e98a..a7f1ac53f6 100644 --- a/src/storage/meta/entry/segment_index_entry.cpp +++ b/src/storage/meta/entry/segment_index_entry.cpp @@ -62,6 +62,7 @@ import txn_store; import secondary_index_in_mem; import emvb_index; import emvb_index_in_mem; +import bmp_util; namespace infinity { @@ -797,6 +798,46 @@ void SegmentIndexEntry::CommitOptimize(ChunkIndexEntry *new_chunk, const Vector< // LOG_INFO(ss.str()); } +void SegmentIndexEntry::OptimizeIndex(Txn *txn, const Vector> &opt_params) { + switch (table_index_entry_->index_base()->index_type_) { + case IndexType::kBMP: { + Optional ret = BMPUtil::ParseBMPOptimizeOptions(opt_params); + if (!ret) { + return; + } + const auto &options = ret.value(); + const auto [chunk_index_entries, memory_index_entry] = this->GetBMPIndexSnapshot(); + + auto optimize_index = [&](AbstractBMP index) { + std::visit( + [&](auto &index) { + using T = std::decay_t; + if constexpr (std::is_same_v) { + UnrecoverableError("Invalid index type."); + } else { + index->Optimize(options); + } + }, + index); + }; + for (const auto &chunk_index_entry : chunk_index_entries) { + BufferHandle buffer_handle = chunk_index_entry->GetIndex(); + auto abstract_bmp = static_cast(buffer_handle.GetFileWorkerMut())->GetAbstractIndex(); + optimize_index(abstract_bmp); + } + if (memory_index_entry.get() != nullptr) { + BufferHandle buffer_handle = memory_index_entry->GetIndex(); + auto abstract_bmp = static_cast(buffer_handle.GetFileWorkerMut())->GetAbstractIndex(); + optimize_index(abstract_bmp); + } + break; + } + default: { + UnrecoverableError("Not implemented"); + } + } +} + bool SegmentIndexEntry::Flush(TxnTimeStamp checkpoint_ts) { auto index_type = table_index_entry_->index_base()->index_type_; if (index_type == IndexType::kFullText || index_type == IndexType::kHnsw) { diff --git a/src/storage/meta/entry/segment_index_entry.cppm b/src/storage/meta/entry/segment_index_entry.cppm index 342fca9539..91ab0f88d7 100644 --- a/src/storage/meta/entry/segment_index_entry.cppm +++ b/src/storage/meta/entry/segment_index_entry.cppm @@ -33,6 +33,7 @@ import cleanup_scanner; import chunk_index_entry; import memory_indexer; import default_values; +import statement_common; namespace infinity { @@ -95,6 +96,8 @@ public: void CommitOptimize(ChunkIndexEntry *new_chunk, const Vector &old_chunks, TxnTimeStamp commit_ts); + void OptimizeIndex(Txn *txn, const Vector> &opt_params); + bool Flush(TxnTimeStamp checkpoint_ts); void Cleanup() final; diff --git a/src/storage/meta/entry/table_index_entry.cpp b/src/storage/meta/entry/table_index_entry.cpp index 001e55ed56..f5aed13061 100644 --- a/src/storage/meta/entry/table_index_entry.cpp +++ b/src/storage/meta/entry/table_index_entry.cpp @@ -393,4 +393,17 @@ void TableIndexEntry::UpdateEntryReplay(TransactionID txn_id, TxnTimeStamp begin txn_id_ = txn_id; } +void TableIndexEntry::OptimizeIndex(Txn *txn, const Vector> &opt_params) { + Vector segment_indexes; + { + SegmentIndexesGuard segment_indexes_guard = this->GetSegmentIndexesGuard(); + for (const auto &[segment_id, segment_index_entry] : segment_indexes_guard.index_by_segment_) { + segment_indexes.push_back(segment_index_entry.get()); + } + } + for (auto *segment_index_entry : segment_indexes) { + segment_index_entry->OptimizeIndex(txn, opt_params); + } +} + } // namespace infinity diff --git a/src/storage/meta/entry/table_index_entry.cppm b/src/storage/meta/entry/table_index_entry.cppm index 365d5f7fbe..7a99f6f8d1 100644 --- a/src/storage/meta/entry/table_index_entry.cppm +++ b/src/storage/meta/entry/table_index_entry.cppm @@ -27,7 +27,7 @@ import block_index; import third_party; import status; import random; - +import statement_common; import cleanup_scanner; import meta_entry_interface; import index_file_worker; @@ -139,6 +139,8 @@ public: // replay void UpdateEntryReplay(TransactionID txn_id, TxnTimeStamp begin_ts, TxnTimeStamp commit_ts); + void OptimizeIndex(Txn *txn, const Vector> &opt_params); + private: static SharedPtr DetermineIndexDir(const String &parent_dir, const String &index_name) { return DetermineRandomString(parent_dir, fmt::format("index_{}", index_name)); diff --git a/src/unit_test/storage/knnindex/knn_sparse/test_bmp_index.cpp b/src/unit_test/storage/knnindex/knn_sparse/test_bmp_index.cpp index 53132f1e3d..cb2eba542c 100644 --- a/src/unit_test/storage/knnindex/knn_sparse/test_bmp_index.cpp +++ b/src/unit_test/storage/knnindex/knn_sparse/test_bmp_index.cpp @@ -46,8 +46,8 @@ class BMPIndexTest : public BaseTest { u32 topk = 10; // u32 gt_size = std::min(nrow, topk); - f32 alpha = 1.0; - f32 beta = 1.0; + BmpSearchOptions options; + options.use_lock_ = false; f32 accuracy_all = 0.9; @@ -64,7 +64,7 @@ class BMPIndexTest : public BaseTest { for (SparseMatrixIter iter(query_set); iter.HasNext(); iter.Next()) { SparseVecRef vec = iter.val(); - auto [indices, scores] = index.SearchKnn(vec, topk, alpha, beta); + auto [indices, scores] = index.SearchKnn(vec, topk, options); u32 query_id = iter.row_id(); const i32 *gt_indices = gt_indices_list.get() + query_id * topk; @@ -90,8 +90,9 @@ class BMPIndexTest : public BaseTest { index.AddDoc(vec, row_id); } + BMPOptimizeOptions optimize_options{.topk_ = static_cast(topk)}; test_query(index); - index.Optimize(topk); + index.Optimize(optimize_options); test_query(index); auto [file_handler, status] = fs.OpenFile(save_path, FileFlags::WRITE_FLAG | FileFlags::CREATE_FLAG, FileLockType::kNoLock); diff --git a/test/sql/dql/knn/sparse/test_knn_sparse_bmp.slt b/test/sql/dql/knn/sparse/test_knn_sparse_bmp.slt index 8c12a8aef9..9ed8df8f45 100644 --- a/test/sql/dql/knn/sparse/test_knn_sparse_bmp.slt +++ b/test/sql/dql/knn/sparse/test_knn_sparse_bmp.slt @@ -9,3 +9,57 @@ COPY test_knn_sparse_bmp FROM '/var/infinity/test_data/sparse_knn.csv' WITH (FOR statement ok CREATE INDEX idx1 ON test_knn_sparse_bmp (col2) USING Bmp WITH (block_size = 8, compress_type = compress); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +2 +1 + +statement ok +OPTIMIZE idx1 ON test_knn_sparse_bmp WITH (topk = 3); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +2 +1 + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3) WITH (alpha = 1.0, beta = 1.0, tail = T); +---- +4 +2 +1 + +statement ok +COPY test_knn_sparse_bmp FROM '/var/infinity/test_data/sparse_knn.csv' WITH (FORMAT CSV); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +2 + +statement ok +COPY test_knn_sparse_bmp FROM '/var/infinity/test_data/sparse_knn.csv' WITH (FORMAT CSV); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +4 + +statement ok +OPTIMIZE idx1 ON test_knn_sparse_bmp WITH (topk = 3); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +4 diff --git a/test/sql/dql/knn/sparse/test_knn_sparse_bmp_filter.slt b/test/sql/dql/knn/sparse/test_knn_sparse_bmp_filter.slt new file mode 100644 index 0000000000..5811c41e36 --- /dev/null +++ b/test/sql/dql/knn/sparse/test_knn_sparse_bmp_filter.slt @@ -0,0 +1,53 @@ +statement ok +DROP TABLE IF EXISTS test_knn_sparse_bmp_filter; + +statement ok +CREATE TABLE test_knn_sparse_bmp_filter (col1 INT, col2 SPARSE(FLOAT,100)); + +statement ok +CREATE INDEX idx1 ON test_knn_sparse_bmp_filter (col2) USING Bmp WITH (block_size = 8, compress_type = compress); + +statement ok +COPY test_knn_sparse_bmp_filter FROM '/var/infinity/test_data/sparse_knn.csv' WITH (FORMAT CSV); + +query I +SELECT col1 FROM test_knn_sparse_bmp_filter SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +2 +1 + +query I +SELECT col1 FROM test_knn_sparse_bmp_filter SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3) WHERE col1 < 4; +---- +2 +1 +3 + +statement ok +COPY test_knn_sparse_bmp_filter FROM '/var/infinity/test_data/sparse_knn.csv' WITH (FORMAT CSV); + +query I +SELECT col1 FROM test_knn_sparse_bmp_filter SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +2 + +query I +SELECT col1 FROM test_knn_sparse_bmp_filter SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3) where col1 < 4; +---- +2 +2 +1 + +statement ok +DELETE FROM test_knn_sparse_bmp_filter WHERE col1 = 4; + +query I +SELECT col1 FROM test_knn_sparse_bmp_filter SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +2 +2 +1 + diff --git a/test/sql/dql/knn/sparse/test_knn_sparse_bmp_realtime.slt b/test/sql/dql/knn/sparse/test_knn_sparse_bmp_realtime.slt index 5c48347021..b7e9e763e4 100644 --- a/test/sql/dql/knn/sparse/test_knn_sparse_bmp_realtime.slt +++ b/test/sql/dql/knn/sparse/test_knn_sparse_bmp_realtime.slt @@ -15,3 +15,58 @@ INSERT INTO test_knn_sparse_bmp VALUES (2,[0:2.0,20:2.0,40:2.0,60:2.0,80:2.0]), (3,[0:3.0,30:3.0,60:3.0,90:3.0]), (4,[0:4.0,40:4.0,80:4.0]), (5,[]); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +2 +1 + +statement ok +OPTIMIZE idx1 ON test_knn_sparse_bmp WITH (topk = 3); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +2 +1 + +statement ok +INSERT INTO test_knn_sparse_bmp VALUES (1,[0:1.0,10:1.0,20:1.0,30:1.0,40:1.0,50:1.0,60:1.0,70:1.0,80:1.0,90:1.0]), + (2,[0:2.0,20:2.0,40:2.0,60:2.0,80:2.0]), + (3,[0:3.0,30:3.0,60:3.0,90:3.0]), + (4,[0:4.0,40:4.0,80:4.0]), + (5,[]); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +2 + +statement ok +INSERT INTO test_knn_sparse_bmp VALUES (1,[0:1.0,10:1.0,20:1.0,30:1.0,40:1.0,50:1.0,60:1.0,70:1.0,80:1.0,90:1.0]), + (2,[0:2.0,20:2.0,40:2.0,60:2.0,80:2.0]), + (3,[0:3.0,30:3.0,60:3.0,90:3.0]), + (4,[0:4.0,40:4.0,80:4.0]), + (5,[]); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +4 + +statement ok +OPTIMIZE idx1 ON test_knn_sparse_bmp WITH (topk = 3); + +query I +SELECT col1 FROM test_knn_sparse_bmp SEARCH MATCH SPARSE (col2, [0:1.0,20:2.0,80:3.0], 'ip', 3); +---- +4 +4 +4