From 24404a5cb1ab6bb43d38a0699b3e5049c3f947aa Mon Sep 17 00:00:00 2001 From: shen yushi Date: Thu, 16 May 2024 16:04:37 +0800 Subject: [PATCH] Unified compaction. (#1208) ### What problem does this PR solve? 1. Refactor: implement compaction operator. 2. Refactor: use scheduler to execute compaction. 3. Refactor: fragment plan is a DAG now by connecting multiple plan tree. 4. Fix: Add delete to compaction todelete list in `SegmentEntry::Commit` instead of `SegmentEntry::DeleteData`, because it should be atomic with `SegmentEntry::max_row_ts_` update. Issue link:#1182 ### Type of change - [x] Bug Fix (non-breaking change which fixes an issue) - [x] Refactoring - [x] Test cases --- benchmark/embedding/CMakeLists.txt | 1 + src/executor/fragment/plan_fragment.cpp | 31 + src/executor/fragment/plan_fragment.cppm | 16 +- src/executor/fragment_builder.cpp | 105 +- src/executor/fragment_builder.cppm | 2 +- src/executor/operator/physical_command.cpp | 7 - src/executor/operator/physical_command.cppm | 2 - src/executor/operator/physical_compact.cpp | 213 + src/executor/operator/physical_compact.cppm | 66 + .../operator/physical_compact_finish.cpp | 95 + .../operator/physical_compact_finish.cppm | 75 + .../operator/physical_compact_index_do.cpp | 56 + .../operator/physical_compact_index_do.cppm | 61 + .../physical_compact_index_prepare.cpp | 68 + .../physical_compact_index_prepare.cppm | 63 + src/executor/operator/physical_import.cpp | 4 + src/executor/operator/physical_index_scan.cpp | 6 +- src/executor/operator/physical_knn_scan.cpp | 12 +- src/executor/operator/physical_match.cpp | 10 +- src/executor/operator/physical_sink.cpp | 32 +- src/executor/operator/physical_source.cpp | 3 +- src/executor/operator/physical_table_scan.cpp | 10 +- src/executor/operator_state.cppm | 52 +- src/executor/physical_operator_type.cpp | 8 + src/executor/physical_operator_type.cppm | 6 + src/executor/physical_planner.cpp | 114 +- src/executor/physical_planner.cppm | 7 + src/function/table/compact_state_data.cppm | 123 + src/function/table/create_index_data.cppm | 13 +- src/main/bg_query_state.cppm | 34 + src/main/query_context.cpp | 103 +- src/main/query_context.cppm | 15 +- src/parser/base_statement.h | 1 + src/parser/parser.cpp | 3523 +++++++++-------- src/parser/parser.h | 8 +- src/parser/parser.y | 23 +- src/parser/statement/command_statement.cpp | 1 - src/parser/statement/command_statement.cppm | 1 - src/parser/statement/command_statement.h | 16 - src/parser/statement/compact_statement.cppm | 26 + src/parser/statement/compact_statement.h | 62 + src/planner/bound/base_table_ref.cppm | 5 +- src/planner/bound_compact_statement.cppm | 93 + src/planner/logical_node_type.cppm | 5 + src/planner/logical_planner.cpp | 26 +- src/planner/logical_planner.cppm | 11 +- src/planner/node/logical_command.cpp | 5 - src/planner/node/logical_command.cppm | 3 - src/planner/node/logical_compact.cpp | 56 + src/planner/node/logical_compact.cppm | 50 + src/planner/node/logical_compact_finish.cpp | 56 + src/planner/node/logical_compact_finish.cppm | 49 + src/planner/node/logical_compact_index.cpp | 57 + src/planner/node/logical_compact_index.cppm | 47 + src/planner/query_binder.cpp | 25 +- src/planner/query_binder.cppm | 4 + src/scheduler/fragment_context.cpp | 193 +- src/scheduler/fragment_context.cppm | 10 +- src/scheduler/task_scheduler.cpp | 25 +- src/scheduler/task_scheduler.cppm | 2 - src/storage/background_process.cpp | 1 - src/storage/bg_task/compact_segments_task.cpp | 327 -- .../bg_task/compact_segments_task.cppm | 138 - .../update_segment_bloom_filter_task.cpp | 1 - src/storage/common/block_index.cpp | 73 +- src/storage/common/block_index.cppm | 57 +- src/storage/compaction/DBT_compaction_alg.cpp | 10 +- .../compaction/DBT_compaction_alg.cppm | 2 +- src/storage/compaction/compaction_alg.cppm | 10 +- src/storage/compaction_process.cpp | 114 +- src/storage/compaction_process.cppm | 16 +- .../invertedindex/search/query_builder.cpp | 4 +- src/storage/meta/catalog.cpp | 5 +- src/storage/meta/catalog.cppm | 7 +- src/storage/meta/entry/segment_entry.cpp | 87 +- src/storage/meta/entry/segment_entry.cppm | 11 +- src/storage/meta/entry/table_entry.cpp | 71 +- src/storage/meta/entry/table_entry.cppm | 11 +- src/storage/meta/entry/table_index_entry.cpp | 28 +- src/storage/meta/entry/table_index_entry.cppm | 12 +- .../secondary_index/common_query_filter.cpp | 7 +- src/storage/storage.cpp | 3 + src/storage/storage.cppm | 2 + src/storage/txn/txn.cpp | 12 +- src/storage/txn/txn.cppm | 5 +- src/storage/txn/txn_store.cpp | 14 +- src/storage/txn/txn_store.cppm | 9 +- src/storage/wal/wal_manager.cpp | 15 +- .../storage/bg_task/cleanup_task.cpp | 33 +- .../storage/bg_task/compact_segments_task.cpp | 353 +- .../storage/compaction/DBT_compaction_alg.cpp | 307 +- .../storage/wal/catalog_delta_replay.cpp | 34 +- src/unit_test/storage/wal/checkpoint.cpp | 17 +- src/unit_test/storage/wal/wal_replay.cpp | 15 +- src/unit_test/test_helper/sql_runner.cpp | 4 +- test/data/csv/embedding_2.csv | 4 + test/data/csv/embedding_3.csv | 4 + .../dml/compact/test_compact_many_index.slt | 75 + tools/generate_compact.py | 12 +- tools/generate_many_import.py | 29 +- 100 files changed, 4616 insertions(+), 3119 deletions(-) create mode 100644 src/executor/operator/physical_compact.cpp create mode 100644 src/executor/operator/physical_compact.cppm create mode 100644 src/executor/operator/physical_compact_finish.cpp create mode 100644 src/executor/operator/physical_compact_finish.cppm create mode 100644 src/executor/operator/physical_compact_index_do.cpp create mode 100644 src/executor/operator/physical_compact_index_do.cppm create mode 100644 src/executor/operator/physical_compact_index_prepare.cpp create mode 100644 src/executor/operator/physical_compact_index_prepare.cppm create mode 100644 src/function/table/compact_state_data.cppm create mode 100644 src/main/bg_query_state.cppm create mode 100644 src/parser/statement/compact_statement.cppm create mode 100644 src/parser/statement/compact_statement.h create mode 100644 src/planner/bound_compact_statement.cppm create mode 100644 src/planner/node/logical_compact.cpp create mode 100644 src/planner/node/logical_compact.cppm create mode 100644 src/planner/node/logical_compact_finish.cpp create mode 100644 src/planner/node/logical_compact_finish.cppm create mode 100644 src/planner/node/logical_compact_index.cpp create mode 100644 src/planner/node/logical_compact_index.cppm delete mode 100644 src/storage/bg_task/compact_segments_task.cpp delete mode 100644 src/storage/bg_task/compact_segments_task.cppm create mode 100644 test/data/csv/embedding_2.csv create mode 100644 test/data/csv/embedding_3.csv create mode 100644 test/sql/dml/compact/test_compact_many_index.slt diff --git a/benchmark/embedding/CMakeLists.txt b/benchmark/embedding/CMakeLists.txt index 31c06991c3..757de32167 100644 --- a/benchmark/embedding/CMakeLists.txt +++ b/benchmark/embedding/CMakeLists.txt @@ -22,6 +22,7 @@ target_link_libraries( ann_ivfflat_benchmark infinity_core sql_parser + zsv_parser benchmark_profiler onnxruntime_mlas newpfor diff --git a/src/executor/fragment/plan_fragment.cpp b/src/executor/fragment/plan_fragment.cpp index 72b62ba1f0..f37dc1ee3d 100644 --- a/src/executor/fragment/plan_fragment.cpp +++ b/src/executor/fragment/plan_fragment.cpp @@ -57,4 +57,35 @@ SharedPtr> PlanFragment::ToString() { SharedPtr PlanFragment::GetResult() { return context_->GetResult(); } +void PlanFragment::AddNext(SharedPtr root, PlanFragment *next) { + Vector next_leaves; + next->GetStartFragments(next_leaves); + for (auto &leaf : next_leaves) { + leaf->AddChild(root); + } +} + +SizeT PlanFragment::GetStartFragments(Vector &leaf_fragments) { + SizeT all_fragment_n = 0; + HashSet visited; + std::function TraversePlanFragmentGraph = [&](PlanFragment *fragment) { + if (visited.find(fragment) != visited.end()) { + return; + } + visited.insert(fragment); + if (fragment->GetContext()) { + all_fragment_n += fragment->GetContext()->Tasks().size(); + } + if (!fragment->HasChild()) { + leaf_fragments.emplace_back(fragment); + return; + } + for (auto &child : fragment->Children()) { + TraversePlanFragmentGraph(child.get()); + } + }; + TraversePlanFragmentGraph(this); + return all_fragment_n; +} + } // namespace infinity diff --git a/src/executor/fragment/plan_fragment.cppm b/src/executor/fragment/plan_fragment.cppm index f34833d519..0767ea2197 100644 --- a/src/executor/fragment/plan_fragment.cppm +++ b/src/executor/fragment/plan_fragment.cppm @@ -59,14 +59,14 @@ public: [[nodiscard]] inline PhysicalSink *GetSinkNode() const { return sink_.get(); } - [[nodiscard]] inline PlanFragment *GetParent() const { return parent_; } + [[nodiscard]] inline Vector GetParents() const { return parents_; } - inline void AddChild(UniquePtr child_fragment) { - child_fragment->parent_ = this; + inline void AddChild(SharedPtr child_fragment) { + child_fragment->parents_.emplace_back(this); children_.emplace_back(std::move(child_fragment)); } - inline Vector> &Children() { return children_; } + inline Vector> &Children() { return children_; } inline bool HasChild() { return !children_.empty(); } @@ -80,6 +80,10 @@ public: SharedPtr GetResult(); + static void AddNext(SharedPtr root, PlanFragment *next); + + SizeT GetStartFragments(Vector &leaf_fragments); + private: u64 fragment_id_{}; @@ -89,9 +93,9 @@ private: UniquePtr source_{}; - PlanFragment *parent_{}; + Vector parents_{}; - Vector> children_{}; + Vector> children_{}; UniquePtr context_{}; diff --git a/src/executor/fragment_builder.cpp b/src/executor/fragment_builder.cpp index 6df9139b22..904ede9438 100644 --- a/src/executor/fragment_builder.cpp +++ b/src/executor/fragment_builder.cpp @@ -14,6 +14,8 @@ module; +#include + module fragment_builder; import stl; @@ -35,14 +37,23 @@ import explain_statement; namespace infinity { -UniquePtr FragmentBuilder::BuildFragment(PhysicalOperator *phys_op) { - auto plan_fragment = MakeUnique(GetFragmentId()); - plan_fragment->SetSinkNode(query_context_ptr_, SinkType::kResult, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); - BuildFragments(phys_op, plan_fragment.get()); - if (plan_fragment->GetSourceNode() == nullptr) { - plan_fragment->SetSourceNode(query_context_ptr_, SourceType::kEmpty, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); +SharedPtr FragmentBuilder::BuildFragment(const Vector &phys_ops) { + SharedPtr result = nullptr; + for (auto *phys_op : phys_ops) { + auto plan_fragment = MakeUnique(GetFragmentId()); + plan_fragment->SetSinkNode(query_context_ptr_, SinkType::kResult, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); + BuildFragments(phys_op, plan_fragment.get()); + if (plan_fragment->GetSourceNode() == nullptr) { + plan_fragment->SetSourceNode(query_context_ptr_, SourceType::kEmpty, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); + } + if (result.get() == nullptr) { + result = std::move(plan_fragment); + } else { + PlanFragment::AddNext(result, plan_fragment.get()); + result = std::move(plan_fragment); + } } - return plan_fragment; + return result; } void FragmentBuilder::BuildExplain(PhysicalOperator *phys_op, PlanFragment *current_fragment_ptr) { @@ -64,7 +75,8 @@ void FragmentBuilder::BuildExplain(PhysicalOperator *phys_op, PlanFragment *curr case ExplainType::kPipeline: { // Build explain pipeline fragment SharedPtr>> texts_ptr = MakeShared>>(); - auto explain_child_fragment = this->BuildFragment(phys_op->left()); + Vector phys_ops{phys_op->left()}; + auto explain_child_fragment = this->BuildFragment(phys_ops); // Generate explain context of the child fragment ExplainFragment::Explain(explain_child_fragment.get(), texts_ptr); @@ -290,6 +302,83 @@ void FragmentBuilder::BuildFragments(PhysicalOperator *phys_op, PlanFragment *cu current_fragment_ptr->AddChild(std::move(next_plan_fragment)); return; } + case PhysicalOperatorType::kCompact: { + if (phys_op->left() != nullptr || phys_op->right() != nullptr) { + UnrecoverableError(fmt::format("Invalid input node of {}", phys_op->GetName())); + } + current_fragment_ptr->AddOperator(phys_op); + current_fragment_ptr->SetFragmentType(FragmentType::kParallelMaterialize); + current_fragment_ptr->SetSourceNode(query_context_ptr_, SourceType::kEmpty, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); + return; + } + case PhysicalOperatorType::kCompactIndexPrepare: { + if (phys_op->right() != nullptr) { + UnrecoverableError(fmt::format("Invalid input node of {}", phys_op->GetName())); + } + current_fragment_ptr->AddOperator(phys_op); + current_fragment_ptr->SetFragmentType(FragmentType::kSerialMaterialize); + current_fragment_ptr->SetSourceNode(query_context_ptr_, SourceType::kEmpty, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); + if (phys_op->left() != nullptr) { + auto next_plan_fragment = MakeUnique(GetFragmentId()); + next_plan_fragment->SetSinkNode(query_context_ptr_, + SinkType::kLocalQueue, + phys_op->left()->GetOutputNames(), + phys_op->left()->GetOutputTypes()); + BuildFragments(phys_op->left(), next_plan_fragment.get()); + current_fragment_ptr->AddChild(std::move(next_plan_fragment)); + } + if (phys_op->right() != nullptr) { + UnrecoverableError(fmt::format("Invalid input node of {}", phys_op->GetName())); + } + return; + } + case PhysicalOperatorType::kCompactIndexDo: { + if (phys_op->left() == nullptr || phys_op->right() != nullptr) { + UnrecoverableError(fmt::format("Invalid input node of {}", phys_op->GetName())); + } + current_fragment_ptr->AddOperator(phys_op); + current_fragment_ptr->SetFragmentType(FragmentType::kParallelMaterialize); + current_fragment_ptr->SetSourceNode(query_context_ptr_, SourceType::kLocalQueue, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); + + auto next_plan_fragment = MakeUnique(GetFragmentId()); + next_plan_fragment->SetSinkNode(query_context_ptr_, + SinkType::kLocalQueue, + phys_op->left()->GetOutputNames(), + phys_op->left()->GetOutputTypes()); + BuildFragments(phys_op->left(), next_plan_fragment.get()); + + current_fragment_ptr->AddChild(std::move(next_plan_fragment)); + return; + } + case PhysicalOperatorType::kCompactFinish: { + if (phys_op->left() == nullptr) { + UnrecoverableError(fmt::format("Invalid input node of {}", phys_op->GetName())); + } + current_fragment_ptr->AddOperator(phys_op); + current_fragment_ptr->SetFragmentType(FragmentType::kSerialMaterialize); + current_fragment_ptr->SetSourceNode(query_context_ptr_, SourceType::kLocalQueue, phys_op->GetOutputNames(), phys_op->GetOutputTypes()); + if (phys_op->left() != nullptr) { + auto next_plan_fragment1 = MakeUnique(GetFragmentId()); + next_plan_fragment1->SetSinkNode(query_context_ptr_, + SinkType::kLocalQueue, + phys_op->left()->GetOutputNames(), + phys_op->left()->GetOutputTypes()); + BuildFragments(phys_op->left(), next_plan_fragment1.get()); + current_fragment_ptr->AddChild(std::move(next_plan_fragment1)); + if (phys_op->right() != nullptr) { + auto next_plan_fragment2 = MakeUnique(GetFragmentId()); + next_plan_fragment2->SetSinkNode(query_context_ptr_, + SinkType::kLocalQueue, + phys_op->right()->GetOutputNames(), + phys_op->right()->GetOutputTypes()); + BuildFragments(phys_op->right(), next_plan_fragment2.get()); + current_fragment_ptr->AddChild(std::move(next_plan_fragment2)); + } + } else if (phys_op->right() != nullptr) { + UnrecoverableError(fmt::format("Invalid input node of {}", phys_op->GetName())); + } + return; + } default: { UnrecoverableError(fmt::format("Invalid operator type: {} in Fragment Builder", phys_op->GetName())); } diff --git a/src/executor/fragment_builder.cppm b/src/executor/fragment_builder.cppm index 72bf804b76..80ae55cfab 100644 --- a/src/executor/fragment_builder.cppm +++ b/src/executor/fragment_builder.cppm @@ -27,7 +27,7 @@ export class FragmentBuilder { public: explicit FragmentBuilder(QueryContext *query_context_ptr) : query_context_ptr_(query_context_ptr) {} - UniquePtr BuildFragment(PhysicalOperator *phys_op); + SharedPtr BuildFragment(const Vector &physical_plans); private: void BuildFragments(PhysicalOperator *phys_op, PlanFragment *current_fragment_ptr); diff --git a/src/executor/operator/physical_command.cpp b/src/executor/operator/physical_command.cpp index 6a6a027034..bf65bc02ee 100644 --- a/src/executor/operator/physical_command.cpp +++ b/src/executor/operator/physical_command.cpp @@ -34,7 +34,6 @@ import defer_op; import config; import status; import infinity_exception; -import compact_segments_task; import variables; namespace infinity { @@ -170,12 +169,6 @@ bool PhysicalCommand::Execute(QueryContext *query_context, OperatorState *operat case CommandType::kCheckTable: { break; } - case CommandType::kCompactTable: { - auto *txn = query_context->GetTxn(); - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry_, txn); // copy the table ref shared_ptr here - compact_task->Execute(); - break; - } default: { UnrecoverableError("Invalid command type."); } diff --git a/src/executor/operator/physical_command.cppm b/src/executor/operator/physical_command.cppm index f9f250a20a..c1014794b0 100644 --- a/src/executor/operator/physical_command.cppm +++ b/src/executor/operator/physical_command.cppm @@ -57,8 +57,6 @@ public: inline SharedPtr>> GetOutputTypes() const override { return output_types_; } public: - TableEntry *table_entry_ = nullptr; // only used for compact command - const SharedPtr command_info_{}; const SharedPtr> output_names_{}; diff --git a/src/executor/operator/physical_compact.cpp b/src/executor/operator/physical_compact.cpp new file mode 100644 index 0000000000..fdb2eec7e4 --- /dev/null +++ b/src/executor/operator/physical_compact.cpp @@ -0,0 +1,213 @@ +// 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 + +module physical_compact; + +import stl; +import internal_types; +import operator_state; +import base_table_ref; +import block_index; +import column_vector; +import catalog; +import table_entry; +import segment_entry; +import block_entry; +import compact_state_data; +import default_values; +import logger; +import infinity_exception; +import third_party; + +namespace infinity { + +class GreedyCompactableSegmentsGenerator { +public: + GreedyCompactableSegmentsGenerator(const BaseTableRef *base_table_ref, SizeT max_segment_size) : max_segment_size_(max_segment_size) { + const auto &block_index = *base_table_ref->block_index_; + for (const auto &[segment_id, segment_snapshot] : block_index.segment_block_index_) { + SegmentEntry *segment_entry = segment_snapshot.segment_entry_; + if (segment_entry->status() == SegmentStatus::kSealed) { + SizeT row_count = segment_entry->actual_row_count(); + segments_.emplace(row_count, segment_entry); + } + } + } + + // find the largest segment to fill the free space + Vector generate() { + Vector result; + do { + result.clear(); + SizeT segment_size = max_segment_size_; + + while (true) { + auto iter = segments_.upper_bound(segment_size); + if (iter == segments_.begin()) { + break; + } + --iter; + auto [row_count, segment_entry] = *iter; + segments_.erase(iter); + result.push_back(segment_entry); + segment_size -= row_count; + } + } while (result.size() == 1 && (result[0]->actual_row_count() == result[0]->row_count())); + // FIXME: compact single segment with too much delete row + return result; + } + +private: + MultiMap segments_; // TODO(opt): use Map replace MultiMap + + const SizeT max_segment_size_; +}; + +void PhysicalCompact::Init() { + if (compact_type_ == CompactStatementType::kManual) { + TableEntry *table_entry = base_table_ref_->table_entry_ptr_; + LOG_INFO(fmt::format("Manual compact {} start", *table_entry->GetTableName())); + if (!table_entry->CompactPrepare()) { + LOG_WARN(fmt::format("Table {} is not compactable.", *table_entry->GetTableName())); + return; + } + GreedyCompactableSegmentsGenerator generator(base_table_ref_.get(), DEFAULT_SEGMENT_CAPACITY); + while (true) { + Vector compactible_segments = generator.generate(); + if (compactible_segments.empty()) { + break; + } + compactible_segments_group_.push_back(compactible_segments); + } + } else { + TableEntry *table_entry = base_table_ref_->table_entry_ptr_; + LOG_INFO(fmt::format("Auto compact {} start", *table_entry->GetTableName())); + Vector compactible_segments; + const auto &block_index = *base_table_ref_->block_index_; + for (const auto &[segment_id, segment_snapshot] : block_index.segment_block_index_) { + SegmentEntry *segment_entry = segment_snapshot.segment_entry_; + if (segment_entry->status() == SegmentStatus::kSealed) { + compactible_segments.push_back(segment_entry); + } + } + compactible_segments_group_.push_back(compactible_segments); + } +} + +bool PhysicalCompact::Execute(QueryContext *query_context, OperatorState *operator_state) { + auto *compact_operator_state = static_cast(operator_state); + SizeT group_idx = compact_operator_state->compact_idx_; + CompactStateData *compact_state_data = compact_operator_state->compact_state_data_.get(); + RowIDRemap &remapper = compact_state_data->remapper_; + if (group_idx == compact_state_data->segment_data_list_.size()) { + compact_operator_state->SetComplete(); + return true; + } + CompactSegmentData &compact_segment_data = compact_state_data->segment_data_list_[group_idx]; + compact_segment_data.old_segments_ = compactible_segments_group_[group_idx]; + const auto &compactible_segments = compact_segment_data.old_segments_; + + for (auto *compactible_segment : compactible_segments) { + if (!compactible_segment->TrySetCompacting(compact_state_data)) { + UnrecoverableError("Segment should be compactible."); + } + } + + auto *txn = query_context->GetTxn(); + auto *buffer_mgr = query_context->storage()->buffer_manager(); + TxnTimeStamp begin_ts = txn->BeginTS(); + + TableEntry *table_entry = base_table_ref_->table_entry_ptr_; + BlockIndex *block_index = base_table_ref_->block_index_.get(); + + SizeT column_count = table_entry->ColumnCount(); + + compact_segment_data.new_segment_ = SegmentEntry::NewSegmentEntry(table_entry, Catalog::GetNextSegmentID(table_entry), txn); + auto *new_segment = compact_segment_data.new_segment_.get(); + compact_state_data->AddNewSegment(new_segment, txn); + SegmentID new_segment_id = new_segment->segment_id(); + + UniquePtr new_block = BlockEntry::NewBlockEntry(new_segment, new_segment->GetNextBlockID(), 0 /*checkpoint_ts*/, column_count, txn); + const SizeT block_capacity = new_block->row_capacity(); + + for (SegmentEntry *segment : compactible_segments) { + SegmentID segment_id = segment->segment_id(); + const auto &segment_info = block_index->segment_block_index_.at(segment_id); + for (const auto *block_entry : segment_info.block_map_) { + BlockID block_id = block_entry->block_id(); + Vector input_column_vectors; + for (ColumnID column_id = 0; column_id < column_count; ++column_id) { + auto *column_block_entry = block_entry->GetColumnBlockEntry(column_id); + input_column_vectors.emplace_back(column_block_entry->GetColumnVector(buffer_mgr)); + } + SizeT read_offset = 0; + while (true) { + auto [row_begin, row_end] = block_entry->GetVisibleRange(begin_ts, read_offset); + SizeT read_size = row_end - row_begin; + if (read_size == 0) { + break; + } + + auto block_entry_append = [&](SizeT row_begin, SizeT read_size1) { + if (read_size1 == 0) { + return; + } + new_block->AppendBlock(input_column_vectors, row_begin, read_size1, buffer_mgr); + RowID new_row_id(new_segment_id, new_block->block_id() * block_capacity + new_block->row_count()); + remapper.AddMap(segment_id, block_id, row_begin, new_row_id); + read_offset = row_begin + read_size1; + }; + + if (read_size + new_block->row_count() > block_capacity) { + SizeT read_size1 = block_capacity - new_block->row_count(); + block_entry_append(row_begin, read_size1); + row_begin += read_size1; + read_size -= read_size1; + new_segment->AppendBlockEntry(std::move(new_block)); + + new_block = BlockEntry::NewBlockEntry(new_segment, new_segment->GetNextBlockID(), 0, column_count, txn); + } + block_entry_append(row_begin, read_size); + } + } + } + if (new_block->row_count() > 0) { + new_segment->AppendBlockEntry(std::move(new_block)); + } + compact_operator_state->compact_idx_ = ++group_idx; + if (group_idx == compact_state_data->segment_data_list_.size()) { + compact_operator_state->SetComplete(); + } + compact_operator_state->SetComplete(); + + return true; +} + +Vector>> PhysicalCompact::PlanCompact(SizeT parallel_count) { + if (parallel_count > compactible_segments_group_.size()) { + UnrecoverableError( + fmt::format("parallel_count {} is larger than compactible_segments_group_ size {}", parallel_count, compactible_segments_group_.size())); + } + Vector>> result(parallel_count); + for (SizeT i = 0; i < compactible_segments_group_.size(); ++i) { + result[i % parallel_count].push_back(compactible_segments_group_[i]); + } + return result; +} + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact.cppm b/src/executor/operator/physical_compact.cppm new file mode 100644 index 0000000000..b644b30cf5 --- /dev/null +++ b/src/executor/operator/physical_compact.cppm @@ -0,0 +1,66 @@ +// 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; + +export module physical_compact; + +import stl; +import physical_operator; +import physical_operator_type; +import base_table_ref; +import query_context; +import operator_state; +import load_meta; +import data_type; +import segment_entry; +import compact_statement; + +namespace infinity { + +export class PhysicalCompact : public PhysicalOperator { +public: + PhysicalCompact(u64 id, + SharedPtr base_table_ref, + CompactStatementType compact_type, + SharedPtr> output_names, + SharedPtr>> output_types, + SharedPtr> load_metas) + : PhysicalOperator(PhysicalOperatorType::kCompact, nullptr, nullptr, id, load_metas), base_table_ref_(base_table_ref), + compact_type_(compact_type), output_names_(output_names), output_types_(output_types) {} + + ~PhysicalCompact() override = default; + + void Init() override; + + bool Execute(QueryContext *query_context, OperatorState *operator_state) final; + + SizeT TaskletCount() override { return compactible_segments_group_.size(); } + + Vector>> PlanCompact(SizeT parallel_count); + + SharedPtr> GetOutputNames() const override { return output_names_; } + + SharedPtr>> GetOutputTypes() const override { return output_types_; } + +private: + SharedPtr base_table_ref_; + CompactStatementType compact_type_; + Vector> compactible_segments_group_; + + SharedPtr> output_names_{}; + SharedPtr>> output_types_{}; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact_finish.cpp b/src/executor/operator/physical_compact_finish.cpp new file mode 100644 index 0000000000..f3ccf56454 --- /dev/null +++ b/src/executor/operator/physical_compact_finish.cpp @@ -0,0 +1,95 @@ +// 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 + +module physical_compact_finish; + +import stl; +import operator_state; +import table_entry; +import segment_entry; +import base_table_ref; +import compact_state_data; +import wal_manager; +import wal_entry; +import logger; +import txn; +import internal_types; + +namespace infinity { + +bool PhysicalCompactFinish::Execute(QueryContext *query_context, OperatorState *operator_state) { + auto *compact_finish_operator_state = static_cast(operator_state); + const CompactStateData *compact_state_data = compact_finish_operator_state->compact_state_data_.get(); + if (!ApplyDeletes(query_context, compact_state_data)) { + LOG_WARN("Failed to apply deletes in compact finish"); + return true; + } + SaveSegmentData(query_context, compact_state_data); + compact_finish_operator_state->SetComplete(); + return true; +} + +void PhysicalCompactFinish::SaveSegmentData(QueryContext *query_context, const CompactStateData *compact_state_data) { + auto *txn = query_context->GetTxn(); + auto *table_entry = base_table_ref_->table_entry_ptr_; + + Vector segment_infos; + Vector old_segment_ids; + Vector, Vector>> segment_data; + for (const auto &compact_segment_data : compact_state_data->segment_data_list_) { + auto *new_segment = compact_segment_data.new_segment_.get(); + new_segment->FlushNewData(); + segment_infos.emplace_back(new_segment); + + for (const auto *old_segment : compact_segment_data.old_segments_) { + old_segment_ids.push_back(old_segment->segment_id()); + } + segment_data.emplace_back(compact_segment_data.new_segment_, compact_segment_data.old_segments_); + } + + txn->Compact(table_entry, std::move(segment_data), compact_type_); + String db_name = *table_entry->GetDBName(); + String table_name = *table_entry->GetTableName(); + txn->AddWalCmd(MakeShared(std::move(db_name), std::move(table_name), std::move(segment_infos), std::move(old_segment_ids))); +} + +bool PhysicalCompactFinish::ApplyDeletes(QueryContext *query_context, const CompactStateData *compact_state_data) { + auto *txn = query_context->GetTxn(); + auto *table_entry = base_table_ref_->table_entry_ptr_; + + for (const auto &compact_segment_data : compact_state_data->segment_data_list_) { + for (auto *old_segment : compact_segment_data.old_segments_) { + if (!old_segment->SetNoDelete()) { + return false; + } + } + } + const HashMap> &to_delete = compact_state_data->GetToDelete(); + Vector row_ids; + for (const auto &[segment_id, delete_offsets] : to_delete) { + for (SegmentOffset offset : delete_offsets) { + RowID old_row_id(segment_id, offset); + RowID new_row_id = compact_state_data->remapper_.GetNewRowID(old_row_id); + row_ids.push_back(new_row_id); + } + } + txn->Delete(table_entry, row_ids, false); + return true; +} + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact_finish.cppm b/src/executor/operator/physical_compact_finish.cppm new file mode 100644 index 0000000000..155ae6a0ac --- /dev/null +++ b/src/executor/operator/physical_compact_finish.cppm @@ -0,0 +1,75 @@ +// 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; + +export module physical_compact_finish; + +import stl; +import physical_operator; +import physical_operator_type; +import operator_state; +import query_context; +import load_meta; +import data_type; +import infinity_exception; +import base_table_ref; +import compact_state_data; +import compact_statement; + +namespace infinity { + +export class PhysicalCompactFinish : public PhysicalOperator { +public: + explicit PhysicalCompactFinish(u64 id, + UniquePtr left, + UniquePtr right, + SharedPtr base_table_ref, + CompactStatementType compact_type, + SharedPtr> output_names, + SharedPtr>> output_types, + SharedPtr> load_metas) + : PhysicalOperator(PhysicalOperatorType::kCompactFinish, std::move(left), std::move(right), id, load_metas), base_table_ref_(base_table_ref), + compact_type_(compact_type), output_names_(output_names), output_types_(output_types) {} + + ~PhysicalCompactFinish() override = default; + + void Init() override {} + + bool Execute(QueryContext *query_context, OperatorState *operator_state) override; + + inline SharedPtr> GetOutputNames() const override { return output_names_; } + + inline SharedPtr>> GetOutputTypes() const override { return output_types_; } + + SizeT TaskletCount() override { + UnrecoverableError("Not implement: TaskletCount not Implement"); + return 0; + } + +private: + void SaveSegmentData(QueryContext *query_context, const CompactStateData *compact_state_data); + + bool ApplyDeletes(QueryContext *query_context, const CompactStateData *compact_state_data); + +public: + SharedPtr base_table_ref_; + CompactStatementType compact_type_; + +private: + SharedPtr> output_names_{}; + SharedPtr>> output_types_{}; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact_index_do.cpp b/src/executor/operator/physical_compact_index_do.cpp new file mode 100644 index 0000000000..c7a2f66ec2 --- /dev/null +++ b/src/executor/operator/physical_compact_index_do.cpp @@ -0,0 +1,56 @@ +// 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; + +module physical_compact_index_do; + +import stl; +import operator_state; +import txn; +import query_context; +import table_index_entry; +import block_index; +import status; +import base_table_ref; + +namespace infinity { + +bool PhysicalCompactIndexDo::Execute(QueryContext *query_context, OperatorState *operator_state) { + auto *compact_index_do_operator_state = static_cast(operator_state); + auto *index_index = base_table_ref_->index_index_.get(); + SizeT create_index_idx = compact_index_do_operator_state->create_index_idx_; + if (create_index_idx == index_index->index_snapshots_vec_.size()) { + operator_state->SetComplete(); + return true; + } + auto *table_index_entry = index_index->index_snapshots_vec_[create_index_idx]->table_index_entry_; + const String &index_name = *table_index_entry->GetIndexName(); + + auto *compact_state_data = compact_index_do_operator_state->compact_state_data_.get(); + BaseTableRef *new_table_ref = compact_state_data->GetNewTableRef(); + + Txn *txn = query_context->GetTxn(); + auto &create_index_idxes = (*compact_index_do_operator_state->create_index_shared_data_)[create_index_idx]->create_index_idxes_; + auto status = txn->CreateIndexDo(new_table_ref, index_name, create_index_idxes); + + ++create_index_idx; + compact_index_do_operator_state->create_index_idx_ = create_index_idx; + if (create_index_idx == index_index->index_snapshots_vec_.size()) { + compact_index_do_operator_state->SetComplete(); + } + return true; +} + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact_index_do.cppm b/src/executor/operator/physical_compact_index_do.cppm new file mode 100644 index 0000000000..64a249638c --- /dev/null +++ b/src/executor/operator/physical_compact_index_do.cppm @@ -0,0 +1,61 @@ +// 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; + +export module physical_compact_index_do; + +import stl; +import physical_operator; +import physical_operator_type; +import operator_state; +import query_context; +import load_meta; +import data_type; +import base_table_ref; + +namespace infinity { + +export class PhysicalCompactIndexDo : public PhysicalOperator { +public: + explicit PhysicalCompactIndexDo(u64 id, + UniquePtr left, + SharedPtr base_table_ref, + SharedPtr> output_names, + SharedPtr>> output_types, + SharedPtr> load_metas) + : PhysicalOperator(PhysicalOperatorType::kCompactIndexDo, std::move(left), nullptr, id, load_metas), base_table_ref_(base_table_ref), + output_names_(output_names), output_types_(output_types) {} + + ~PhysicalCompactIndexDo() override = default; + + void Init() override {} + + bool Execute(QueryContext *query_context, OperatorState *operator_state) override; + + SizeT TaskletCount() override { return 0; } + + inline SharedPtr> GetOutputNames() const override { return output_names_; } + + inline SharedPtr>> GetOutputTypes() const override { return output_types_; } + +public: + SharedPtr base_table_ref_; + +private: + SharedPtr> output_names_{}; + SharedPtr>> output_types_{}; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact_index_prepare.cpp b/src/executor/operator/physical_compact_index_prepare.cpp new file mode 100644 index 0000000000..1f7fcbd8ad --- /dev/null +++ b/src/executor/operator/physical_compact_index_prepare.cpp @@ -0,0 +1,68 @@ +// 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; + +module physical_compact_index_prepare; + +import stl; +import operator_state; +import query_context; +import table_index_entry; +import block_index; +import txn; +import status; +import base_table_ref; + +namespace infinity { + +bool PhysicalCompactIndexPrepare::Execute(QueryContext *query_context, OperatorState *operator_state) { + auto *compact_index_prepare_operator_state = static_cast(operator_state); + auto *index_index = base_table_ref_->index_index_.get(); + + SizeT create_index_idx = compact_index_prepare_operator_state->create_index_idx_; + if (create_index_idx == index_index->index_snapshots_vec_.size()) { + operator_state->SetComplete(); + return true; + } + auto *table_index_entry = index_index->index_snapshots_vec_[create_index_idx]->table_index_entry_; + + auto *compact_state_data = compact_index_prepare_operator_state->compact_state_data_.get(); + BaseTableRef *new_table_ref = compact_state_data->GetNewTableRef(); + + if (compact_index_prepare_operator_state->create_index_shared_data_.get() != nullptr) { + (*compact_index_prepare_operator_state->create_index_shared_data_)[create_index_idx]->Init(new_table_ref->block_index_.get()); + } + + Txn *txn = query_context->GetTxn(); + auto status = txn->CreateIndexPrepare(table_index_entry, new_table_ref, prepare_, false); + compact_state_data->AddNewIndex(table_index_entry, txn); + if (!status.ok()) { + operator_state->status_ = status; + return true; + } + + compact_index_prepare_operator_state->create_index_idx_ = ++create_index_idx; + if (create_index_idx == index_index->index_snapshots_vec_.size()) { + compact_index_prepare_operator_state->SetComplete(); + } + return true; +} + +SizeT PhysicalCompactIndexPrepare::TaskletCount() { + auto *index_index = base_table_ref_->index_index_.get(); + return index_index->index_snapshots_vec_.size(); +} + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_compact_index_prepare.cppm b/src/executor/operator/physical_compact_index_prepare.cppm new file mode 100644 index 0000000000..c21471dc29 --- /dev/null +++ b/src/executor/operator/physical_compact_index_prepare.cppm @@ -0,0 +1,63 @@ +// 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; + +export module physical_compact_index_prepare; + +import stl; +import physical_operator; +import physical_operator_type; +import operator_state; +import query_context; +import load_meta; +import data_type; +import base_table_ref; + +namespace infinity { + +export class PhysicalCompactIndexPrepare : public PhysicalOperator { +public: + explicit PhysicalCompactIndexPrepare(u64 id, + UniquePtr left, + SharedPtr base_table_ref, + bool prepare, + SharedPtr> output_names, + SharedPtr>> output_types, + SharedPtr> load_metas) + : PhysicalOperator(PhysicalOperatorType::kCompactIndexPrepare, std::move(left), nullptr, id, load_metas), base_table_ref_(base_table_ref), + prepare_(prepare), output_names_(output_names), output_types_(output_types) {} + + ~PhysicalCompactIndexPrepare() override = default; + + void Init() override {} + + bool Execute(QueryContext *query_context, OperatorState *operator_state) override; + + SizeT TaskletCount() override; + + inline SharedPtr> GetOutputNames() const override { return output_names_; } + + inline SharedPtr>> GetOutputTypes() const override { return output_types_; } + +public: + SharedPtr base_table_ref_; + bool prepare_{false}; + +private: + SharedPtr> output_names_{}; + SharedPtr>> output_types_{}; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/executor/operator/physical_import.cpp b/src/executor/operator/physical_import.cpp index cb8c996aba..0211690816 100644 --- a/src/executor/operator/physical_import.cpp +++ b/src/executor/operator/physical_import.cpp @@ -460,6 +460,10 @@ void PhysicalImport::CSVRowHandler(void *context) { column_count, table_entry->ColumnCount())); LOG_ERROR(*err_msg); + for (SizeT i = 0; i < column_count; ++i) { + ZsvCell cell = parser_context->parser_.GetCell(i); + LOG_ERROR(fmt::format("Column {}: {}", i, std::string_view((char *)cell.str, cell.len))); + } RecoverableError(Status::ColumnCountMismatch(*err_msg)); } diff --git a/src/executor/operator/physical_index_scan.cpp b/src/executor/operator/physical_index_scan.cpp index 0e47700806..726335192c 100644 --- a/src/executor/operator/physical_index_scan.cpp +++ b/src/executor/operator/physical_index_scan.cpp @@ -673,11 +673,11 @@ void PhysicalIndexScan::ExecuteInternal(QueryContext *query_context, IndexScanOp auto segment_id = segment_ids[next_idx]; SegmentEntry *segment_entry = nullptr; - auto &segment_index_hashmap = base_table_ref_->block_index_->segment_index_; - if (auto iter = segment_index_hashmap.find(segment_id); iter == segment_index_hashmap.end()) { + const auto &segment_block_index_ = base_table_ref_->block_index_->segment_block_index_; + if (auto iter = segment_block_index_.find(segment_id); iter == segment_block_index_.end()) { UnrecoverableError(fmt::format("Cannot find SegmentEntry for segment id: {}", segment_id)); } else { - segment_entry = iter->second; + segment_entry = iter->second.segment_entry_; } // check FastRoughFilter const auto &fast_rough_filter = *segment_entry->GetFastRoughFilter(); diff --git a/src/executor/operator/physical_knn_scan.cpp b/src/executor/operator/physical_knn_scan.cpp index 39b52f1144..4ebe395241 100644 --- a/src/executor/operator/physical_knn_scan.cpp +++ b/src/executor/operator/physical_knn_scan.cpp @@ -223,12 +223,12 @@ void PhysicalKnnScan::PlanWithIndex(QueryContext *query_context) { // TODO: retu // Generate task set: index segment and no index block BlockIndex *block_index = base_table_ref_->block_index_.get(); - for (SegmentEntry *segment_entry : block_index->segments_) { - if (auto iter = index_entry_map.find(segment_entry->segment_id()); iter != index_entry_map.end()) { + for (const auto &[segment_id, segment_info] : block_index->segment_block_index_) { + if (auto iter = index_entry_map.find(segment_id); iter != index_entry_map.end()) { index_entries_->emplace_back(iter->second.get()); } else { - BlockEntryIter block_entry_iter(segment_entry); - for (auto *block_entry = block_entry_iter.Next(); block_entry != nullptr; block_entry = block_entry_iter.Next()) { + const auto &block_map = segment_info.block_map_; + for (const auto *block_entry : block_map) { BlockColumnEntry *block_column_entry = block_entry->GetColumnBlockEntry(knn_column_id); block_column_entries_->emplace_back(block_column_entry); } @@ -321,11 +321,11 @@ void PhysicalKnnScan::ExecuteInternal(QueryContext *query_context, KnnScanOperat auto segment_id = segment_index_entry->segment_id(); SegmentEntry *segment_entry = nullptr; - auto &segment_index_hashmap = base_table_ref_->block_index_->segment_index_; + const auto &segment_index_hashmap = base_table_ref_->block_index_->segment_block_index_; if (auto iter = segment_index_hashmap.find(segment_id); iter == segment_index_hashmap.end()) { UnrecoverableError(fmt::format("Cannot find SegmentEntry for segment id: {}", segment_id)); } else { - segment_entry = iter->second; + segment_entry = iter->second.segment_entry_; } if (auto it = common_query_filter_->filter_result_.find(segment_id); it != common_query_filter_->filter_result_.end()) { LOG_TRACE(fmt::format("KnnScan: {} index {}/{} not skipped after common_query_filter", diff --git a/src/executor/operator/physical_match.cpp b/src/executor/operator/physical_match.cpp index e891589d9d..3e3c7cfabe 100644 --- a/src/executor/operator/physical_match.cpp +++ b/src/executor/operator/physical_match.cpp @@ -41,6 +41,7 @@ import infinity_exception; import value; import third_party; import base_table_ref; +import block_index; import load_meta; import block_entry; import block_column_entry; @@ -81,7 +82,7 @@ class FilterIteratorBase : public QueryIteratorT { const SizeT filter_result_count_ = common_query_filter_->filter_result_count_; const Map, Bitmask>> *filter_result_ptr_ = &common_query_filter_->filter_result_; const BaseExpression *secondary_index_filter_ = common_query_filter_->secondary_index_filter_qualified_.get(); - const HashMap *segment_index_ = &common_query_filter_->base_table_ref_->block_index_->segment_index_; + const Map &segment_index = common_query_filter_->base_table_ref_->block_index_->segment_block_index_; const TxnTimeStamp begin_ts_ = common_query_filter_->begin_ts_; SegmentID current_segment_id_ = filter_result_ptr_->size() ? filter_result_ptr_->begin()->first : INVALID_SEGMENT_ID; @@ -104,8 +105,9 @@ class FilterIteratorBase : public QueryIteratorT { RowID SelfBlockLastDocID() const { if (current_segment_id_ != cache_segment_id_) { cache_segment_id_ = current_segment_id_; - cache_segment_entry_ = segment_index_->at(cache_segment_id_); - cache_segment_offset_ = cache_segment_entry_->row_count(); + const auto &segment_info = segment_index.at(cache_segment_id_); + cache_segment_entry_ = segment_info.segment_entry_; + cache_segment_offset_ = segment_info.segment_offset_; cache_need_check_delete_ = cache_segment_entry_->CheckAnyDelete(begin_ts_); } return RowID(current_segment_id_, cache_segment_offset_); @@ -138,7 +140,7 @@ class FilterIteratorBase : public QueryIteratorT { assert(doc_id_no_beyond.segment_id_ == current_segment_id_); if (current_segment_id_ != cache_segment_id_) { cache_segment_id_ = current_segment_id_; - cache_segment_entry_ = segment_index_->at(cache_segment_id_); + cache_segment_entry_ = segment_index.at(cache_segment_id_).segment_entry_; cache_segment_offset_ = cache_segment_entry_->row_count(); cache_need_check_delete_ = cache_segment_entry_->CheckAnyDelete(begin_ts_); } diff --git a/src/executor/operator/physical_sink.cpp b/src/executor/operator/physical_sink.cpp index d2960de9d6..ca10e4a4b4 100644 --- a/src/executor/operator/physical_sink.cpp +++ b/src/executor/operator/physical_sink.cpp @@ -342,8 +342,20 @@ void PhysicalSink::FillSinkStateFromLastOperatorState(ResultSinkState *result_si }; break; } + case PhysicalOperatorType::kCompactFinish: { + auto *output_state = static_cast(task_operator_state); + if (!output_state->Ok()) { + result_sink_state->status_ = std::move(output_state->status_); + break; + } + result_sink_state->result_def_ = { + MakeShared(0, MakeShared(LogicalType::kInteger), "OK", HashSet()), + }; + break; + } default: { - RecoverableError(Status::NotSupport(fmt::format("{} isn't supported here.", PhysicalOperatorToString(task_operator_state->operator_type_)))); + RecoverableError( + Status::NotSupport(fmt::format("{} isn't supported here.", PhysicalOperatorToString(task_operator_state->operator_type_)))); } } } @@ -370,8 +382,24 @@ void PhysicalSink::FillSinkStateFromLastOperatorState(MessageSinkState *message_ message_sink_state->message_ = std::move(create_index_do_output_state->result_msg_); break; } + case PhysicalOperatorType::kCompact: { + // auto *compact_output_state = static_cast(task_operator_state); + message_sink_state->message_ = MakeUnique("Tmp for test"); + break; + } + case PhysicalOperatorType::kCompactIndexPrepare: { + // auto *compact_index_output_state = static_cast(task_operator_state); + message_sink_state->message_ = MakeUnique("Tmp for test"); + break; + } + case PhysicalOperatorType::kCompactIndexDo: { + // auto *compact_index_output_state = static_cast(task_operator_state); + message_sink_state->message_ = MakeUnique("Tmp for test"); + break; + } default: { - RecoverableError(Status::NotSupport(fmt::format("{} isn't supported here.", PhysicalOperatorToString(task_operator_state->operator_type_)))); + RecoverableError( + Status::NotSupport(fmt::format("{} isn't supported here.", PhysicalOperatorToString(task_operator_state->operator_type_)))); break; } } diff --git a/src/executor/operator/physical_source.cpp b/src/executor/operator/physical_source.cpp index b3c7dff02c..c17dbbc1cd 100644 --- a/src/executor/operator/physical_source.cpp +++ b/src/executor/operator/physical_source.cpp @@ -43,7 +43,8 @@ bool PhysicalSource::Execute(QueryContext *, SourceState *source_state) { } case SourceStateType::kKnnScan: case SourceStateType::kTableScan: - case SourceStateType::kIndexScan: { + case SourceStateType::kIndexScan: + case SourceStateType::kCompact: { return true; } case SourceStateType::kEmpty: { diff --git a/src/executor/operator/physical_table_scan.cpp b/src/executor/operator/physical_table_scan.cpp index 524c6f0b3f..96870e320a 100644 --- a/src/executor/operator/physical_table_scan.cpp +++ b/src/executor/operator/physical_table_scan.cpp @@ -104,14 +104,20 @@ Vector>> PhysicalTableScan::PlanBlockEntries(i64 u64 block_per_task = all_block_count / parallel_count; u64 residual = all_block_count % parallel_count; + Vector global_blocks; + for (const auto &[segment_id, segment_info] : block_index->segment_block_index_) { + for (const auto *block_entry : segment_info.block_map_) { + global_blocks.emplace_back(GlobalBlockID{segment_id, block_entry->block_id()}); + } + } Vector>> result(parallel_count, nullptr); for (SizeT task_id = 0, global_block_id = 0, residual_idx = 0; (i64)task_id < parallel_count; ++task_id) { result[task_id] = MakeShared>(); for (u64 block_id_in_task = 0; block_id_in_task < block_per_task; ++block_id_in_task) { - result[task_id]->emplace_back(block_index->global_blocks_[global_block_id++]); + result[task_id]->emplace_back(global_blocks[global_block_id++]); } if (residual_idx < residual) { - result[task_id]->emplace_back(block_index->global_blocks_[global_block_id++]); + result[task_id]->emplace_back(global_blocks[global_block_id++]); ++residual_idx; } } diff --git a/src/executor/operator_state.cppm b/src/executor/operator_state.cppm index c2115679f5..9380e2f14c 100644 --- a/src/executor/operator_state.cppm +++ b/src/executor/operator_state.cppm @@ -23,6 +23,7 @@ import fragment_data; import data_block; import table_scan_function_data; import knn_scan_data; +import compact_state_data; import table_def; import merge_knn_data; @@ -33,6 +34,7 @@ import status; import internal_types; import column_def; import data_type; +import segment_entry; namespace infinity { @@ -360,8 +362,56 @@ export struct FusionOperatorState : public OperatorState { Map>> input_data_blocks_{}; }; +// Compact +export struct CompactOperatorState : public OperatorState { + inline explicit CompactOperatorState(SizeT compact_idx, SharedPtr compact_state_data) + : OperatorState(PhysicalOperatorType::kCompact), compact_idx_(compact_idx), compact_state_data_(compact_state_data) {} + + SizeT compact_idx_{}; + + SharedPtr compact_state_data_{}; +}; + +export struct CompactIndexPrepareOperatorState : public OperatorState { + inline explicit CompactIndexPrepareOperatorState(SharedPtr compact_state_data, + SharedPtr>> create_index_shared_data) + : OperatorState(PhysicalOperatorType::kCompactIndexPrepare), compact_state_data_(compact_state_data), + create_index_shared_data_(create_index_shared_data) {} + + SharedPtr compact_state_data_{}; + SharedPtr>> create_index_shared_data_{}; + SizeT create_index_idx_{}; +}; + +export struct CompactIndexDoOperatorState : public OperatorState { + inline explicit CompactIndexDoOperatorState(SharedPtr compact_state_data, + SharedPtr>> create_index_shared_data) + : OperatorState(PhysicalOperatorType::kCompactIndexDo), compact_state_data_(compact_state_data), + create_index_shared_data_(create_index_shared_data) {} + + SharedPtr compact_state_data_{}; + SharedPtr>> create_index_shared_data_{}; + SizeT create_index_idx_{}; +}; + +export struct CompactFinishOperatorState : public OperatorState { + explicit CompactFinishOperatorState(SharedPtr compact_state_data) + : OperatorState(PhysicalOperatorType::kCompactFinish), compact_state_data_(compact_state_data) {} + + SharedPtr compact_state_data_{}; +}; + // Source -export enum class SourceStateType { kInvalid, kQueue, kAggregate, kTableScan, kIndexScan, kKnnScan, kEmpty }; +export enum class SourceStateType { + kInvalid, + kQueue, + kAggregate, + kTableScan, + kIndexScan, + kKnnScan, + kCompact, + kEmpty, +}; export struct SourceState { inline explicit SourceState(SourceStateType state_type) : state_type_(state_type) {} diff --git a/src/executor/physical_operator_type.cpp b/src/executor/physical_operator_type.cpp index 2e7b725936..476c8d0066 100644 --- a/src/executor/physical_operator_type.cpp +++ b/src/executor/physical_operator_type.cpp @@ -124,6 +124,14 @@ String PhysicalOperatorToString(PhysicalOperatorType type) { return "MergeKnn"; case PhysicalOperatorType::kCommand: return "Command"; + case PhysicalOperatorType::kCompact: + return "Compact"; + case PhysicalOperatorType::kCompactIndexPrepare: + return "CompactIndexPrepare"; + case PhysicalOperatorType::kCompactIndexDo: + return "CompactIndexDo"; + case PhysicalOperatorType::kCompactFinish: + return "CompactFinish"; case PhysicalOperatorType::kMatch: return "Match"; case PhysicalOperatorType::kFusion: diff --git a/src/executor/physical_operator_type.cppm b/src/executor/physical_operator_type.cppm index bcd8849aa6..4752d31e90 100644 --- a/src/executor/physical_operator_type.cppm +++ b/src/executor/physical_operator_type.cppm @@ -95,6 +95,12 @@ export enum class PhysicalOperatorType : i8 { kOptimize, kCommand, + // Compact + kCompact, + kCompactIndexPrepare, + kCompactIndexDo, + kCompactFinish, + kSink, kSource, }; diff --git a/src/executor/physical_planner.cpp b/src/executor/physical_planner.cpp index 6117dffae0..dc4614b314 100644 --- a/src/executor/physical_planner.cpp +++ b/src/executor/physical_planner.cpp @@ -71,6 +71,10 @@ import physical_union_all; import physical_update; import physical_drop_index; import physical_command; +import physical_compact; +import physical_compact_index_prepare; +import physical_compact_index_do; +import physical_compact_finish; import physical_match; import physical_fusion; import physical_create_index_prepare; @@ -111,6 +115,9 @@ import logical_dummy_scan; import logical_explain; import logical_drop_index; import logical_command; +import logical_compact; +import logical_compact_index; +import logical_compact_finish; import logical_match; import logical_fusion; @@ -124,6 +131,7 @@ import create_index_info; import command_statement; import explain_statement; import load_meta; +import block_index; namespace infinity { @@ -289,10 +297,22 @@ UniquePtr PhysicalPlanner::BuildPhysicalOperator(const SharedP result = BuildCommand(logical_operator); break; } + case LogicalNodeType::kCompact: { + result = BuildCompact(logical_operator); + break; + } case LogicalNodeType::kExplain: { result = BuildExplain(logical_operator); break; } + case LogicalNodeType::kCompactIndex: { + result = BuildCompactIndex(logical_operator); + break; + } + case LogicalNodeType::kCompactFinish: { + result = BuildCompactFinish(logical_operator); + break; + } default: { UnrecoverableError(fmt::format("Unknown logical node type: {}", logical_operator->name())); // result = MakeShared(numeric_limits::max()); @@ -892,15 +912,93 @@ UniquePtr PhysicalPlanner::BuildKnn(const SharedPtr PhysicalPlanner::BuildCommand(const SharedPtr &logical_operator) const { auto *logical_command = (LogicalCommand *)(logical_operator.get()); auto command_info = logical_command->command_info(); - auto ret = MakeUnique(logical_command->node_id(), - command_info, - logical_command->GetOutputNames(), - logical_command->GetOutputTypes(), - logical_operator->load_metas()); - if (command_info->type() == CommandType::kCompactTable) { - ret->table_entry_ = logical_command->table_entry_; + return MakeUnique(logical_command->node_id(), + command_info, + logical_command->GetOutputNames(), + logical_command->GetOutputTypes(), + logical_operator->load_metas()); +} + +UniquePtr PhysicalPlanner::BuildCompact(const SharedPtr &logical_operator) const { + const auto *logical_compact = static_cast(logical_operator.get()); + if (logical_compact->left_node().get() != nullptr || logical_compact->right_node().get() != nullptr) { + UnrecoverableError("Compact node shouldn't have child."); + } + return MakeUnique(logical_compact->node_id(), + logical_compact->base_table_ref_, + logical_compact->compact_type_, + logical_compact->GetOutputNames(), + logical_compact->GetOutputTypes(), + logical_operator->load_metas()); +} + +UniquePtr PhysicalPlanner::BuildCompactIndex(const SharedPtr &logical_operator) const { + const auto *logical_compact_index = static_cast(logical_operator.get()); + if (logical_compact_index->right_node().get() != nullptr) { + UnrecoverableError("Compact index node shouldn't have right child."); + } + UniquePtr left{}; + if (logical_compact_index->left_node().get() != nullptr) { + const auto &left_node = logical_compact_index->left_node(); + left = BuildPhysicalOperator(left_node); + } + auto &index_index = logical_compact_index->base_table_ref_->index_index_; + + bool use_prepare = false; + for (const auto &[index_name, index_snapshot] : index_index->index_snapshots_) { + const auto *index_base = index_snapshot->table_index_entry_->index_base(); + if (index_base->index_type_ == IndexType::kHnsw) { + use_prepare = true; + break; + } + } + if (!use_prepare) { + return MakeUnique(logical_compact_index->node_id(), + std::move(left), + logical_compact_index->base_table_ref_, + false, + logical_compact_index->GetOutputNames(), + logical_compact_index->GetOutputTypes(), + logical_operator->load_metas()); + } + auto compact_index_prepare = MakeUnique(logical_compact_index->node_id(), + std::move(left), + logical_compact_index->base_table_ref_, + true, + logical_compact_index->GetOutputNames(), + logical_compact_index->GetOutputTypes(), + logical_operator->load_metas()); + + auto compact_index_do = MakeUnique(logical_compact_index->node_id(), + std::move(compact_index_prepare), + logical_compact_index->base_table_ref_, + logical_compact_index->GetOutputNames(), + logical_compact_index->GetOutputTypes(), + logical_operator->load_metas()); + return compact_index_do; +} + +UniquePtr PhysicalPlanner::BuildCompactFinish(const SharedPtr &logical_operator) const { + const auto *logical_compact_finish = static_cast(logical_operator.get()); + UniquePtr left{}, right{}; + if (logical_compact_finish->left_node().get() != nullptr) { + const auto &left_logical_node = logical_compact_finish->left_node(); + left = BuildPhysicalOperator(left_logical_node); + if (logical_compact_finish->right_node().get() != nullptr) { + const auto &right_logical_node = logical_compact_finish->right_node(); + right = BuildPhysicalOperator(right_logical_node); + } + } else if (logical_compact_finish->right_node().get() != nullptr) { + UnrecoverableError("Compact finish node shouldn't have right child."); } - return ret; + return MakeUnique(logical_compact_finish->node_id(), + std::move(left), + std::move(right), + logical_compact_finish->base_table_ref_, + logical_compact_finish->compact_type_, + logical_compact_finish->GetOutputNames(), + logical_compact_finish->GetOutputTypes(), + logical_operator->load_metas()); } UniquePtr PhysicalPlanner::BuildExplain(const SharedPtr &logical_operator) const { diff --git a/src/executor/physical_planner.cppm b/src/executor/physical_planner.cppm index b08b6369f4..25c6262236 100644 --- a/src/executor/physical_planner.cppm +++ b/src/executor/physical_planner.cppm @@ -123,6 +123,13 @@ private: // Command [[nodiscard]] UniquePtr BuildCommand(const SharedPtr &logical_operator) const; + // Compact + [[nodiscard]] UniquePtr BuildCompact(const SharedPtr &logical_operator) const; + + [[nodiscard]] UniquePtr BuildCompactIndex(const SharedPtr &logical_operator) const; + + [[nodiscard]] UniquePtr BuildCompactFinish(const SharedPtr &logical_operator) const; + // Explain [[nodiscard]] UniquePtr BuildExplain(const SharedPtr &logical_operator) const; }; diff --git a/src/function/table/compact_state_data.cppm b/src/function/table/compact_state_data.cppm new file mode 100644 index 0000000000..b65a890e8d --- /dev/null +++ b/src/function/table/compact_state_data.cppm @@ -0,0 +1,123 @@ +// 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 + +export module compact_state_data; + +import stl; +import infinity_exception; +import global_block_id; +import internal_types; +import default_values; +import table_entry; +import table_index_entry; +import segment_entry; +import base_table_ref; +import block_index; +import txn; + +namespace infinity { + +export class RowIDRemap { + using RowIDMap = HashMap>, GlobalBlockIDHash>; + +public: + RowIDRemap(SizeT block_capacity = DEFAULT_BLOCK_CAPACITY) : block_capacity_(block_capacity) {} + + void AddMap(SegmentID segment_id, BlockID block_id, BlockOffset block_offset, RowID new_row_id) { + std::lock_guard lock(mutex_); + auto &block_vec = row_id_map_[GlobalBlockID(segment_id, block_id)]; + block_vec.emplace_back(block_offset, new_row_id); + } + + RowID GetNewRowID(SegmentID segment_id, BlockID block_id, BlockOffset block_offset) const { + auto &block_vec = row_id_map_.at(GlobalBlockID(segment_id, block_id)); + auto iter = + std::upper_bound(block_vec.begin(), + block_vec.end(), + block_offset, + [](BlockOffset block_offset, const Pair &pair) { return block_offset < pair.first; } // NOLINT + ); + if (iter == block_vec.begin()) { + UnrecoverableError("RowID not found"); + } + --iter; + RowID rtn = iter->second; + rtn.segment_offset_ += iter->first - block_offset; + return rtn; + } + + void AddMap(RowID old_row_id, RowID new_row_id) { + std::lock_guard lock(mutex_); + AddMap(old_row_id.segment_id_, old_row_id.segment_offset_ / block_capacity_, old_row_id.segment_offset_ % block_capacity_, new_row_id); + } + + RowID GetNewRowID(RowID old_row_id) const { + return GetNewRowID(old_row_id.segment_id_, old_row_id.segment_offset_ / block_capacity_, old_row_id.segment_offset_ % block_capacity_); + } + +private: + std::mutex mutex_; + const SizeT block_capacity_; + + RowIDMap row_id_map_; +}; + +export class CompactSegmentData { +public: + SharedPtr new_segment_{}; + Vector old_segments_{}; +}; + +export class CompactStateData { +public: + CompactStateData(TableEntry *table_entry) : new_table_ref_(MakeShared(table_entry, MakeShared())){}; + + void AddToDelete(SegmentID segment_id, const Vector &delete_offsets) { + std::lock_guard lock(mutex_); + to_delete_[segment_id].insert(to_delete_[segment_id].end(), delete_offsets.begin(), delete_offsets.end()); + } + + const HashMap> &GetToDelete() const { return to_delete_; } + + void AddNewSegment(SegmentEntry *new_segment, Txn *txn) { + std::lock_guard lock(mutex2_); + auto *block_index = new_table_ref_->block_index_.get(); + block_index->Insert(new_segment, txn); + } + + void AddNewIndex(TableIndexEntry *table_index_entry, Txn *txn) { + std::lock_guard lock(mutex2_); + auto *index_index = new_table_ref_->index_index_.get(); + index_index->Insert(table_index_entry, txn); + } + + BaseTableRef *GetNewTableRef() const { return new_table_ref_.get(); } + +public: + Vector segment_data_list_; + RowIDRemap remapper_{}; + +private: + std::mutex mutex_; + HashMap> to_delete_; + + std::mutex mutex2_; + SharedPtr new_table_ref_{}; // table ref after compact +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/function/table/create_index_data.cppm b/src/function/table/create_index_data.cppm index 1afc14afd5..4e3391c563 100644 --- a/src/function/table/create_index_data.cppm +++ b/src/function/table/create_index_data.cppm @@ -25,12 +25,15 @@ import infinity_exception; namespace infinity { export struct CreateIndexSharedData { - explicit CreateIndexSharedData(BlockIndex *block_index) { - SizeT segment_count = block_index->segments_.size(); - for (SizeT i = 0; i < segment_count; ++i) { - auto [iter, insert_ok] = create_index_idxes_.emplace(block_index->segments_[i]->segment_id(), 0); + CreateIndexSharedData() = default; + + explicit CreateIndexSharedData(BlockIndex *block_index) { Init(block_index); } + + void Init(BlockIndex *block_index) { + for (const auto &[segment_id, segment_info] : block_index->segment_block_index_) { + auto [iter, insert_ok] = create_index_idxes_.emplace(segment_id, 0); if (!insert_ok) { - UnrecoverableError(fmt::format("Duplicate segment id: %u", block_index->segments_[i]->segment_id())); + UnrecoverableError(fmt::format("Duplicate segment id: %u", segment_id)); } } } diff --git a/src/main/bg_query_state.cppm b/src/main/bg_query_state.cppm new file mode 100644 index 0000000000..b33ef43b81 --- /dev/null +++ b/src/main/bg_query_state.cppm @@ -0,0 +1,34 @@ +// 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; + +export module bg_query_state; + +import stl; +import logical_node; +import physical_operator; +import plan_fragment; +import fragment_context; + +namespace infinity { + +export struct BGQueryState { + Vector> logical_plans{}; + Vector> physical_plans{}; + SharedPtr plan_fragment{}; + UniquePtr notifier{}; +}; + +} // namespace infinity diff --git a/src/main/query_context.cpp b/src/main/query_context.cpp index cd70503f08..24326530e5 100644 --- a/src/main/query_context.cpp +++ b/src/main/query_context.cpp @@ -51,6 +51,7 @@ import base_statement; import parser_result; import parser_assert; import plan_fragment; +import bg_query_state; namespace infinity { @@ -107,10 +108,12 @@ QueryResult QueryContext::Query(const String &query) { QueryResult QueryContext::QueryStatement(const BaseStatement *statement) { QueryResult query_result; - SharedPtr logical_plan = nullptr; - UniquePtr plan_fragment = nullptr; - UniquePtr physical_plan = nullptr; + Vector> logical_plans{}; + Vector> physical_plans{}; + SharedPtr plan_fragment{}; + UniquePtr notifier{}; + this->BeginTxn(); // ProfilerStart("Query"); // BaseProfiler profiler; // profiler.Begin(); @@ -132,35 +135,44 @@ QueryResult QueryContext::QueryStatement(const BaseStatement *statement) { } current_max_node_id_ = bind_context->GetNewLogicalNodeId(); - logical_plan = logical_planner_->LogicalPlan(); + logical_plans = logical_planner_->LogicalPlans(); StopProfile(QueryPhase::kLogicalPlan); // LOG_WARN(fmt::format("Before optimizer cost: {}", profiler.ElapsedToString())); // Apply optimized rule to the logical plan StartProfile(QueryPhase::kOptimizer); - optimizer_->optimize(logical_plan, statement->type_); + for (auto &logical_plan : logical_plans) { + optimizer_->optimize(logical_plan, statement->type_); + } StopProfile(QueryPhase::kOptimizer); // Build physical plan StartProfile(QueryPhase::kPhysicalPlan); - physical_plan = physical_planner_->BuildPhysicalOperator(logical_plan); + for (auto &logical_plan : logical_plans) { + auto physical_plan = physical_planner_->BuildPhysicalOperator(logical_plan); + physical_plans.push_back(std::move(physical_plan)); + } StopProfile(QueryPhase::kPhysicalPlan); // LOG_WARN(fmt::format("Before pipeline cost: {}", profiler.ElapsedToString())); StartProfile(QueryPhase::kPipelineBuild); // Fragment Builder, only for test now. - // SharedPtr plan_fragment = fragment_builder.Build(physical_plan); - plan_fragment = fragment_builder_->BuildFragment(physical_plan.get()); + { + Vector physical_plan_ptrs; + for (auto &physical_plan : physical_plans) { + physical_plan_ptrs.push_back(physical_plan.get()); + } + plan_fragment = fragment_builder_->BuildFragment(physical_plan_ptrs); + } StopProfile(QueryPhase::kPipelineBuild); - auto notifier = MakeUnique(); - StartProfile(QueryPhase::kTaskBuild); + notifier = MakeUnique(); FragmentContext::BuildTask(this, nullptr, plan_fragment.get(), notifier.get()); StopProfile(QueryPhase::kTaskBuild); // LOG_WARN(fmt::format("Before execution cost: {}", profiler.ElapsedToString())); StartProfile(QueryPhase::kExecution); scheduler_->Schedule(plan_fragment.get(), statement); query_result.result_table_ = plan_fragment->GetResult(); - query_result.root_operator_type_ = logical_plan->operator_type(); + query_result.root_operator_type_ = logical_plans.back()->operator_type(); StopProfile(QueryPhase::kExecution); // LOG_WARN(fmt::format("Before commit cost: {}", profiler.ElapsedToString())); StartProfile(QueryPhase::kCommit); @@ -196,6 +208,70 @@ QueryResult QueryContext::QueryStatement(const BaseStatement *statement) { return query_result; } +bool QueryContext::ExecuteBGStatement(BaseStatement *statement, BGQueryState &state) { + QueryResult query_result; + try { + SharedPtr bind_context; + auto status = logical_planner_->Build(statement, bind_context); + if (!status.ok()) { + RecoverableError(status); + } + current_max_node_id_ = bind_context->GetNewLogicalNodeId(); + state.logical_plans = logical_planner_->LogicalPlans(); + + for (auto &logical_plan : state.logical_plans) { + auto physical_plan = physical_planner_->BuildPhysicalOperator(logical_plan); + state.physical_plans.push_back(std::move(physical_plan)); + } + + { + Vector physical_plan_ptrs; + for (auto &physical_plan : state.physical_plans) { + physical_plan_ptrs.push_back(physical_plan.get()); + } + state.plan_fragment = fragment_builder_->BuildFragment(physical_plan_ptrs); + } + + state.notifier = MakeUnique(); + FragmentContext::BuildTask(this, nullptr, state.plan_fragment.get(), state.notifier.get()); + + scheduler_->Schedule(state.plan_fragment.get(), statement); + } catch (RecoverableException &e) { + this->RollbackTxn(); + query_result.result_table_ = nullptr; + query_result.status_.Init(e.ErrorCode(), e.what()); + return false; + + } catch (UnrecoverableException &e) { + LOG_CRITICAL(e.what()); + raise(SIGUSR1); + } + return true; +} + +bool QueryContext::JoinBGStatement(BGQueryState &state, TxnTimeStamp &commit_ts, bool rollback) { + QueryResult query_result; + if (rollback) { + query_result.result_table_ = state.plan_fragment->GetResult(); + this->RollbackTxn(); + return false; + } + try { + query_result.result_table_ = state.plan_fragment->GetResult(); + query_result.root_operator_type_ = state.logical_plans.back()->operator_type(); + commit_ts = this->CommitTxn(); + } catch (RecoverableException &e) { + query_result.result_table_ = nullptr; + query_result.status_.Init(e.ErrorCode(), e.what()); + this->RollbackTxn(); + return false; + } catch (UnrecoverableException &e) { + LOG_CRITICAL(e.what()); + raise(SIGUSR1); + } + return true; +} + void QueryContext::BeginTxn() { if (session_ptr_->GetTxn() == nullptr) { Txn* new_txn = storage_->txn_manager()->BeginTxn(nullptr); @@ -203,12 +279,13 @@ void QueryContext::BeginTxn() { } } -void QueryContext::CommitTxn() { +TxnTimeStamp QueryContext::CommitTxn() { Txn* txn = session_ptr_->GetTxn(); - storage_->txn_manager()->CommitTxn(txn); + TxnTimeStamp commit_ts = storage_->txn_manager()->CommitTxn(txn); session_ptr_->SetTxn(nullptr); session_ptr_->IncreaseCommittedTxnCount(); storage_->txn_manager()->IncreaseCommittedTxnCount(); + return commit_ts; } void QueryContext::RollbackTxn() { diff --git a/src/main/query_context.cppm b/src/main/query_context.cppm index c312945e3a..732964a86e 100644 --- a/src/main/query_context.cppm +++ b/src/main/query_context.cppm @@ -38,6 +38,7 @@ class LogicalPlanner; class PhysicalPlanner; class FragmentBuilder; class TaskScheduler; +struct BGQueryState; export class QueryContext { @@ -65,6 +66,10 @@ public: QueryResult QueryStatement(const BaseStatement *statement); + bool ExecuteBGStatement(BaseStatement *statement, BGQueryState &state); + + bool JoinBGStatement(BGQueryState &state, TxnTimeStamp &commit_ts, bool rollback = false); + inline void set_current_schema(const String ¤t_schema) { session_ptr_->set_current_schema(current_schema); } [[nodiscard]] inline const String &schema_name() const { return session_ptr_->current_database(); } @@ -85,13 +90,21 @@ public: void BeginTxn(); - void CommitTxn(); + TxnTimeStamp CommitTxn(); void RollbackTxn(); [[nodiscard]] Txn *GetTxn() const { return session_ptr_->GetTxn(); } + bool SetTxn(Txn *txn) const { + if (session_ptr_->GetTxn() == nullptr) { + session_ptr_->SetTxn(txn); + return true; + } + return false; + } + [[nodiscard]] inline Storage *storage() const { return storage_; } [[nodiscard]] inline TaskScheduler *scheduler() const { return scheduler_; } diff --git a/src/parser/base_statement.h b/src/parser/base_statement.h index 6d3af832cb..8a75383670 100644 --- a/src/parser/base_statement.h +++ b/src/parser/base_statement.h @@ -35,6 +35,7 @@ enum class StatementType { kFlush, kOptimize, kCommand, + kCompact, }; class BaseStatement { diff --git a/src/parser/parser.cpp b/src/parser/parser.cpp index 76e0ccdfe2..760b7111e9 100644 --- a/src/parser/parser.cpp +++ b/src/parser/parser.cpp @@ -353,58 +353,59 @@ enum yysymbol_kind_t YYSYMBOL_flush_statement = 239, /* flush_statement */ YYSYMBOL_optimize_statement = 240, /* optimize_statement */ YYSYMBOL_command_statement = 241, /* command_statement */ - YYSYMBOL_expr_array = 242, /* expr_array */ - YYSYMBOL_expr_array_list = 243, /* expr_array_list */ - YYSYMBOL_expr_alias = 244, /* expr_alias */ - YYSYMBOL_expr = 245, /* expr */ - YYSYMBOL_operand = 246, /* operand */ - YYSYMBOL_knn_expr = 247, /* knn_expr */ - YYSYMBOL_match_expr = 248, /* match_expr */ - YYSYMBOL_query_expr = 249, /* query_expr */ - YYSYMBOL_fusion_expr = 250, /* fusion_expr */ - YYSYMBOL_sub_search_array = 251, /* sub_search_array */ - YYSYMBOL_function_expr = 252, /* function_expr */ - YYSYMBOL_conjunction_expr = 253, /* conjunction_expr */ - YYSYMBOL_between_expr = 254, /* between_expr */ - YYSYMBOL_in_expr = 255, /* in_expr */ - YYSYMBOL_case_expr = 256, /* case_expr */ - YYSYMBOL_case_check_array = 257, /* case_check_array */ - YYSYMBOL_cast_expr = 258, /* cast_expr */ - YYSYMBOL_subquery_expr = 259, /* subquery_expr */ - YYSYMBOL_column_expr = 260, /* column_expr */ - YYSYMBOL_constant_expr = 261, /* constant_expr */ - YYSYMBOL_array_expr = 262, /* array_expr */ - YYSYMBOL_long_array_expr = 263, /* long_array_expr */ - YYSYMBOL_unclosed_long_array_expr = 264, /* unclosed_long_array_expr */ - YYSYMBOL_double_array_expr = 265, /* double_array_expr */ - YYSYMBOL_unclosed_double_array_expr = 266, /* unclosed_double_array_expr */ - YYSYMBOL_interval_expr = 267, /* interval_expr */ - YYSYMBOL_copy_option_list = 268, /* copy_option_list */ - YYSYMBOL_copy_option = 269, /* copy_option */ - YYSYMBOL_file_path = 270, /* file_path */ - YYSYMBOL_if_exists = 271, /* if_exists */ - YYSYMBOL_if_not_exists = 272, /* if_not_exists */ - YYSYMBOL_semicolon = 273, /* semicolon */ - YYSYMBOL_if_not_exists_info = 274, /* if_not_exists_info */ - YYSYMBOL_with_index_param_list = 275, /* with_index_param_list */ - YYSYMBOL_optional_table_properties_list = 276, /* optional_table_properties_list */ - YYSYMBOL_index_param_list = 277, /* index_param_list */ - YYSYMBOL_index_param = 278, /* index_param */ - YYSYMBOL_index_info_list = 279 /* index_info_list */ + YYSYMBOL_compact_statement = 242, /* compact_statement */ + YYSYMBOL_expr_array = 243, /* expr_array */ + YYSYMBOL_expr_array_list = 244, /* expr_array_list */ + YYSYMBOL_expr_alias = 245, /* expr_alias */ + YYSYMBOL_expr = 246, /* expr */ + YYSYMBOL_operand = 247, /* operand */ + YYSYMBOL_knn_expr = 248, /* knn_expr */ + YYSYMBOL_match_expr = 249, /* match_expr */ + YYSYMBOL_query_expr = 250, /* query_expr */ + YYSYMBOL_fusion_expr = 251, /* fusion_expr */ + YYSYMBOL_sub_search_array = 252, /* sub_search_array */ + YYSYMBOL_function_expr = 253, /* function_expr */ + YYSYMBOL_conjunction_expr = 254, /* conjunction_expr */ + YYSYMBOL_between_expr = 255, /* between_expr */ + YYSYMBOL_in_expr = 256, /* in_expr */ + YYSYMBOL_case_expr = 257, /* case_expr */ + YYSYMBOL_case_check_array = 258, /* case_check_array */ + YYSYMBOL_cast_expr = 259, /* cast_expr */ + YYSYMBOL_subquery_expr = 260, /* subquery_expr */ + YYSYMBOL_column_expr = 261, /* column_expr */ + YYSYMBOL_constant_expr = 262, /* constant_expr */ + YYSYMBOL_array_expr = 263, /* array_expr */ + YYSYMBOL_long_array_expr = 264, /* long_array_expr */ + YYSYMBOL_unclosed_long_array_expr = 265, /* unclosed_long_array_expr */ + YYSYMBOL_double_array_expr = 266, /* double_array_expr */ + YYSYMBOL_unclosed_double_array_expr = 267, /* unclosed_double_array_expr */ + YYSYMBOL_interval_expr = 268, /* interval_expr */ + YYSYMBOL_copy_option_list = 269, /* copy_option_list */ + YYSYMBOL_copy_option = 270, /* copy_option */ + YYSYMBOL_file_path = 271, /* file_path */ + YYSYMBOL_if_exists = 272, /* if_exists */ + YYSYMBOL_if_not_exists = 273, /* if_not_exists */ + YYSYMBOL_semicolon = 274, /* semicolon */ + YYSYMBOL_if_not_exists_info = 275, /* if_not_exists_info */ + YYSYMBOL_with_index_param_list = 276, /* with_index_param_list */ + YYSYMBOL_optional_table_properties_list = 277, /* optional_table_properties_list */ + YYSYMBOL_index_param_list = 278, /* index_param_list */ + YYSYMBOL_index_param = 279, /* index_param */ + YYSYMBOL_index_info_list = 280 /* index_info_list */ }; typedef enum yysymbol_kind_t yysymbol_kind_t; /* Unqualified %code blocks. */ -#line 95 "parser.y" +#line 96 "parser.y" // Avoid warnings with the error counter. #if defined(__GNUC__) || defined(__clang__) #pragma GCC diagnostic ignored "-Wunused-but-set-variable" #endif -#line 408 "parser.cpp" +#line 409 "parser.cpp" #ifdef short # undef short @@ -728,18 +729,18 @@ union yyalloc #endif /* !YYCOPY_NEEDED */ /* YYFINAL -- State number of the termination state. */ -#define YYFINAL 83 +#define YYFINAL 84 /* YYLAST -- Last index in YYTABLE. */ -#define YYLAST 936 +#define YYLAST 938 /* YYNTOKENS -- Number of terminals. */ #define YYNTOKENS 184 /* YYNNTS -- Number of nonterminals. */ -#define YYNNTS 96 +#define YYNNTS 97 /* YYNRULES -- Number of rules. */ -#define YYNRULES 371 +#define YYNRULES 373 /* YYNSTATES -- Number of states. */ -#define YYNSTATES 737 +#define YYNSTATES 739 /* YYMAXUTOK -- Last valid token kind. */ #define YYMAXUTOK 423 @@ -805,44 +806,44 @@ static const yytype_uint8 yytranslate[] = /* YYRLINE[YYN] -- Source line where rule number YYN was defined. */ static const yytype_int16 yyrline[] = { - 0, 473, 473, 477, 483, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 503, 504, 505, - 506, 507, 508, 509, 510, 511, 512, 513, 520, 537, - 553, 582, 598, 616, 645, 649, 655, 658, 664, 702, - 738, 739, 740, 741, 742, 743, 744, 745, 746, 747, - 748, 749, 750, 751, 752, 753, 754, 755, 756, 759, - 761, 762, 763, 764, 767, 768, 769, 770, 771, 772, - 773, 774, 775, 776, 777, 778, 779, 780, 781, 782, - 783, 784, 785, 786, 787, 788, 789, 790, 809, 813, - 823, 826, 829, 832, 836, 839, 844, 849, 856, 862, - 872, 888, 922, 935, 938, 945, 951, 954, 957, 960, - 963, 966, 969, 972, 979, 992, 996, 1001, 1014, 1027, - 1042, 1057, 1072, 1095, 1136, 1181, 1184, 1187, 1196, 1206, - 1209, 1213, 1218, 1240, 1243, 1248, 1264, 1267, 1271, 1275, - 1280, 1286, 1289, 1292, 1296, 1300, 1302, 1306, 1308, 1311, - 1315, 1318, 1322, 1327, 1331, 1334, 1338, 1341, 1345, 1348, - 1352, 1355, 1358, 1361, 1369, 1372, 1387, 1387, 1389, 1403, - 1412, 1417, 1426, 1431, 1436, 1442, 1449, 1452, 1456, 1459, - 1464, 1476, 1483, 1497, 1500, 1503, 1506, 1509, 1512, 1515, - 1521, 1525, 1529, 1533, 1537, 1544, 1548, 1552, 1556, 1562, - 1568, 1574, 1585, 1596, 1607, 1619, 1631, 1644, 1658, 1669, - 1687, 1691, 1695, 1703, 1717, 1723, 1728, 1734, 1740, 1748, - 1754, 1760, 1766, 1772, 1780, 1786, 1792, 1798, 1804, 1812, - 1818, 1824, 1840, 1844, 1849, 1853, 1880, 1886, 1890, 1891, - 1892, 1893, 1894, 1896, 1899, 1905, 1908, 1909, 1910, 1911, - 1912, 1913, 1914, 1915, 1917, 2084, 2092, 2103, 2109, 2118, - 2124, 2134, 2138, 2142, 2146, 2150, 2154, 2158, 2162, 2167, - 2175, 2183, 2192, 2199, 2206, 2213, 2220, 2227, 2235, 2243, - 2251, 2259, 2267, 2275, 2283, 2291, 2299, 2307, 2315, 2323, - 2353, 2361, 2370, 2378, 2387, 2395, 2401, 2408, 2414, 2421, - 2426, 2433, 2440, 2448, 2472, 2478, 2484, 2491, 2499, 2506, - 2513, 2518, 2528, 2533, 2538, 2543, 2548, 2553, 2558, 2563, - 2568, 2573, 2576, 2579, 2582, 2586, 2589, 2593, 2597, 2602, - 2607, 2611, 2616, 2621, 2627, 2633, 2639, 2645, 2651, 2657, - 2663, 2669, 2675, 2681, 2687, 2698, 2702, 2707, 2729, 2739, - 2745, 2749, 2750, 2752, 2753, 2755, 2756, 2768, 2776, 2780, - 2783, 2787, 2790, 2794, 2798, 2803, 2808, 2816, 2823, 2834, - 2882, 2931 + 0, 476, 476, 480, 486, 493, 494, 495, 496, 497, + 498, 499, 500, 501, 502, 503, 504, 505, 507, 508, + 509, 510, 511, 512, 513, 514, 515, 516, 517, 518, + 525, 542, 558, 587, 603, 621, 650, 654, 660, 663, + 669, 707, 743, 744, 745, 746, 747, 748, 749, 750, + 751, 752, 753, 754, 755, 756, 757, 758, 759, 760, + 761, 764, 766, 767, 768, 769, 772, 773, 774, 775, + 776, 777, 778, 779, 780, 781, 782, 783, 784, 785, + 786, 787, 788, 789, 790, 791, 792, 793, 794, 795, + 814, 818, 828, 831, 834, 837, 841, 844, 849, 854, + 861, 867, 877, 893, 927, 940, 943, 950, 956, 959, + 962, 965, 968, 971, 974, 977, 984, 997, 1001, 1006, + 1019, 1032, 1047, 1062, 1077, 1100, 1141, 1186, 1189, 1192, + 1201, 1211, 1214, 1218, 1223, 1245, 1248, 1253, 1269, 1272, + 1276, 1280, 1285, 1291, 1294, 1297, 1301, 1305, 1307, 1311, + 1313, 1316, 1320, 1323, 1327, 1332, 1336, 1339, 1343, 1346, + 1350, 1353, 1357, 1360, 1363, 1366, 1374, 1377, 1392, 1392, + 1394, 1408, 1417, 1422, 1431, 1436, 1441, 1447, 1454, 1457, + 1461, 1464, 1469, 1481, 1488, 1502, 1505, 1508, 1511, 1514, + 1517, 1520, 1526, 1530, 1534, 1538, 1542, 1549, 1553, 1557, + 1561, 1567, 1573, 1579, 1590, 1601, 1612, 1624, 1636, 1649, + 1663, 1674, 1692, 1696, 1700, 1708, 1722, 1728, 1733, 1739, + 1745, 1753, 1759, 1765, 1771, 1777, 1785, 1791, 1797, 1803, + 1809, 1817, 1823, 1830, 1847, 1851, 1856, 1860, 1887, 1893, + 1897, 1898, 1899, 1900, 1901, 1903, 1906, 1912, 1915, 1916, + 1917, 1918, 1919, 1920, 1921, 1922, 1924, 2091, 2099, 2110, + 2116, 2125, 2131, 2141, 2145, 2149, 2153, 2157, 2161, 2165, + 2169, 2174, 2182, 2190, 2199, 2206, 2213, 2220, 2227, 2234, + 2242, 2250, 2258, 2266, 2274, 2282, 2290, 2298, 2306, 2314, + 2322, 2330, 2360, 2368, 2377, 2385, 2394, 2402, 2408, 2415, + 2421, 2428, 2433, 2440, 2447, 2455, 2479, 2485, 2491, 2498, + 2506, 2513, 2520, 2525, 2535, 2540, 2545, 2550, 2555, 2560, + 2565, 2570, 2575, 2580, 2583, 2586, 2589, 2593, 2596, 2600, + 2604, 2609, 2614, 2618, 2623, 2628, 2634, 2640, 2646, 2652, + 2658, 2664, 2670, 2676, 2682, 2688, 2694, 2705, 2709, 2714, + 2736, 2746, 2752, 2756, 2757, 2759, 2760, 2762, 2763, 2775, + 2783, 2787, 2790, 2794, 2797, 2801, 2805, 2810, 2815, 2823, + 2830, 2841, 2889, 2938 }; #endif @@ -900,10 +901,11 @@ static const char *const yytname[] = "table_reference_name", "table_name", "table_alias", "with_clause", "with_expr_list", "with_expr", "join_clause", "join_type", "show_statement", "flush_statement", "optimize_statement", - "command_statement", "expr_array", "expr_array_list", "expr_alias", - "expr", "operand", "knn_expr", "match_expr", "query_expr", "fusion_expr", - "sub_search_array", "function_expr", "conjunction_expr", "between_expr", - "in_expr", "case_expr", "case_check_array", "cast_expr", "subquery_expr", + "command_statement", "compact_statement", "expr_array", + "expr_array_list", "expr_alias", "expr", "operand", "knn_expr", + "match_expr", "query_expr", "fusion_expr", "sub_search_array", + "function_expr", "conjunction_expr", "between_expr", "in_expr", + "case_expr", "case_check_array", "cast_expr", "subquery_expr", "column_expr", "constant_expr", "array_expr", "long_array_expr", "unclosed_long_array_expr", "double_array_expr", "unclosed_double_array_expr", "interval_expr", "copy_option_list", @@ -920,12 +922,12 @@ yysymbol_name (yysymbol_kind_t yysymbol) } #endif -#define YYPACT_NINF (-647) +#define YYPACT_NINF (-644) #define yypact_value_is_default(Yyn) \ ((Yyn) == YYPACT_NINF) -#define YYTABLE_NINF (-359) +#define YYTABLE_NINF (-361) #define yytable_value_is_error(Yyn) \ ((Yyn) == YYTABLE_NINF) @@ -934,80 +936,80 @@ yysymbol_name (yysymbol_kind_t yysymbol) STATE-NUM. */ static const yytype_int16 yypact[] = { - 702, 173, 44, 200, 46, 28, 46, -96, 428, 58, - 53, 296, 125, 46, 135, -18, -55, 184, -20, -647, - -647, -647, -647, -647, -647, -647, -647, 158, -647, -647, - 178, -647, -647, -647, -647, 118, 118, 118, 118, -2, - 46, 120, 120, 120, 120, 120, 30, 213, 46, 94, - 230, 251, 260, -647, -647, -647, -647, -647, -647, -647, - 723, 269, 46, -647, -647, -647, 137, 201, -647, 271, - -647, 46, -647, -647, -647, -647, -647, 222, 110, -647, - 312, 147, 176, -647, 6, -647, 317, -647, -647, 0, - 287, -647, 290, 315, 367, 46, 46, 46, 401, 344, - 235, 356, 437, 46, 46, 46, 449, 473, 474, 416, - 483, 483, 49, 60, 69, -647, -647, -647, -647, -647, - -647, -647, 158, -647, -647, -647, -647, -647, 224, -647, - 488, -647, 491, -647, -647, 316, 135, 483, -647, -647, - -647, -647, 0, -647, -647, -647, 445, 442, 429, 425, - -647, -44, -647, 235, -647, 46, 498, 108, -647, -647, - -647, -647, -647, 439, -647, 339, -53, -647, 445, -647, - -647, 430, 431, -647, -647, -647, -647, -647, -647, -647, - -647, -647, -647, -647, -647, -647, -647, -647, 508, 506, - -647, -647, -647, -647, -647, 178, -647, -647, 333, 334, - 342, -647, -647, 675, 500, 343, 350, 307, 519, 527, - 529, 534, -647, -647, 536, 364, 365, 366, 371, 372, - 531, 531, -647, 374, 414, -36, -647, -12, 615, -647, - -647, -647, -647, -647, -647, -647, -647, -647, -647, -647, - 381, -647, -647, -105, -647, -102, -647, 445, 445, 482, - -647, -55, 20, 507, 386, -647, -137, 388, -647, 46, - 445, 474, -647, 248, 389, 390, -647, 412, 393, -647, - -647, 194, -647, -647, -647, -647, -647, -647, -647, -647, - -647, -647, -647, -647, 531, 397, 652, 499, 445, 445, - -35, 138, -647, -647, -647, -647, 675, -647, 573, 445, - 578, 580, 585, 247, 247, -647, -647, 410, 75, 4, - 445, 432, 591, 445, 445, -51, 419, 48, 531, 531, - 531, 531, 531, 531, 531, 531, 531, 531, 531, 531, - 531, 531, 7, -647, 589, -647, 594, 417, -647, 23, - 248, 445, -647, 158, 740, 486, 434, 70, -647, -647, - -647, -55, 498, 436, -647, 604, 445, 426, -647, 248, - -647, 375, 375, 602, -647, -647, 445, -647, 82, 499, - 472, 441, -9, -45, 159, -647, 445, 445, 551, 61, - 440, 146, 160, -647, -647, -55, 446, 341, -647, 57, - -647, -647, 71, 416, -647, -647, 485, 452, 531, 414, - 520, -647, 116, 116, 254, 254, 161, 116, 116, 254, - 254, 247, 247, -647, -647, -647, -647, -647, -647, -647, - 445, -647, -647, -647, 248, -647, -647, -647, -647, -647, - -647, -647, -647, -647, -647, -647, 462, -647, -647, -647, - -647, -647, -647, -647, -647, -647, -647, 466, 467, 469, - 127, 470, 498, 622, 20, 158, 162, 498, -647, 164, - 475, 647, 650, -647, 168, -647, 169, 605, 174, -647, - 479, -647, 740, 445, -647, 445, -47, -30, 531, 484, - 656, -647, 658, -647, 660, 16, 4, 610, -647, -647, - -647, -647, -647, -647, 611, -647, 667, -647, -647, -647, - -647, -647, 492, 618, 414, 116, 496, 182, -647, 531, - -647, 674, 345, 546, 679, 559, 562, -647, -647, 63, - 127, -647, -647, 498, 189, 505, -647, -647, 532, 195, - -647, 445, -647, -647, -647, 375, -647, 680, -647, -647, - 510, 248, -39, -647, 445, 384, 504, -647, -647, 203, - 512, 513, 57, 341, 4, 4, 509, 71, 640, 644, - 515, 204, -647, -647, 652, 208, 517, 518, 523, 524, - 537, 538, 539, 540, 541, 548, 557, 558, 560, 561, - 564, 568, 569, 570, 571, 577, 579, 581, 582, 583, - -647, -647, -647, -647, -647, 232, -647, 699, 716, 592, - 233, -647, -647, -647, -647, 248, -647, 738, -647, 757, - -647, -647, -647, -647, 704, 498, -647, -647, -647, -647, - 445, 445, -647, -647, -647, -647, 762, 768, 772, 773, - 774, 776, 777, 791, 792, 793, 795, 809, 810, 811, - 812, 813, 814, 844, 845, 847, 848, 851, 854, 855, - 856, -647, 694, 261, -647, 785, 862, -647, 683, 687, - 445, 295, 685, 248, 689, 690, 691, 692, 693, 695, - 696, 697, 698, 703, 705, 706, 707, 708, 709, 710, - 711, 712, 713, 714, 715, 717, 718, 719, 720, 277, - -647, 699, 700, -647, 785, 870, -647, 248, -647, -647, - -647, -647, -647, -647, -647, -647, -647, -647, -647, -647, - -647, -647, -647, -647, -647, -647, -647, -647, -647, -647, - -647, -647, -647, -647, -647, -647, -647, -647, 699, -647, - 701, 302, 876, -647, 721, 785, -647 + 630, 33, 28, 175, 73, 45, 73, 79, 287, 193, + 60, 234, 134, 73, 164, -32, -43, 190, 66, -644, + -644, -644, -644, -644, -644, -644, -644, 35, -644, -644, + 184, -644, -644, -644, -644, -644, 219, 219, 219, 219, + -19, 73, 224, 224, 224, 224, 224, 83, 269, 73, + 6, 312, 328, 346, -644, -644, -644, -644, -644, -644, + -644, 259, 352, 73, -644, -644, -644, 68, 81, -644, + 354, -644, 73, -644, -644, -644, -644, -644, 246, 176, + -644, 362, 180, 189, -644, 171, -644, 358, -644, -644, + -2, 309, -644, 323, 322, 406, 73, 73, 73, 410, + 359, 236, 353, 426, 73, 73, 73, 433, 439, 455, + 392, 461, 461, 10, 14, 43, -644, -644, -644, -644, + -644, -644, -644, 35, -644, -644, -644, -644, -644, -644, + 194, -644, 468, -644, 475, -644, -644, 304, 164, 461, + -644, -644, -644, -644, -2, -644, -644, -644, 415, 430, + 419, 422, -644, -46, -644, 236, -644, 73, 496, 24, + -644, -644, -644, -644, -644, 441, -644, 336, -49, -644, + 415, -644, -644, 434, 435, -644, -644, -644, -644, -644, + -644, -644, -644, -644, -644, -644, -644, -644, -644, -644, + 512, 516, -644, -644, -644, -644, -644, 184, -644, -644, + 343, 344, 349, -644, -644, 471, 446, 351, 356, 329, + 532, 534, 539, 540, -644, -644, 549, 366, 367, 386, + 387, 389, 548, 548, -644, 282, 360, -55, -644, -35, + 603, -644, -644, -644, -644, -644, -644, -644, -644, -644, + -644, -644, 388, -644, -644, 46, -644, 64, -644, 415, + 415, 500, -644, -43, 8, 518, 396, -644, -111, 397, + -644, 73, 415, 455, -644, 191, 398, 399, -644, 296, + 403, -644, -644, 215, -644, -644, -644, -644, -644, -644, + -644, -644, -644, -644, -644, -644, 548, 400, 680, 505, + 415, 415, -50, 165, -644, -644, -644, -644, 471, -644, + 581, 415, 582, 586, 587, 272, 272, -644, -644, 416, + -33, 9, 415, 431, 594, 415, 415, -62, 436, -29, + 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, + 548, 548, 548, 548, 5, -644, 608, -644, 611, 438, + -644, -44, 191, 415, -644, 35, 805, 497, 445, -48, + -644, -644, -644, -43, 496, 447, -644, 624, 415, 449, + -644, 191, -644, 429, 429, 622, -644, -644, 415, -644, + 116, 505, 481, 450, -27, -57, 177, -644, 415, 415, + 557, -75, 452, 121, 123, -644, -644, -43, 464, 499, + -644, 44, -644, -644, -100, 392, -644, -644, 493, 469, + 548, 360, 536, -644, 717, 717, 84, 84, 643, 717, + 717, 84, 84, 272, 272, -644, -644, -644, -644, -644, + -644, -644, 415, -644, -644, -644, 191, -644, -644, -644, + -644, -644, -644, -644, -644, -644, -644, -644, 480, -644, + -644, -644, -644, -644, -644, -644, -644, -644, -644, 482, + 483, 484, 91, 485, 496, 626, 8, 35, 129, 496, + -644, 130, 486, 656, 663, -644, 131, -644, 163, 617, + 178, -644, 488, -644, 805, 415, -644, 415, -3, -47, + 548, 492, 666, -644, 667, -644, 668, 13, 9, 618, + -644, -644, -644, -644, -644, -644, 620, -644, 677, -644, + -644, -644, -644, -644, 498, 629, 360, 717, 503, 210, + -644, 548, -644, 678, 280, 308, 376, 564, 567, -644, + -644, 37, 91, -644, -644, 496, 227, 509, -644, -644, + 537, 228, -644, 415, -644, -644, -644, 429, -644, 683, + -644, -644, 511, 191, 23, -644, 415, 562, 510, -644, + -644, 243, 514, 517, 44, 499, 9, 9, 513, -100, + 642, 644, 521, 244, -644, -644, 680, 257, 515, 523, + 524, 525, 528, 529, 533, 535, 541, 545, 546, 547, + 556, 558, 565, 566, 569, 570, 572, 573, 574, 575, + 576, 577, -644, -644, -644, -644, -644, 261, -644, 716, + 720, 597, 277, -644, -644, -644, -644, 191, -644, 736, + -644, 757, -644, -644, -644, -644, 699, 496, -644, -644, + -644, -644, 415, 415, -644, -644, -644, -644, 758, 759, + 761, 762, 763, 764, 765, 774, 775, 776, 783, 784, + 786, 787, 788, 789, 790, 791, 792, 793, 794, 795, + 796, 798, 799, -644, 637, 292, -644, 684, 804, -644, + 625, 631, 415, 297, 627, 191, 645, 646, 647, 649, + 650, 651, 652, 653, 654, 655, 657, 658, 659, 661, + 662, 664, 665, 681, 682, 686, 687, 688, 689, 690, + 691, 288, -644, 716, 669, -644, 684, 820, -644, 191, + -644, -644, -644, -644, -644, -644, -644, -644, -644, -644, + -644, -644, -644, -644, -644, -644, -644, -644, -644, -644, + -644, -644, -644, -644, -644, -644, -644, -644, -644, -644, + 716, -644, 660, 299, 830, -644, 692, 684, -644 }; /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM. @@ -1015,110 +1017,110 @@ static const yytype_int16 yypact[] = means the default is an error. */ static const yytype_int16 yydefact[] = { - 177, 0, 0, 0, 0, 0, 0, 0, 113, 0, - 0, 0, 0, 0, 0, 0, 177, 0, 356, 3, - 5, 10, 12, 13, 11, 6, 7, 9, 126, 125, - 0, 8, 14, 15, 16, 354, 354, 354, 354, 354, - 0, 352, 352, 352, 352, 352, 170, 0, 0, 0, - 0, 0, 0, 107, 111, 108, 109, 110, 112, 106, - 177, 0, 0, 191, 192, 190, 0, 0, 193, 0, - 195, 0, 210, 211, 212, 214, 213, 0, 176, 178, - 0, 0, 0, 1, 177, 2, 160, 162, 163, 0, - 149, 131, 137, 0, 0, 0, 0, 0, 0, 0, - 104, 0, 0, 0, 0, 0, 0, 0, 0, 155, - 0, 0, 0, 0, 0, 105, 17, 22, 24, 23, - 18, 19, 21, 20, 25, 26, 27, 200, 201, 196, - 0, 197, 0, 194, 231, 0, 0, 0, 130, 129, - 4, 161, 0, 127, 128, 148, 0, 0, 145, 0, - 28, 0, 29, 104, 357, 0, 0, 177, 351, 118, - 120, 119, 121, 0, 171, 0, 155, 115, 0, 100, - 350, 0, 0, 218, 220, 219, 216, 217, 223, 225, - 224, 221, 222, 228, 230, 229, 226, 227, 0, 0, - 203, 202, 208, 198, 199, 0, 179, 215, 0, 0, - 308, 312, 315, 316, 0, 0, 0, 0, 0, 0, - 0, 0, 313, 314, 0, 0, 0, 0, 0, 0, - 0, 0, 310, 0, 177, 151, 232, 237, 238, 250, - 251, 252, 253, 247, 242, 241, 240, 248, 249, 239, - 246, 245, 323, 0, 324, 0, 322, 0, 0, 147, - 353, 177, 0, 0, 0, 98, 0, 0, 102, 0, - 0, 0, 114, 154, 0, 0, 209, 204, 0, 134, - 133, 0, 334, 333, 336, 335, 338, 337, 340, 339, - 342, 341, 344, 343, 0, 0, 274, 177, 0, 0, - 0, 0, 317, 318, 319, 320, 0, 321, 0, 0, - 0, 0, 0, 276, 275, 331, 328, 0, 0, 0, - 0, 153, 0, 0, 0, 0, 0, 0, 0, 0, + 179, 0, 0, 0, 0, 0, 0, 0, 115, 0, + 0, 0, 0, 0, 0, 0, 179, 0, 358, 3, + 5, 10, 12, 13, 11, 6, 7, 9, 128, 127, + 0, 8, 14, 15, 16, 17, 356, 356, 356, 356, + 356, 0, 354, 354, 354, 354, 354, 172, 0, 0, + 0, 0, 0, 0, 109, 113, 110, 111, 112, 114, + 108, 179, 0, 0, 193, 194, 192, 0, 0, 195, + 0, 197, 0, 212, 213, 214, 216, 215, 0, 178, + 180, 0, 0, 0, 1, 179, 2, 162, 164, 165, + 0, 151, 133, 139, 0, 0, 0, 0, 0, 0, + 0, 106, 0, 0, 0, 0, 0, 0, 0, 0, + 157, 0, 0, 0, 0, 0, 107, 18, 23, 25, + 24, 19, 20, 22, 21, 26, 27, 28, 29, 202, + 203, 198, 0, 199, 0, 196, 233, 0, 0, 0, + 132, 131, 4, 163, 0, 129, 130, 150, 0, 0, + 147, 0, 30, 0, 31, 106, 359, 0, 0, 179, + 353, 120, 122, 121, 123, 0, 173, 0, 157, 117, + 0, 102, 352, 0, 0, 220, 222, 221, 218, 219, + 225, 227, 226, 223, 224, 230, 232, 231, 228, 229, + 0, 0, 205, 204, 210, 200, 201, 0, 181, 217, + 0, 0, 310, 314, 317, 318, 0, 0, 0, 0, + 0, 0, 0, 0, 315, 316, 0, 0, 0, 0, + 0, 0, 0, 0, 312, 0, 179, 153, 234, 239, + 240, 252, 253, 254, 255, 249, 244, 243, 242, 250, + 251, 241, 248, 247, 325, 0, 326, 0, 324, 0, + 0, 149, 355, 179, 0, 0, 0, 100, 0, 0, + 104, 0, 0, 0, 116, 156, 0, 0, 211, 206, + 0, 136, 135, 0, 336, 335, 338, 337, 340, 339, + 342, 341, 344, 343, 346, 345, 0, 0, 276, 179, + 0, 0, 0, 0, 319, 320, 321, 322, 0, 323, + 0, 0, 0, 0, 0, 278, 277, 333, 330, 0, + 0, 0, 0, 155, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 327, 0, 330, 0, 136, 138, 143, - 144, 0, 132, 31, 0, 0, 0, 0, 34, 36, - 37, 177, 0, 33, 103, 0, 0, 101, 122, 117, - 116, 0, 0, 0, 205, 180, 0, 269, 0, 177, - 0, 0, 0, 0, 0, 299, 0, 0, 0, 0, - 0, 0, 0, 244, 243, 177, 150, 164, 166, 175, - 167, 233, 0, 155, 236, 292, 293, 0, 0, 177, - 0, 273, 283, 284, 287, 288, 0, 290, 282, 285, - 286, 278, 277, 279, 280, 281, 309, 311, 329, 332, - 0, 141, 142, 140, 146, 40, 43, 44, 41, 42, - 45, 46, 60, 47, 49, 48, 63, 50, 51, 52, - 53, 54, 55, 56, 57, 58, 59, 0, 0, 0, - 95, 0, 0, 362, 0, 32, 0, 0, 99, 0, - 0, 0, 0, 349, 0, 345, 0, 206, 0, 270, - 0, 304, 0, 0, 297, 0, 0, 0, 0, 0, - 0, 257, 0, 259, 0, 0, 0, 0, 184, 185, - 186, 187, 183, 188, 0, 173, 0, 168, 261, 262, - 263, 264, 152, 159, 177, 291, 0, 0, 272, 0, - 139, 0, 0, 0, 0, 0, 0, 91, 92, 0, - 95, 88, 38, 0, 0, 0, 30, 35, 371, 0, - 234, 0, 348, 347, 124, 0, 123, 0, 271, 305, - 0, 301, 0, 300, 0, 0, 0, 325, 326, 0, - 0, 0, 175, 165, 0, 0, 172, 0, 0, 157, - 0, 0, 306, 295, 294, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 329, 0, 332, 0, 138, + 140, 145, 146, 0, 134, 33, 0, 0, 0, 0, + 36, 38, 39, 179, 0, 35, 105, 0, 0, 103, + 124, 119, 118, 0, 0, 0, 207, 182, 0, 271, + 0, 179, 0, 0, 0, 0, 0, 301, 0, 0, + 0, 0, 0, 0, 0, 246, 245, 179, 152, 166, + 168, 177, 169, 235, 0, 157, 238, 294, 295, 0, + 0, 179, 0, 275, 285, 286, 289, 290, 0, 292, + 284, 287, 288, 280, 279, 281, 282, 283, 311, 313, + 331, 334, 0, 143, 144, 142, 148, 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, 97, 0, 0, 364, 0, 34, 0, 0, + 101, 0, 0, 0, 0, 351, 0, 347, 0, 208, + 0, 272, 0, 306, 0, 0, 299, 0, 0, 0, + 0, 0, 0, 259, 0, 261, 0, 0, 0, 0, + 186, 187, 188, 189, 185, 190, 0, 175, 0, 170, + 263, 264, 265, 266, 154, 161, 179, 293, 0, 0, + 274, 0, 141, 0, 0, 0, 0, 0, 0, 93, + 94, 0, 97, 90, 40, 0, 0, 0, 32, 37, + 373, 0, 236, 0, 350, 349, 126, 0, 125, 0, + 273, 307, 0, 303, 0, 302, 0, 0, 0, 327, + 328, 0, 0, 0, 177, 167, 0, 0, 174, 0, + 0, 159, 0, 0, 308, 297, 296, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 93, 90, 94, 89, 39, 0, 97, 0, 0, 0, - 0, 346, 207, 303, 298, 302, 289, 0, 255, 0, - 258, 260, 169, 181, 0, 0, 265, 266, 267, 268, - 0, 0, 135, 307, 296, 62, 0, 0, 0, 0, + 0, 0, 95, 92, 96, 91, 41, 0, 99, 0, + 0, 0, 0, 348, 209, 305, 300, 304, 291, 0, + 257, 0, 260, 262, 171, 183, 0, 0, 267, 268, + 269, 270, 0, 0, 137, 309, 298, 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 96, 365, 0, 363, 360, 0, 235, 0, 0, - 0, 0, 158, 156, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 98, 367, 0, 365, 362, 0, 237, + 0, 0, 0, 0, 160, 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 361, 0, 0, 369, 360, 0, 256, 182, 174, 61, - 67, 68, 65, 66, 69, 70, 71, 64, 83, 84, - 81, 82, 85, 86, 87, 80, 75, 76, 73, 74, - 77, 78, 79, 72, 366, 368, 367, 364, 0, 370, - 0, 0, 0, 359, 0, 360, 254 + 0, 0, 363, 0, 0, 371, 362, 0, 258, 184, + 176, 63, 69, 70, 67, 68, 71, 72, 73, 66, + 85, 86, 83, 84, 87, 88, 89, 82, 77, 78, + 75, 76, 79, 80, 81, 74, 368, 370, 369, 366, + 0, 372, 0, 0, 0, 361, 0, 362, 256 }; /* YYPGOTO[NTERM-NUM]. */ static const yytype_int16 yypgoto[] = { - -647, -647, -647, 819, -647, 836, -647, 450, -647, 433, - -647, 387, 391, -647, -348, 846, 849, 755, -647, -647, - 850, -647, 651, 853, 857, -57, 898, -14, 724, 778, - 2, -647, -647, 495, -647, -647, -647, -647, -647, -647, - -161, -647, -647, -647, -647, 435, -85, 22, 370, -647, - -647, 780, -647, -647, 858, 863, 864, 865, -270, -647, - 616, -168, -170, -383, -381, -365, -362, -647, -647, -647, - -647, -647, -647, 637, -647, -647, -647, 409, -647, 451, - -647, 453, -647, 722, 567, 396, -29, 272, 286, -647, - -647, -646, -647, 205, 243, -647 + -644, -644, -644, 778, -644, 785, -644, 391, -644, 390, + -644, 355, 357, -644, -350, 812, 813, 721, -644, -644, + 814, -644, 615, 819, 821, -58, 865, -15, 718, 739, + -59, -644, -644, 462, -644, -644, -644, -644, -644, -644, + -163, -644, -644, -644, -644, 428, -236, 19, 331, -644, + -644, 780, -644, -644, 858, 861, 863, 864, 866, -271, + -644, 614, -170, -172, -387, -385, -370, -368, -644, -644, + -644, -644, -644, -644, 636, -644, -644, -644, 408, -644, + 451, -644, 453, -644, 714, 571, 394, -34, 157, 239, + -644, -644, -643, -644, 203, 245, -644 }; /* YYDEFGOTO[NTERM-NUM]. */ static const yytype_int16 yydefgoto[] = { - 0, 17, 18, 19, 115, 20, 347, 348, 349, 450, - 520, 521, 522, 350, 256, 21, 22, 157, 23, 60, - 24, 166, 167, 25, 26, 27, 28, 29, 91, 143, - 92, 148, 337, 338, 423, 249, 342, 146, 311, 393, - 169, 622, 559, 89, 386, 387, 388, 389, 497, 30, - 78, 79, 390, 494, 31, 32, 33, 34, 225, 357, - 226, 227, 228, 229, 230, 231, 232, 502, 233, 234, - 235, 236, 237, 291, 238, 239, 240, 241, 546, 242, - 243, 244, 245, 246, 464, 465, 171, 102, 94, 85, - 99, 693, 526, 653, 654, 353 + 0, 17, 18, 19, 116, 20, 349, 350, 351, 452, + 522, 523, 524, 352, 258, 21, 22, 159, 23, 61, + 24, 168, 169, 25, 26, 27, 28, 29, 92, 145, + 93, 150, 339, 340, 425, 251, 344, 148, 313, 395, + 171, 624, 561, 90, 388, 389, 390, 391, 499, 30, + 79, 80, 392, 496, 31, 32, 33, 34, 35, 227, + 359, 228, 229, 230, 231, 232, 233, 234, 504, 235, + 236, 237, 238, 239, 293, 240, 241, 242, 243, 548, + 244, 245, 246, 247, 248, 466, 467, 173, 103, 95, + 86, 100, 695, 528, 655, 656, 355 }; /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM. If @@ -1126,198 +1128,198 @@ static const yytype_int16 yydefgoto[] = number is the opposite. If YYTABLE_NINF, syntax error. */ static const yytype_int16 yytable[] = { - 263, 368, 82, 122, 456, 262, -355, 46, 90, 498, - 416, 499, 251, 1, 168, 2, 3, 4, 5, 6, - 7, 8, 9, 344, 14, 397, 47, 500, 49, 10, - 501, 11, 12, 13, 286, 76, 86, 543, 87, 290, - 88, 309, 473, 354, 312, 604, 355, 472, 729, 46, - 303, 304, 173, 289, 174, 175, 308, 544, 50, 51, - 495, -358, 100, 178, 52, 179, 180, 201, 202, 203, - 109, 93, 183, 333, 184, 185, 335, 40, 334, 339, - 340, 336, 172, 71, 128, 14, 459, 61, 62, 736, - 63, 144, 359, 134, 421, 422, 468, 398, 313, 314, - 313, 314, 64, 65, 524, 48, 313, 314, 197, 529, - 313, 314, 176, 496, 286, 313, 314, 151, 152, 153, - 372, 373, 400, 181, 16, 160, 161, 162, 75, 507, - 261, 379, 186, 313, 314, 252, 313, 314, 77, 345, - 80, 346, 257, 258, 199, 395, 396, 310, 402, 403, - 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, - 414, 415, 84, 15, 208, 209, 210, 211, 313, 314, - 401, 110, 111, 424, 616, 595, 617, 254, 86, 142, - 87, 417, 88, 385, 83, 16, 90, 14, 212, 213, - 214, 93, 618, 101, 343, 619, 552, 200, 201, 202, - 203, 515, 35, 36, 37, 177, 313, 314, 476, 477, - 307, 107, 66, 67, 38, 39, 182, 68, 69, 70, - 313, 314, 375, 216, 376, 187, 377, 108, 505, 41, - 42, 43, 503, 112, 561, 370, 217, 218, 219, 317, - 223, 44, 45, 474, 479, 475, 516, 377, 517, 518, - 453, 519, 339, 454, 113, 384, -359, -359, 320, 321, - 366, 600, 469, 114, -359, 310, 188, 661, 204, 205, - 189, 190, 127, 371, 133, 191, 192, 206, 135, 207, - 724, 358, 725, 726, 317, -359, 325, 326, 327, 328, - 329, 330, 331, 136, 455, 208, 209, 210, 211, 129, - 130, 318, 319, 320, 321, 541, 509, 542, 545, 323, - 200, 201, 202, 203, 103, 104, 105, 106, 137, 212, - 213, 214, 95, 96, 97, 98, 481, 138, 485, 482, - 324, 325, 326, 327, 328, 329, 330, 331, 141, 564, - 483, 215, 528, 484, 530, 355, 216, 310, 534, 536, - 662, 535, 535, 145, 538, 470, 139, 310, 147, 217, - 218, 219, 563, 131, 132, 310, 220, 221, 222, 596, - 150, 223, 355, 224, 367, 599, 605, 317, 355, 305, - 306, 204, 205, 608, 624, 506, 609, 310, 625, 149, - 206, 626, 207, 313, 314, 289, -359, -359, 487, -189, - 488, 489, 490, 491, 154, 492, 493, 155, 208, 209, - 210, 211, 651, 657, 156, 355, 310, 200, 201, 202, - 203, 329, 330, 331, -359, -359, 327, 328, 329, 330, - 331, 158, 212, 213, 214, 566, 567, 568, 569, 570, - 159, 690, 571, 572, 691, 72, 73, 74, 200, 201, - 202, 203, 163, 663, 215, 461, 462, 463, 370, 216, - 363, 364, 573, 53, 54, 55, 56, 57, 58, 613, - 614, 59, 217, 218, 219, 698, 164, 165, 355, 220, - 221, 222, 733, 168, 223, 691, 224, 170, 204, 205, - 560, 193, 697, 14, 194, 195, 247, 206, 248, 207, - 250, 255, 259, 200, 201, 202, 203, 317, 260, 264, - 265, 266, 267, 269, 270, 208, 209, 210, 211, 204, - 205, 271, 287, 292, 318, 319, 320, 321, 206, 288, - 207, 293, 323, 294, 200, 201, 202, 203, 295, 212, - 213, 214, 296, 298, 299, 300, 208, 209, 210, 211, - 301, 302, 341, 324, 325, 326, 327, 328, 329, 330, - 331, 215, 332, 351, 606, 352, 216, 356, 361, 362, - 212, 213, 214, 365, 284, 285, 369, 378, 14, 217, - 218, 219, 380, 206, 381, 207, 220, 221, 222, 382, - 383, 223, 215, 224, 394, 418, 392, 216, 399, 419, - 420, 208, 209, 210, 211, 284, 451, 458, 467, 460, - 217, 218, 219, 452, 206, 457, 207, 220, 221, 222, - 398, 471, 223, 480, 224, 212, 213, 214, 478, 486, - 313, 504, 208, 209, 210, 211, 574, 575, 576, 577, - 578, 511, 508, 579, 580, 512, 513, 215, 514, 523, - 525, 532, 216, 533, 531, 537, 212, 213, 214, 539, - 549, 223, 550, 581, 551, 217, 218, 219, 554, 555, - 556, 558, 220, 221, 222, 557, 562, 223, 215, 224, - 565, 590, 591, 216, 597, 598, 602, 607, 615, 315, - 603, 316, 610, 611, 620, 623, 217, 218, 219, 621, - 627, 628, 652, 220, 221, 222, 629, 630, 223, 1, - 224, 2, 3, 4, 5, 6, 7, 8, 9, 655, - 631, 632, 633, 634, 635, 10, 370, 11, 12, 13, - 1, 636, 2, 3, 4, 5, 6, 7, 317, 9, - 637, 638, 658, 639, 640, 656, 10, 641, 11, 12, - 13, 642, 643, 644, 645, 318, 319, 320, 321, 322, - 646, 659, 647, 323, 648, 649, 650, 660, 664, 582, - 583, 584, 585, 586, 665, 317, 587, 588, 666, 667, - 668, 14, 669, 670, 324, 325, 326, 327, 328, 329, - 330, 331, 318, 319, 320, 321, 589, 671, 672, 673, - 323, 674, 14, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 282, 283, 675, 676, 677, 678, 679, - 680, 324, 325, 326, 327, 328, 329, 330, 331, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, - 681, 682, 446, 683, 684, 447, 448, 685, 449, 15, - 686, 687, 688, 689, 692, 694, 695, 696, 310, 699, - 700, 701, 702, 703, 730, 704, 705, 706, 707, 728, - 15, 16, 734, 708, 732, 709, 710, 711, 712, 713, - 714, 715, 716, 717, 718, 719, 116, 720, 721, 722, - 723, 735, 16, 140, 527, 540, 117, 593, 253, 118, - 119, 594, 360, 120, 81, 510, 196, 121, 123, 268, - 198, 553, 612, 124, 125, 126, 391, 374, 592, 466, - 547, 601, 548, 731, 727, 0, 297 + 265, 83, 370, 123, 458, 264, 91, 500, 418, 501, + 253, 346, 47, 175, 399, 176, 177, 180, 170, 181, + 182, 314, 311, 48, 502, 50, 503, 423, 424, 474, + 475, 146, 77, 87, 288, 88, 14, 89, 291, 292, + 546, 203, 204, 205, -360, 402, 185, 497, 186, 187, + 305, 306, 218, 731, 94, 87, 310, 88, 259, 89, + 101, 41, 36, 37, 38, 219, 220, 221, 110, 356, + 315, 316, 357, 178, 39, 40, 47, 183, 174, 341, + 342, 545, 130, 111, 112, 201, 400, 461, 315, 316, + 72, 136, 361, 403, 738, 315, 316, 470, 315, 316, + 498, 315, 316, 14, 526, 199, 188, 606, 481, 531, + 315, 316, 315, 316, 288, 153, 154, 155, 315, 316, + 374, 375, 49, 162, 163, 164, 81, 347, 312, 348, + 509, 381, 455, 254, 263, 456, 16, 76, 210, 211, + 212, 213, 315, 316, 260, 397, 398, 386, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 417, 214, 215, 216, 517, 179, 78, 315, 316, + 184, -357, 618, 426, 619, 597, 256, 144, 1, 419, + 2, 3, 4, 5, 6, 7, 8, 9, 387, 620, + 84, 621, 91, 554, 10, 345, 11, 12, 13, 189, + 104, 105, 106, 107, 42, 43, 44, 319, 478, 479, + 518, 309, 519, 520, 225, 521, 45, 46, 202, 203, + 204, 205, 62, 63, 335, 64, -361, -361, 507, 336, + 131, 132, 505, 51, 52, 563, 190, 65, 66, 53, + 191, 192, 337, 133, 134, 193, 194, 338, 85, 377, + 14, 378, 341, 379, -361, -361, 329, 330, 331, 332, + 333, 476, 602, 477, 108, 379, 1, 663, 2, 3, + 4, 5, 6, 7, 373, 9, 96, 97, 98, 99, + 360, 368, 10, 109, 11, 12, 13, 307, 308, 206, + 207, 726, 94, 727, 728, 457, 471, 102, 208, 312, + 209, 483, 137, 485, 484, 543, 486, 544, 547, 530, + 532, 536, 357, 312, 537, 113, 210, 211, 212, 213, + 615, 616, 54, 55, 56, 57, 58, 59, 15, 487, + 60, 114, 202, 203, 204, 205, 315, 316, 14, 566, + 214, 215, 216, 538, 365, 366, 537, 67, 68, 115, + 16, 664, 69, 70, 71, 129, 472, 135, 540, 138, + 140, 312, 217, 202, 203, 204, 205, 218, 139, 141, + 568, 569, 570, 571, 572, 147, 607, 573, 574, 143, + 219, 220, 221, 73, 74, 75, 508, 222, 223, 224, + 565, 149, 225, 312, 226, 369, 151, 575, 576, 577, + 578, 579, 580, 206, 207, 581, 582, 598, 601, 152, + 357, 357, 208, 156, 209, 158, 15, 291, 202, 203, + 204, 205, 157, 610, 626, 583, 611, 312, 160, 161, + 210, 211, 212, 213, 206, 207, 165, 627, 16, 14, + 628, 653, 166, 208, 357, 209, 331, 332, 333, 202, + 203, 204, 205, 665, 214, 215, 216, 659, 167, 170, + 312, 210, 211, 212, 213, 172, 584, 585, 586, 587, + 588, 195, 692, 589, 590, 693, 217, 700, 196, 735, + 357, 218, 693, 197, 249, 214, 215, 216, 250, 206, + 207, 562, 699, 591, 219, 220, 221, 252, 208, 257, + 209, 222, 223, 224, 261, 262, 225, 217, 226, 463, + 464, 465, 218, 266, 267, 268, 210, 211, 212, 213, + 286, 287, 269, 271, 272, 219, 220, 221, 273, 208, + 289, 209, 222, 223, 224, 290, 294, 225, 295, 226, + 214, 215, 216, 296, 297, 300, 301, 210, 211, 212, + 213, 202, 203, 204, 205, 298, 489, -191, 490, 491, + 492, 493, 217, 494, 495, 302, 303, 218, 304, 334, + 343, 214, 215, 216, 353, 354, 358, 363, 364, 371, + 219, 220, 221, 367, 14, 380, 382, 222, 223, 224, + 383, 384, 225, 217, 226, 394, 385, 396, 218, 274, + 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, + 285, 219, 220, 221, 420, 401, 421, 453, 222, 223, + 224, 422, 286, 225, 454, 226, 459, 460, 469, 400, + 473, 208, 462, 209, 480, 482, 372, 1, 315, 2, + 3, 4, 5, 6, 7, 8, 9, 488, 506, 210, + 211, 212, 213, 10, 527, 11, 12, 13, 510, 513, + 534, 514, 515, 516, 525, 533, 535, 539, 541, 225, + 551, 552, 553, 214, 215, 216, 556, 317, 557, 318, + 558, 559, 560, 564, 567, 319, 592, 593, 599, 604, + 600, 605, 617, 609, 612, 217, 622, 613, 629, 623, + 218, 625, 320, 321, 322, 323, 630, 631, 632, 14, + 325, 633, 634, 219, 220, 221, 635, 372, 636, 654, + 222, 223, 224, 657, 637, 225, 319, 226, 638, 639, + 640, 326, 327, 328, 329, 330, 331, 332, 333, 641, + 660, 642, 608, 320, 321, 322, 323, 324, 643, 644, + 658, 325, 645, 646, 372, 647, 648, 649, 650, 651, + 652, 661, 662, 694, 666, 667, 319, 668, 669, 670, + 671, 672, 326, 327, 328, 329, 330, 331, 332, 333, + 673, 674, 675, 320, 321, 322, 323, 15, 511, 676, + 677, 325, 678, 679, 680, 681, 682, 683, 684, 685, + 686, 687, 688, 319, 689, 690, 691, 696, 697, 16, + 312, 698, 326, 327, 328, 329, 330, 331, 332, 333, + 320, 321, 322, 323, 732, 701, 702, 703, 325, 704, + 705, 706, 707, 708, 709, 710, 736, 711, 712, 713, + 319, 714, 715, 734, 716, 717, 117, 529, 730, 326, + 327, 328, 329, 330, 331, 332, 333, -361, -361, 322, + 323, 718, 719, 142, 542, -361, 720, 721, 722, 723, + 724, 725, 737, 118, 119, 120, 255, 595, 362, 596, + 121, 82, 122, 200, 512, 614, -361, 327, 328, 329, + 330, 331, 332, 333, 427, 428, 429, 430, 431, 432, + 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, + 443, 444, 445, 446, 447, 270, 555, 448, 198, 124, + 449, 450, 125, 451, 126, 127, 393, 128, 376, 594, + 299, 603, 549, 733, 550, 468, 0, 0, 729 }; static const yytype_int16 yycheck[] = { - 168, 271, 16, 60, 352, 166, 0, 3, 8, 392, - 3, 392, 56, 7, 67, 9, 10, 11, 12, 13, - 14, 15, 16, 3, 79, 76, 4, 392, 6, 23, - 392, 25, 26, 27, 204, 13, 20, 84, 22, 207, - 24, 77, 87, 180, 56, 84, 183, 56, 694, 3, - 220, 221, 3, 88, 5, 6, 224, 87, 154, 155, - 3, 63, 40, 3, 160, 5, 6, 4, 5, 6, - 48, 73, 3, 178, 5, 6, 178, 33, 183, 247, - 248, 183, 111, 30, 62, 79, 356, 29, 30, 735, - 32, 89, 260, 71, 71, 72, 366, 148, 145, 146, - 145, 146, 44, 45, 452, 77, 145, 146, 137, 457, - 145, 146, 63, 56, 284, 145, 146, 95, 96, 97, - 288, 289, 74, 63, 179, 103, 104, 105, 3, 399, - 183, 299, 63, 145, 146, 179, 145, 146, 3, 119, - 158, 121, 34, 157, 142, 313, 314, 183, 318, 319, - 320, 321, 322, 323, 324, 325, 326, 327, 328, 329, - 330, 331, 182, 157, 101, 102, 103, 104, 145, 146, - 122, 77, 78, 341, 557, 523, 557, 155, 20, 179, - 22, 174, 24, 179, 0, 179, 8, 79, 125, 126, - 127, 73, 557, 73, 251, 557, 180, 3, 4, 5, - 6, 74, 29, 30, 31, 156, 145, 146, 376, 377, - 224, 181, 154, 155, 41, 42, 156, 159, 160, 161, - 145, 146, 84, 152, 86, 156, 88, 14, 398, 29, - 30, 31, 393, 3, 504, 74, 165, 166, 167, 123, - 177, 41, 42, 84, 183, 86, 119, 88, 121, 122, - 180, 124, 420, 183, 3, 180, 140, 141, 142, 143, - 66, 531, 180, 3, 148, 183, 42, 615, 74, 75, - 46, 47, 3, 287, 3, 51, 52, 83, 56, 85, - 3, 259, 5, 6, 123, 169, 170, 171, 172, 173, - 174, 175, 176, 183, 351, 101, 102, 103, 104, 162, - 163, 140, 141, 142, 143, 473, 145, 475, 478, 148, - 3, 4, 5, 6, 42, 43, 44, 45, 6, 125, - 126, 127, 36, 37, 38, 39, 180, 180, 385, 183, - 169, 170, 171, 172, 173, 174, 175, 176, 21, 509, - 180, 147, 180, 183, 180, 183, 152, 183, 180, 180, - 620, 183, 183, 66, 180, 369, 180, 183, 68, 165, - 166, 167, 180, 162, 163, 183, 172, 173, 174, 180, - 3, 177, 183, 179, 180, 180, 544, 123, 183, 5, - 6, 74, 75, 180, 180, 399, 183, 183, 180, 74, - 83, 183, 85, 145, 146, 88, 142, 143, 57, 58, - 59, 60, 61, 62, 3, 64, 65, 63, 101, 102, - 103, 104, 180, 180, 179, 183, 183, 3, 4, 5, - 6, 174, 175, 176, 170, 171, 172, 173, 174, 175, - 176, 75, 125, 126, 127, 90, 91, 92, 93, 94, - 3, 180, 97, 98, 183, 149, 150, 151, 3, 4, - 5, 6, 3, 621, 147, 80, 81, 82, 74, 152, - 48, 49, 117, 35, 36, 37, 38, 39, 40, 554, - 555, 43, 165, 166, 167, 180, 3, 3, 183, 172, - 173, 174, 180, 67, 177, 183, 179, 4, 74, 75, - 504, 3, 660, 79, 3, 179, 54, 83, 69, 85, - 75, 3, 63, 3, 4, 5, 6, 123, 169, 79, - 79, 3, 6, 180, 180, 101, 102, 103, 104, 74, - 75, 179, 179, 4, 140, 141, 142, 143, 83, 179, - 85, 4, 148, 4, 3, 4, 5, 6, 4, 125, - 126, 127, 6, 179, 179, 179, 101, 102, 103, 104, - 179, 179, 70, 169, 170, 171, 172, 173, 174, 175, - 176, 147, 181, 56, 180, 179, 152, 179, 179, 179, - 125, 126, 127, 180, 74, 75, 179, 4, 79, 165, - 166, 167, 4, 83, 4, 85, 172, 173, 174, 4, - 180, 177, 147, 179, 3, 6, 164, 152, 179, 5, - 183, 101, 102, 103, 104, 74, 120, 3, 6, 183, - 165, 166, 167, 179, 83, 179, 85, 172, 173, 174, - 148, 180, 177, 183, 179, 125, 126, 127, 77, 183, - 145, 179, 101, 102, 103, 104, 90, 91, 92, 93, - 94, 179, 122, 97, 98, 179, 179, 147, 179, 179, - 28, 4, 152, 3, 179, 50, 125, 126, 127, 180, - 4, 177, 4, 117, 4, 165, 166, 167, 58, 58, - 3, 53, 172, 173, 174, 183, 180, 177, 147, 179, - 6, 122, 120, 152, 179, 153, 6, 183, 179, 74, - 180, 76, 180, 180, 54, 180, 165, 166, 167, 55, - 183, 183, 3, 172, 173, 174, 183, 183, 177, 7, - 179, 9, 10, 11, 12, 13, 14, 15, 16, 3, - 183, 183, 183, 183, 183, 23, 74, 25, 26, 27, - 7, 183, 9, 10, 11, 12, 13, 14, 123, 16, - 183, 183, 4, 183, 183, 153, 23, 183, 25, 26, - 27, 183, 183, 183, 183, 140, 141, 142, 143, 144, - 183, 4, 183, 148, 183, 183, 183, 63, 6, 90, - 91, 92, 93, 94, 6, 123, 97, 98, 6, 6, - 6, 79, 6, 6, 169, 170, 171, 172, 173, 174, - 175, 176, 140, 141, 142, 143, 117, 6, 6, 6, - 148, 6, 79, 128, 129, 130, 131, 132, 133, 134, - 135, 136, 137, 138, 139, 6, 6, 6, 6, 6, - 6, 169, 170, 171, 172, 173, 174, 175, 176, 89, - 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, - 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 6, 6, 112, 6, 6, 115, 116, 6, 118, 157, - 6, 6, 6, 169, 79, 3, 183, 180, 183, 180, - 180, 180, 180, 180, 4, 180, 180, 180, 180, 179, - 157, 179, 6, 180, 183, 180, 180, 180, 180, 180, - 180, 180, 180, 180, 180, 180, 60, 180, 180, 180, - 180, 180, 179, 84, 454, 472, 60, 520, 153, 60, - 60, 520, 261, 60, 16, 420, 136, 60, 60, 195, - 142, 486, 552, 60, 60, 60, 310, 290, 519, 362, - 479, 535, 479, 728, 691, -1, 214 + 170, 16, 273, 61, 354, 168, 8, 394, 3, 394, + 56, 3, 3, 3, 76, 5, 6, 3, 67, 5, + 6, 56, 77, 4, 394, 6, 394, 71, 72, 56, + 87, 90, 13, 20, 206, 22, 79, 24, 88, 209, + 87, 4, 5, 6, 63, 74, 3, 3, 5, 6, + 222, 223, 152, 696, 73, 20, 226, 22, 34, 24, + 41, 33, 29, 30, 31, 165, 166, 167, 49, 180, + 145, 146, 183, 63, 41, 42, 3, 63, 112, 249, + 250, 84, 63, 77, 78, 144, 148, 358, 145, 146, + 30, 72, 262, 122, 737, 145, 146, 368, 145, 146, + 56, 145, 146, 79, 454, 139, 63, 84, 183, 459, + 145, 146, 145, 146, 286, 96, 97, 98, 145, 146, + 290, 291, 77, 104, 105, 106, 158, 119, 183, 121, + 401, 301, 180, 179, 183, 183, 179, 3, 101, 102, + 103, 104, 145, 146, 159, 315, 316, 180, 320, 321, + 322, 323, 324, 325, 326, 327, 328, 329, 330, 331, + 332, 333, 125, 126, 127, 74, 156, 3, 145, 146, + 156, 0, 559, 343, 559, 525, 157, 179, 7, 174, + 9, 10, 11, 12, 13, 14, 15, 16, 179, 559, + 0, 559, 8, 180, 23, 253, 25, 26, 27, 156, + 43, 44, 45, 46, 29, 30, 31, 123, 378, 379, + 119, 226, 121, 122, 177, 124, 41, 42, 3, 4, + 5, 6, 29, 30, 178, 32, 142, 143, 400, 183, + 162, 163, 395, 154, 155, 506, 42, 44, 45, 160, + 46, 47, 178, 162, 163, 51, 52, 183, 182, 84, + 79, 86, 422, 88, 170, 171, 172, 173, 174, 175, + 176, 84, 533, 86, 181, 88, 7, 617, 9, 10, + 11, 12, 13, 14, 289, 16, 37, 38, 39, 40, + 261, 66, 23, 14, 25, 26, 27, 5, 6, 74, + 75, 3, 73, 5, 6, 353, 180, 73, 83, 183, + 85, 180, 56, 180, 183, 475, 183, 477, 480, 180, + 180, 180, 183, 183, 183, 3, 101, 102, 103, 104, + 556, 557, 35, 36, 37, 38, 39, 40, 157, 387, + 43, 3, 3, 4, 5, 6, 145, 146, 79, 511, + 125, 126, 127, 180, 48, 49, 183, 154, 155, 3, + 179, 622, 159, 160, 161, 3, 371, 3, 180, 183, + 180, 183, 147, 3, 4, 5, 6, 152, 6, 180, + 90, 91, 92, 93, 94, 66, 546, 97, 98, 21, + 165, 166, 167, 149, 150, 151, 401, 172, 173, 174, + 180, 68, 177, 183, 179, 180, 74, 117, 90, 91, + 92, 93, 94, 74, 75, 97, 98, 180, 180, 3, + 183, 183, 83, 3, 85, 179, 157, 88, 3, 4, + 5, 6, 63, 180, 180, 117, 183, 183, 75, 3, + 101, 102, 103, 104, 74, 75, 3, 180, 179, 79, + 183, 180, 3, 83, 183, 85, 174, 175, 176, 3, + 4, 5, 6, 623, 125, 126, 127, 180, 3, 67, + 183, 101, 102, 103, 104, 4, 90, 91, 92, 93, + 94, 3, 180, 97, 98, 183, 147, 180, 3, 180, + 183, 152, 183, 179, 54, 125, 126, 127, 69, 74, + 75, 506, 662, 117, 165, 166, 167, 75, 83, 3, + 85, 172, 173, 174, 63, 169, 177, 147, 179, 80, + 81, 82, 152, 79, 79, 3, 101, 102, 103, 104, + 74, 75, 6, 180, 180, 165, 166, 167, 179, 83, + 179, 85, 172, 173, 174, 179, 4, 177, 4, 179, + 125, 126, 127, 4, 4, 179, 179, 101, 102, 103, + 104, 3, 4, 5, 6, 6, 57, 58, 59, 60, + 61, 62, 147, 64, 65, 179, 179, 152, 179, 181, + 70, 125, 126, 127, 56, 179, 179, 179, 179, 179, + 165, 166, 167, 180, 79, 4, 4, 172, 173, 174, + 4, 4, 177, 147, 179, 164, 180, 3, 152, 128, + 129, 130, 131, 132, 133, 134, 135, 136, 137, 138, + 139, 165, 166, 167, 6, 179, 5, 120, 172, 173, + 174, 183, 74, 177, 179, 179, 179, 3, 6, 148, + 180, 83, 183, 85, 77, 183, 74, 7, 145, 9, + 10, 11, 12, 13, 14, 15, 16, 183, 179, 101, + 102, 103, 104, 23, 28, 25, 26, 27, 122, 179, + 4, 179, 179, 179, 179, 179, 3, 50, 180, 177, + 4, 4, 4, 125, 126, 127, 58, 74, 58, 76, + 3, 183, 53, 180, 6, 123, 122, 120, 179, 6, + 153, 180, 179, 183, 180, 147, 54, 180, 183, 55, + 152, 180, 140, 141, 142, 143, 183, 183, 183, 79, + 148, 183, 183, 165, 166, 167, 183, 74, 183, 3, + 172, 173, 174, 3, 183, 177, 123, 179, 183, 183, + 183, 169, 170, 171, 172, 173, 174, 175, 176, 183, + 4, 183, 180, 140, 141, 142, 143, 144, 183, 183, + 153, 148, 183, 183, 74, 183, 183, 183, 183, 183, + 183, 4, 63, 79, 6, 6, 123, 6, 6, 6, + 6, 6, 169, 170, 171, 172, 173, 174, 175, 176, + 6, 6, 6, 140, 141, 142, 143, 157, 145, 6, + 6, 148, 6, 6, 6, 6, 6, 6, 6, 6, + 6, 6, 6, 123, 6, 6, 169, 3, 183, 179, + 183, 180, 169, 170, 171, 172, 173, 174, 175, 176, + 140, 141, 142, 143, 4, 180, 180, 180, 148, 180, + 180, 180, 180, 180, 180, 180, 6, 180, 180, 180, + 123, 180, 180, 183, 180, 180, 61, 456, 179, 169, + 170, 171, 172, 173, 174, 175, 176, 140, 141, 142, + 143, 180, 180, 85, 474, 148, 180, 180, 180, 180, + 180, 180, 180, 61, 61, 61, 155, 522, 263, 522, + 61, 16, 61, 144, 422, 554, 169, 170, 171, 172, + 173, 174, 175, 176, 89, 90, 91, 92, 93, 94, + 95, 96, 97, 98, 99, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 197, 488, 112, 138, 61, + 115, 116, 61, 118, 61, 61, 312, 61, 292, 521, + 216, 537, 481, 730, 481, 364, -1, -1, 693 }; /* YYSTOS[STATE-NUM] -- The symbol kind of the accessing symbol of @@ -1327,120 +1329,120 @@ static const yytype_int16 yystos[] = 0, 7, 9, 10, 11, 12, 13, 14, 15, 16, 23, 25, 26, 27, 79, 157, 179, 185, 186, 187, 189, 199, 200, 202, 204, 207, 208, 209, 210, 211, - 233, 238, 239, 240, 241, 29, 30, 31, 41, 42, - 33, 29, 30, 31, 41, 42, 3, 231, 77, 231, - 154, 155, 160, 35, 36, 37, 38, 39, 40, 43, - 203, 29, 30, 32, 44, 45, 154, 155, 159, 160, - 161, 30, 149, 150, 151, 3, 231, 3, 234, 235, - 158, 210, 211, 0, 182, 273, 20, 22, 24, 227, - 8, 212, 214, 73, 272, 272, 272, 272, 272, 274, - 231, 73, 271, 271, 271, 271, 271, 181, 14, 231, - 77, 78, 3, 3, 3, 188, 189, 199, 200, 204, - 207, 208, 209, 238, 239, 240, 241, 3, 231, 162, - 163, 162, 163, 3, 231, 56, 183, 6, 180, 180, - 187, 21, 179, 213, 214, 66, 221, 68, 215, 74, - 3, 231, 231, 231, 3, 63, 179, 201, 75, 3, - 231, 231, 231, 3, 3, 3, 205, 206, 67, 224, - 4, 270, 270, 3, 5, 6, 63, 156, 3, 5, - 6, 63, 156, 3, 5, 6, 63, 156, 42, 46, - 47, 51, 52, 3, 3, 179, 235, 270, 213, 214, - 3, 4, 5, 6, 74, 75, 83, 85, 101, 102, - 103, 104, 125, 126, 127, 147, 152, 165, 166, 167, - 172, 173, 174, 177, 179, 242, 244, 245, 246, 247, - 248, 249, 250, 252, 253, 254, 255, 256, 258, 259, - 260, 261, 263, 264, 265, 266, 267, 54, 69, 219, - 75, 56, 179, 201, 231, 3, 198, 34, 211, 63, - 169, 183, 224, 245, 79, 79, 3, 6, 212, 180, - 180, 179, 128, 129, 130, 131, 132, 133, 134, 135, - 136, 137, 138, 139, 74, 75, 246, 179, 179, 88, - 245, 257, 4, 4, 4, 4, 6, 267, 179, 179, - 179, 179, 179, 246, 246, 5, 6, 211, 245, 77, - 183, 222, 56, 145, 146, 74, 76, 123, 140, 141, - 142, 143, 144, 148, 169, 170, 171, 172, 173, 174, - 175, 176, 181, 178, 183, 178, 183, 216, 217, 245, - 245, 70, 220, 209, 3, 119, 121, 190, 191, 192, - 197, 56, 179, 279, 180, 183, 179, 243, 231, 245, - 206, 179, 179, 48, 49, 180, 66, 180, 242, 179, - 74, 211, 245, 245, 257, 84, 86, 88, 4, 245, - 4, 4, 4, 180, 180, 179, 228, 229, 230, 231, - 236, 244, 164, 223, 3, 245, 245, 76, 148, 179, - 74, 122, 246, 246, 246, 246, 246, 246, 246, 246, - 246, 246, 246, 246, 246, 246, 3, 174, 6, 5, - 183, 71, 72, 218, 245, 89, 90, 91, 92, 93, - 94, 95, 96, 97, 98, 99, 100, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 112, 115, 116, 118, - 193, 120, 179, 180, 183, 209, 198, 179, 3, 242, - 183, 80, 81, 82, 268, 269, 268, 6, 242, 180, - 211, 180, 56, 87, 84, 86, 245, 245, 77, 183, - 183, 180, 183, 180, 183, 209, 183, 57, 59, 60, - 61, 62, 64, 65, 237, 3, 56, 232, 247, 248, - 249, 250, 251, 224, 179, 246, 211, 242, 122, 145, - 217, 179, 179, 179, 179, 74, 119, 121, 122, 124, - 194, 195, 196, 179, 198, 28, 276, 191, 180, 198, - 180, 179, 4, 3, 180, 183, 180, 50, 180, 180, - 193, 245, 245, 84, 87, 246, 262, 263, 265, 4, - 4, 4, 180, 229, 58, 58, 3, 183, 53, 226, - 211, 242, 180, 180, 246, 6, 90, 91, 92, 93, + 233, 238, 239, 240, 241, 242, 29, 30, 31, 41, + 42, 33, 29, 30, 31, 41, 42, 3, 231, 77, + 231, 154, 155, 160, 35, 36, 37, 38, 39, 40, + 43, 203, 29, 30, 32, 44, 45, 154, 155, 159, + 160, 161, 30, 149, 150, 151, 3, 231, 3, 234, + 235, 158, 210, 211, 0, 182, 274, 20, 22, 24, + 227, 8, 212, 214, 73, 273, 273, 273, 273, 273, + 275, 231, 73, 272, 272, 272, 272, 272, 181, 14, + 231, 77, 78, 3, 3, 3, 188, 189, 199, 200, + 204, 207, 208, 209, 238, 239, 240, 241, 242, 3, + 231, 162, 163, 162, 163, 3, 231, 56, 183, 6, + 180, 180, 187, 21, 179, 213, 214, 66, 221, 68, + 215, 74, 3, 231, 231, 231, 3, 63, 179, 201, + 75, 3, 231, 231, 231, 3, 3, 3, 205, 206, + 67, 224, 4, 271, 271, 3, 5, 6, 63, 156, + 3, 5, 6, 63, 156, 3, 5, 6, 63, 156, + 42, 46, 47, 51, 52, 3, 3, 179, 235, 271, + 213, 214, 3, 4, 5, 6, 74, 75, 83, 85, + 101, 102, 103, 104, 125, 126, 127, 147, 152, 165, + 166, 167, 172, 173, 174, 177, 179, 243, 245, 246, + 247, 248, 249, 250, 251, 253, 254, 255, 256, 257, + 259, 260, 261, 262, 264, 265, 266, 267, 268, 54, + 69, 219, 75, 56, 179, 201, 231, 3, 198, 34, + 211, 63, 169, 183, 224, 246, 79, 79, 3, 6, + 212, 180, 180, 179, 128, 129, 130, 131, 132, 133, + 134, 135, 136, 137, 138, 139, 74, 75, 247, 179, + 179, 88, 246, 258, 4, 4, 4, 4, 6, 268, + 179, 179, 179, 179, 179, 247, 247, 5, 6, 211, + 246, 77, 183, 222, 56, 145, 146, 74, 76, 123, + 140, 141, 142, 143, 144, 148, 169, 170, 171, 172, + 173, 174, 175, 176, 181, 178, 183, 178, 183, 216, + 217, 246, 246, 70, 220, 209, 3, 119, 121, 190, + 191, 192, 197, 56, 179, 280, 180, 183, 179, 244, + 231, 246, 206, 179, 179, 48, 49, 180, 66, 180, + 243, 179, 74, 211, 246, 246, 258, 84, 86, 88, + 4, 246, 4, 4, 4, 180, 180, 179, 228, 229, + 230, 231, 236, 245, 164, 223, 3, 246, 246, 76, + 148, 179, 74, 122, 247, 247, 247, 247, 247, 247, + 247, 247, 247, 247, 247, 247, 247, 247, 3, 174, + 6, 5, 183, 71, 72, 218, 246, 89, 90, 91, + 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 112, 115, + 116, 118, 193, 120, 179, 180, 183, 209, 198, 179, + 3, 243, 183, 80, 81, 82, 269, 270, 269, 6, + 243, 180, 211, 180, 56, 87, 84, 86, 246, 246, + 77, 183, 183, 180, 183, 180, 183, 209, 183, 57, + 59, 60, 61, 62, 64, 65, 237, 3, 56, 232, + 248, 249, 250, 251, 252, 224, 179, 247, 211, 243, + 122, 145, 217, 179, 179, 179, 179, 74, 119, 121, + 122, 124, 194, 195, 196, 179, 198, 28, 277, 191, + 180, 198, 180, 179, 4, 3, 180, 183, 180, 50, + 180, 180, 193, 246, 246, 84, 87, 247, 263, 264, + 266, 4, 4, 4, 180, 229, 58, 58, 3, 183, + 53, 226, 211, 243, 180, 180, 247, 6, 90, 91, + 92, 93, 94, 97, 98, 117, 90, 91, 92, 93, 94, 97, 98, 117, 90, 91, 92, 93, 94, 97, - 98, 117, 90, 91, 92, 93, 94, 97, 98, 117, - 122, 120, 261, 195, 196, 198, 180, 179, 153, 180, - 242, 269, 6, 180, 84, 245, 180, 183, 180, 183, - 180, 180, 232, 230, 230, 179, 247, 248, 249, 250, - 54, 55, 225, 180, 180, 180, 183, 183, 183, 183, + 98, 117, 122, 120, 262, 195, 196, 198, 180, 179, + 153, 180, 243, 270, 6, 180, 84, 246, 180, 183, + 180, 183, 180, 180, 232, 230, 230, 179, 248, 249, + 250, 251, 54, 55, 225, 180, 180, 180, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, - 183, 180, 3, 277, 278, 3, 153, 180, 4, 4, - 63, 198, 242, 245, 6, 6, 6, 6, 6, 6, + 183, 183, 183, 180, 3, 278, 279, 3, 153, 180, + 4, 4, 63, 198, 243, 246, 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, 169, - 180, 183, 79, 275, 3, 183, 180, 245, 180, 180, + 6, 169, 180, 183, 79, 276, 3, 183, 180, 246, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, - 180, 180, 180, 180, 3, 5, 6, 278, 179, 275, - 4, 277, 183, 180, 6, 180, 275 + 180, 180, 180, 180, 180, 180, 3, 5, 6, 279, + 179, 276, 4, 278, 183, 180, 6, 180, 276 }; /* YYR1[RULE-NUM] -- Symbol kind of the left-hand side of rule RULE-NUM. */ static const yytype_int16 yyr1[] = { 0, 184, 185, 186, 186, 187, 187, 187, 187, 187, - 187, 187, 187, 187, 187, 187, 187, 188, 188, 188, - 188, 188, 188, 188, 188, 188, 188, 188, 189, 189, - 189, 189, 189, 189, 190, 190, 191, 191, 192, 192, + 187, 187, 187, 187, 187, 187, 187, 187, 188, 188, + 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, + 189, 189, 189, 189, 189, 189, 190, 190, 191, 191, + 192, 192, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, - 193, 193, 193, 193, 193, 193, 193, 193, 194, 194, - 195, 195, 195, 195, 196, 196, 197, 197, 198, 198, - 199, 200, 200, 201, 201, 202, 203, 203, 203, 203, - 203, 203, 203, 203, 204, 205, 205, 206, 207, 207, - 207, 207, 207, 208, 208, 209, 209, 209, 209, 210, - 210, 211, 212, 213, 213, 214, 215, 215, 216, 216, - 217, 218, 218, 218, 219, 219, 220, 220, 221, 221, - 222, 222, 223, 223, 224, 224, 225, 225, 226, 226, - 227, 227, 227, 227, 228, 228, 229, 229, 230, 230, - 231, 231, 232, 232, 232, 232, 233, 233, 234, 234, - 235, 236, 236, 237, 237, 237, 237, 237, 237, 237, + 194, 194, 195, 195, 195, 195, 196, 196, 197, 197, + 198, 198, 199, 200, 200, 201, 201, 202, 203, 203, + 203, 203, 203, 203, 203, 203, 204, 205, 205, 206, + 207, 207, 207, 207, 207, 208, 208, 209, 209, 209, + 209, 210, 210, 211, 212, 213, 213, 214, 215, 215, + 216, 216, 217, 218, 218, 218, 219, 219, 220, 220, + 221, 221, 222, 222, 223, 223, 224, 224, 225, 225, + 226, 226, 227, 227, 227, 227, 228, 228, 229, 229, + 230, 230, 231, 231, 232, 232, 232, 232, 233, 233, + 234, 234, 235, 236, 236, 237, 237, 237, 237, 237, + 237, 237, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, - 238, 238, 238, 238, 238, 238, 238, 238, 238, 238, - 239, 239, 239, 240, 241, 241, 241, 241, 241, 241, + 238, 238, 239, 239, 239, 240, 241, 241, 241, 241, 241, 241, 241, 241, 241, 241, 241, 241, 241, 241, - 241, 241, 242, 242, 243, 243, 244, 244, 245, 245, - 245, 245, 245, 246, 246, 246, 246, 246, 246, 246, - 246, 246, 246, 246, 247, 248, 248, 249, 249, 250, - 250, 251, 251, 251, 251, 251, 251, 251, 251, 252, - 252, 252, 252, 252, 252, 252, 252, 252, 252, 252, - 252, 252, 252, 252, 252, 252, 252, 252, 252, 252, - 252, 252, 253, 253, 254, 255, 255, 256, 256, 256, - 256, 257, 257, 258, 259, 259, 259, 259, 260, 260, - 260, 260, 261, 261, 261, 261, 261, 261, 261, 261, - 261, 261, 261, 261, 261, 262, 262, 263, 264, 264, - 265, 266, 266, 267, 267, 267, 267, 267, 267, 267, - 267, 267, 267, 267, 267, 268, 268, 269, 269, 269, - 270, 271, 271, 272, 272, 273, 273, 274, 274, 275, - 275, 276, 276, 277, 277, 278, 278, 278, 278, 279, - 279, 279 + 241, 241, 241, 242, 243, 243, 244, 244, 245, 245, + 246, 246, 246, 246, 246, 247, 247, 247, 247, 247, + 247, 247, 247, 247, 247, 247, 248, 249, 249, 250, + 250, 251, 251, 252, 252, 252, 252, 252, 252, 252, + 252, 253, 253, 253, 253, 253, 253, 253, 253, 253, + 253, 253, 253, 253, 253, 253, 253, 253, 253, 253, + 253, 253, 253, 253, 254, 254, 255, 256, 256, 257, + 257, 257, 257, 258, 258, 259, 260, 260, 260, 260, + 261, 261, 261, 261, 262, 262, 262, 262, 262, 262, + 262, 262, 262, 262, 262, 262, 262, 263, 263, 264, + 265, 265, 266, 267, 267, 268, 268, 268, 268, 268, + 268, 268, 268, 268, 268, 268, 268, 269, 269, 270, + 270, 270, 271, 272, 272, 273, 273, 274, 274, 275, + 275, 276, 276, 277, 277, 278, 278, 279, 279, 279, + 279, 280, 280, 280 }; /* YYR2[RULE-NUM] -- Number of symbols on the right-hand side of rule RULE-NUM. */ @@ -1448,42 +1450,42 @@ static const yytype_int8 yyr2[] = { 0, 2, 2, 1, 3, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 4, 4, - 8, 6, 7, 6, 1, 3, 1, 1, 3, 4, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 4, 4, 8, 6, 7, 6, 1, 3, 1, 1, + 3, 4, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 6, 4, 1, 6, 6, 6, 6, 6, 6, + 1, 1, 1, 6, 4, 1, 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, 1, 2, - 2, 1, 1, 2, 2, 0, 5, 4, 1, 3, - 4, 6, 5, 3, 0, 3, 1, 1, 1, 1, - 1, 1, 1, 0, 5, 1, 3, 3, 4, 4, - 4, 4, 6, 8, 8, 1, 1, 3, 3, 3, - 3, 2, 4, 3, 3, 8, 3, 0, 1, 3, - 2, 1, 1, 0, 2, 0, 2, 0, 1, 0, - 2, 0, 2, 0, 2, 0, 2, 0, 3, 0, - 1, 2, 1, 1, 1, 3, 1, 1, 2, 4, - 1, 3, 2, 1, 5, 0, 2, 0, 1, 3, - 5, 4, 6, 1, 1, 1, 1, 1, 1, 0, - 2, 2, 2, 2, 3, 2, 3, 3, 4, 4, - 3, 3, 4, 4, 5, 6, 7, 9, 4, 5, - 2, 2, 2, 2, 2, 4, 4, 4, 4, 4, + 1, 2, 2, 1, 1, 2, 2, 0, 5, 4, + 1, 3, 4, 6, 5, 3, 0, 3, 1, 1, + 1, 1, 1, 1, 1, 0, 5, 1, 3, 3, + 4, 4, 4, 4, 6, 8, 8, 1, 1, 3, + 3, 3, 3, 2, 4, 3, 3, 8, 3, 0, + 1, 3, 2, 1, 1, 0, 2, 0, 2, 0, + 1, 0, 2, 0, 2, 0, 2, 0, 2, 0, + 3, 0, 1, 2, 1, 1, 1, 3, 1, 1, + 2, 4, 1, 3, 2, 1, 5, 0, 2, 0, + 1, 3, 5, 4, 6, 1, 1, 1, 1, 1, + 1, 0, 2, 2, 2, 2, 3, 2, 3, 3, + 4, 4, 3, 3, 4, 4, 5, 6, 7, 9, + 4, 5, 2, 2, 2, 2, 2, 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, 13, 6, 8, 4, 6, 4, - 6, 1, 1, 1, 1, 3, 3, 3, 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, 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 + 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, 13, 6, 8, 4, + 6, 4, 6, 1, 1, 1, 1, 3, 3, 3, + 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, 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 }; @@ -2047,23 +2049,23 @@ yydestruct (const char *yymsg, switch (yykind) { case YYSYMBOL_IDENTIFIER: /* IDENTIFIER */ -#line 305 "parser.y" +#line 307 "parser.y" { free(((*yyvaluep).str_value)); } -#line 2055 "parser.cpp" +#line 2057 "parser.cpp" break; case YYSYMBOL_STRING: /* STRING */ -#line 305 "parser.y" +#line 307 "parser.y" { free(((*yyvaluep).str_value)); } -#line 2063 "parser.cpp" +#line 2065 "parser.cpp" break; case YYSYMBOL_statement_list: /* statement_list */ -#line 219 "parser.y" +#line 221 "parser.y" { fprintf(stderr, "destroy statement array\n"); if ((((*yyvaluep).stmt_array)) != nullptr) { @@ -2073,11 +2075,11 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).stmt_array)); } } -#line 2077 "parser.cpp" +#line 2079 "parser.cpp" break; case YYSYMBOL_table_element_array: /* table_element_array */ -#line 209 "parser.y" +#line 211 "parser.y" { fprintf(stderr, "destroy table element array\n"); if ((((*yyvaluep).table_element_array_t)) != nullptr) { @@ -2087,48 +2089,48 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).table_element_array_t)); } } -#line 2091 "parser.cpp" +#line 2093 "parser.cpp" break; case YYSYMBOL_column_constraints: /* column_constraints */ -#line 298 "parser.y" +#line 300 "parser.y" { fprintf(stderr, "destroy constraints\n"); if ((((*yyvaluep).column_constraints_t)) != nullptr) { delete (((*yyvaluep).column_constraints_t)); } } -#line 2102 "parser.cpp" +#line 2104 "parser.cpp" break; case YYSYMBOL_default_expr: /* default_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2110 "parser.cpp" +#line 2112 "parser.cpp" break; case YYSYMBOL_identifier_array: /* identifier_array */ -#line 309 "parser.y" +#line 311 "parser.y" { fprintf(stderr, "destroy identifier array\n"); delete (((*yyvaluep).identifier_array_t)); } -#line 2119 "parser.cpp" +#line 2121 "parser.cpp" break; case YYSYMBOL_optional_identifier_array: /* optional_identifier_array */ -#line 309 "parser.y" +#line 311 "parser.y" { fprintf(stderr, "destroy identifier array\n"); delete (((*yyvaluep).identifier_array_t)); } -#line 2128 "parser.cpp" +#line 2130 "parser.cpp" break; case YYSYMBOL_update_expr_array: /* update_expr_array */ -#line 269 "parser.y" +#line 271 "parser.y" { fprintf(stderr, "destroy update expr array\n"); if ((((*yyvaluep).update_expr_array_t)) != nullptr) { @@ -2138,82 +2140,82 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).update_expr_array_t)); } } -#line 2142 "parser.cpp" +#line 2144 "parser.cpp" break; case YYSYMBOL_update_expr: /* update_expr */ -#line 262 "parser.y" +#line 264 "parser.y" { fprintf(stderr, "destroy update expr\n"); if(((*yyvaluep).update_expr_t) != nullptr) { delete ((*yyvaluep).update_expr_t); } } -#line 2153 "parser.cpp" +#line 2155 "parser.cpp" break; case YYSYMBOL_select_statement: /* select_statement */ -#line 344 "parser.y" +#line 346 "parser.y" { if(((*yyvaluep).select_stmt) != nullptr) { delete ((*yyvaluep).select_stmt); } } -#line 2163 "parser.cpp" +#line 2165 "parser.cpp" break; case YYSYMBOL_select_with_paren: /* select_with_paren */ -#line 344 "parser.y" +#line 346 "parser.y" { if(((*yyvaluep).select_stmt) != nullptr) { delete ((*yyvaluep).select_stmt); } } -#line 2173 "parser.cpp" +#line 2175 "parser.cpp" break; case YYSYMBOL_select_without_paren: /* select_without_paren */ -#line 344 "parser.y" +#line 346 "parser.y" { if(((*yyvaluep).select_stmt) != nullptr) { delete ((*yyvaluep).select_stmt); } } -#line 2183 "parser.cpp" +#line 2185 "parser.cpp" break; case YYSYMBOL_select_clause_with_modifier: /* select_clause_with_modifier */ -#line 344 "parser.y" +#line 346 "parser.y" { if(((*yyvaluep).select_stmt) != nullptr) { delete ((*yyvaluep).select_stmt); } } -#line 2193 "parser.cpp" +#line 2195 "parser.cpp" break; case YYSYMBOL_select_clause_without_modifier_paren: /* select_clause_without_modifier_paren */ -#line 344 "parser.y" +#line 346 "parser.y" { if(((*yyvaluep).select_stmt) != nullptr) { delete ((*yyvaluep).select_stmt); } } -#line 2203 "parser.cpp" +#line 2205 "parser.cpp" break; case YYSYMBOL_select_clause_without_modifier: /* select_clause_without_modifier */ -#line 344 "parser.y" +#line 346 "parser.y" { if(((*yyvaluep).select_stmt) != nullptr) { delete ((*yyvaluep).select_stmt); } } -#line 2213 "parser.cpp" +#line 2215 "parser.cpp" break; case YYSYMBOL_order_by_clause: /* order_by_clause */ -#line 252 "parser.y" +#line 254 "parser.y" { fprintf(stderr, "destroy order by expr list\n"); if ((((*yyvaluep).order_by_expr_list_t)) != nullptr) { @@ -2223,11 +2225,11 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).order_by_expr_list_t)); } } -#line 2227 "parser.cpp" +#line 2229 "parser.cpp" break; case YYSYMBOL_order_by_expr_list: /* order_by_expr_list */ -#line 252 "parser.y" +#line 254 "parser.y" { fprintf(stderr, "destroy order by expr list\n"); if ((((*yyvaluep).order_by_expr_list_t)) != nullptr) { @@ -2237,70 +2239,70 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).order_by_expr_list_t)); } } -#line 2241 "parser.cpp" +#line 2243 "parser.cpp" break; case YYSYMBOL_order_by_expr: /* order_by_expr */ -#line 332 "parser.y" +#line 334 "parser.y" { fprintf(stderr, "destroy order by expr\n"); delete ((*yyvaluep).order_by_expr_t)->expr_; delete ((*yyvaluep).order_by_expr_t); } -#line 2251 "parser.cpp" +#line 2253 "parser.cpp" break; case YYSYMBOL_limit_expr: /* limit_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2259 "parser.cpp" +#line 2261 "parser.cpp" break; case YYSYMBOL_offset_expr: /* offset_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2267 "parser.cpp" +#line 2269 "parser.cpp" break; case YYSYMBOL_from_clause: /* from_clause */ -#line 327 "parser.y" +#line 329 "parser.y" { fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2276 "parser.cpp" +#line 2278 "parser.cpp" break; case YYSYMBOL_search_clause: /* search_clause */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2284 "parser.cpp" +#line 2286 "parser.cpp" break; case YYSYMBOL_where_clause: /* where_clause */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2292 "parser.cpp" +#line 2294 "parser.cpp" break; case YYSYMBOL_having_clause: /* having_clause */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2300 "parser.cpp" +#line 2302 "parser.cpp" break; case YYSYMBOL_group_by_clause: /* group_by_clause */ -#line 229 "parser.y" +#line 231 "parser.y" { fprintf(stderr, "destroy expression array\n"); if ((((*yyvaluep).expr_array_t)) != nullptr) { @@ -2310,38 +2312,38 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_t)); } } -#line 2314 "parser.cpp" +#line 2316 "parser.cpp" break; case YYSYMBOL_table_reference: /* table_reference */ -#line 327 "parser.y" +#line 329 "parser.y" { fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2323 "parser.cpp" +#line 2325 "parser.cpp" break; case YYSYMBOL_table_reference_unit: /* table_reference_unit */ -#line 327 "parser.y" +#line 329 "parser.y" { fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2332 "parser.cpp" +#line 2334 "parser.cpp" break; case YYSYMBOL_table_reference_name: /* table_reference_name */ -#line 327 "parser.y" +#line 329 "parser.y" { fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2341 "parser.cpp" +#line 2343 "parser.cpp" break; case YYSYMBOL_table_name: /* table_name */ -#line 289 "parser.y" +#line 291 "parser.y" { fprintf(stderr, "destroy table table_name\n"); if ((((*yyvaluep).table_name_t)) != nullptr) { @@ -2350,20 +2352,20 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).table_name_t)); } } -#line 2354 "parser.cpp" +#line 2356 "parser.cpp" break; case YYSYMBOL_table_alias: /* table_alias */ -#line 322 "parser.y" +#line 324 "parser.y" { fprintf(stderr, "destroy table alias\n"); delete (((*yyvaluep).table_alias_t)); } -#line 2363 "parser.cpp" +#line 2365 "parser.cpp" break; case YYSYMBOL_with_clause: /* with_clause */ -#line 279 "parser.y" +#line 281 "parser.y" { fprintf(stderr, "destroy with expr list\n"); if ((((*yyvaluep).with_expr_list_t)) != nullptr) { @@ -2373,11 +2375,11 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_expr_list_t)); } } -#line 2377 "parser.cpp" +#line 2379 "parser.cpp" break; case YYSYMBOL_with_expr_list: /* with_expr_list */ -#line 279 "parser.y" +#line 281 "parser.y" { fprintf(stderr, "destroy with expr list\n"); if ((((*yyvaluep).with_expr_list_t)) != nullptr) { @@ -2387,30 +2389,30 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_expr_list_t)); } } -#line 2391 "parser.cpp" +#line 2393 "parser.cpp" break; case YYSYMBOL_with_expr: /* with_expr */ -#line 338 "parser.y" +#line 340 "parser.y" { fprintf(stderr, "destroy with expr\n"); delete ((*yyvaluep).with_expr_t)->select_; delete ((*yyvaluep).with_expr_t); } -#line 2401 "parser.cpp" +#line 2403 "parser.cpp" break; case YYSYMBOL_join_clause: /* join_clause */ -#line 327 "parser.y" +#line 329 "parser.y" { fprintf(stderr, "destroy table reference\n"); delete (((*yyvaluep).table_reference_t)); } -#line 2410 "parser.cpp" +#line 2412 "parser.cpp" break; case YYSYMBOL_expr_array: /* expr_array */ -#line 229 "parser.y" +#line 231 "parser.y" { fprintf(stderr, "destroy expression array\n"); if ((((*yyvaluep).expr_array_t)) != nullptr) { @@ -2420,11 +2422,11 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_t)); } } -#line 2424 "parser.cpp" +#line 2426 "parser.cpp" break; case YYSYMBOL_expr_array_list: /* expr_array_list */ -#line 239 "parser.y" +#line 241 "parser.y" { fprintf(stderr, "destroy expression array list\n"); if ((((*yyvaluep).expr_array_list_t)) != nullptr) { @@ -2437,67 +2439,67 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_list_t)); } } -#line 2441 "parser.cpp" +#line 2443 "parser.cpp" break; case YYSYMBOL_expr_alias: /* expr_alias */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2449 "parser.cpp" +#line 2451 "parser.cpp" break; case YYSYMBOL_expr: /* expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2457 "parser.cpp" +#line 2459 "parser.cpp" break; case YYSYMBOL_operand: /* operand */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2465 "parser.cpp" +#line 2467 "parser.cpp" break; case YYSYMBOL_knn_expr: /* knn_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2473 "parser.cpp" +#line 2475 "parser.cpp" break; case YYSYMBOL_match_expr: /* match_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2481 "parser.cpp" +#line 2483 "parser.cpp" break; case YYSYMBOL_query_expr: /* query_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2489 "parser.cpp" +#line 2491 "parser.cpp" break; case YYSYMBOL_fusion_expr: /* fusion_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2497 "parser.cpp" +#line 2499 "parser.cpp" break; case YYSYMBOL_sub_search_array: /* sub_search_array */ -#line 229 "parser.y" +#line 231 "parser.y" { fprintf(stderr, "destroy expression array\n"); if ((((*yyvaluep).expr_array_t)) != nullptr) { @@ -2507,51 +2509,51 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).expr_array_t)); } } -#line 2511 "parser.cpp" +#line 2513 "parser.cpp" break; case YYSYMBOL_function_expr: /* function_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2519 "parser.cpp" +#line 2521 "parser.cpp" break; case YYSYMBOL_conjunction_expr: /* conjunction_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2527 "parser.cpp" +#line 2529 "parser.cpp" break; case YYSYMBOL_between_expr: /* between_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2535 "parser.cpp" +#line 2537 "parser.cpp" break; case YYSYMBOL_in_expr: /* in_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2543 "parser.cpp" +#line 2545 "parser.cpp" break; case YYSYMBOL_case_expr: /* case_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2551 "parser.cpp" +#line 2553 "parser.cpp" break; case YYSYMBOL_case_check_array: /* case_check_array */ -#line 350 "parser.y" +#line 352 "parser.y" { fprintf(stderr, "destroy case check array\n"); if(((*yyvaluep).case_check_array_t) != nullptr) { @@ -2560,110 +2562,110 @@ yydestruct (const char *yymsg, } } } -#line 2564 "parser.cpp" +#line 2566 "parser.cpp" break; case YYSYMBOL_cast_expr: /* cast_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2572 "parser.cpp" +#line 2574 "parser.cpp" break; case YYSYMBOL_subquery_expr: /* subquery_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2580 "parser.cpp" +#line 2582 "parser.cpp" break; case YYSYMBOL_column_expr: /* column_expr */ -#line 314 "parser.y" +#line 316 "parser.y" { delete (((*yyvaluep).expr_t)); } -#line 2588 "parser.cpp" +#line 2590 "parser.cpp" break; case YYSYMBOL_constant_expr: /* constant_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2596 "parser.cpp" +#line 2598 "parser.cpp" break; case YYSYMBOL_array_expr: /* array_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2604 "parser.cpp" +#line 2606 "parser.cpp" break; case YYSYMBOL_long_array_expr: /* long_array_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2612 "parser.cpp" +#line 2614 "parser.cpp" break; case YYSYMBOL_unclosed_long_array_expr: /* unclosed_long_array_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2620 "parser.cpp" +#line 2622 "parser.cpp" break; case YYSYMBOL_double_array_expr: /* double_array_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2628 "parser.cpp" +#line 2630 "parser.cpp" break; case YYSYMBOL_unclosed_double_array_expr: /* unclosed_double_array_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2636 "parser.cpp" +#line 2638 "parser.cpp" break; case YYSYMBOL_interval_expr: /* interval_expr */ -#line 318 "parser.y" +#line 320 "parser.y" { delete (((*yyvaluep).const_expr_t)); } -#line 2644 "parser.cpp" +#line 2646 "parser.cpp" break; case YYSYMBOL_file_path: /* file_path */ -#line 305 "parser.y" +#line 307 "parser.y" { free(((*yyvaluep).str_value)); } -#line 2652 "parser.cpp" +#line 2654 "parser.cpp" break; case YYSYMBOL_if_not_exists_info: /* if_not_exists_info */ -#line 202 "parser.y" +#line 204 "parser.y" { fprintf(stderr, "destroy if not exists info\n"); if ((((*yyvaluep).if_not_exists_info_t)) != nullptr) { delete (((*yyvaluep).if_not_exists_info_t)); } } -#line 2663 "parser.cpp" +#line 2665 "parser.cpp" break; case YYSYMBOL_with_index_param_list: /* with_index_param_list */ -#line 185 "parser.y" +#line 187 "parser.y" { fprintf(stderr, "destroy create index param list\n"); if ((((*yyvaluep).with_index_param_list_t)) != nullptr) { @@ -2673,11 +2675,11 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_index_param_list_t)); } } -#line 2677 "parser.cpp" +#line 2679 "parser.cpp" break; case YYSYMBOL_optional_table_properties_list: /* optional_table_properties_list */ -#line 185 "parser.y" +#line 187 "parser.y" { fprintf(stderr, "destroy create index param list\n"); if ((((*yyvaluep).with_index_param_list_t)) != nullptr) { @@ -2687,11 +2689,11 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).with_index_param_list_t)); } } -#line 2691 "parser.cpp" +#line 2693 "parser.cpp" break; case YYSYMBOL_index_info_list: /* index_info_list */ -#line 175 "parser.y" +#line 177 "parser.y" { fprintf(stderr, "destroy index info list\n"); if ((((*yyvaluep).index_info_list_t)) != nullptr) { @@ -2701,7 +2703,7 @@ yydestruct (const char *yymsg, delete (((*yyvaluep).index_info_list_t)); } } -#line 2705 "parser.cpp" +#line 2707 "parser.cpp" break; default: @@ -2798,7 +2800,7 @@ YYLTYPE yylloc = yyloc_default; /* User initialization code. */ -#line 85 "parser.y" +#line 86 "parser.y" { // Initialize yylloc.first_column = 0; @@ -2809,7 +2811,7 @@ YYLTYPE yylloc = yyloc_default; yylloc.string_length = 0; } -#line 2813 "parser.cpp" +#line 2815 "parser.cpp" yylsp[0] = yylloc; goto yysetstate; @@ -3020,175 +3022,187 @@ YYLTYPE yylloc = yyloc_default; switch (yyn) { case 2: /* input_pattern: statement_list semicolon */ -#line 473 "parser.y" +#line 476 "parser.y" { result->statements_ptr_ = (yyvsp[-1].stmt_array); } -#line 3028 "parser.cpp" +#line 3030 "parser.cpp" break; case 3: /* statement_list: statement */ -#line 477 "parser.y" +#line 480 "parser.y" { (yyvsp[0].base_stmt)->stmt_length_ = yylloc.string_length; yylloc.string_length = 0; (yyval.stmt_array) = new std::vector(); (yyval.stmt_array)->push_back((yyvsp[0].base_stmt)); } -#line 3039 "parser.cpp" +#line 3041 "parser.cpp" break; case 4: /* statement_list: statement_list ';' statement */ -#line 483 "parser.y" +#line 486 "parser.y" { (yyvsp[0].base_stmt)->stmt_length_ = yylloc.string_length; yylloc.string_length = 0; (yyvsp[-2].stmt_array)->push_back((yyvsp[0].base_stmt)); (yyval.stmt_array) = (yyvsp[-2].stmt_array); } -#line 3050 "parser.cpp" +#line 3052 "parser.cpp" break; case 5: /* statement: create_statement */ -#line 490 "parser.y" +#line 493 "parser.y" { (yyval.base_stmt) = (yyvsp[0].create_stmt); } -#line 3056 "parser.cpp" +#line 3058 "parser.cpp" break; case 6: /* statement: drop_statement */ -#line 491 "parser.y" +#line 494 "parser.y" { (yyval.base_stmt) = (yyvsp[0].drop_stmt); } -#line 3062 "parser.cpp" +#line 3064 "parser.cpp" break; case 7: /* statement: copy_statement */ -#line 492 "parser.y" +#line 495 "parser.y" { (yyval.base_stmt) = (yyvsp[0].copy_stmt); } -#line 3068 "parser.cpp" +#line 3070 "parser.cpp" break; case 8: /* statement: show_statement */ -#line 493 "parser.y" +#line 496 "parser.y" { (yyval.base_stmt) = (yyvsp[0].show_stmt); } -#line 3074 "parser.cpp" +#line 3076 "parser.cpp" break; case 9: /* statement: select_statement */ -#line 494 "parser.y" +#line 497 "parser.y" { (yyval.base_stmt) = (yyvsp[0].select_stmt); } -#line 3080 "parser.cpp" +#line 3082 "parser.cpp" break; case 10: /* statement: delete_statement */ -#line 495 "parser.y" +#line 498 "parser.y" { (yyval.base_stmt) = (yyvsp[0].delete_stmt); } -#line 3086 "parser.cpp" +#line 3088 "parser.cpp" break; case 11: /* statement: update_statement */ -#line 496 "parser.y" +#line 499 "parser.y" { (yyval.base_stmt) = (yyvsp[0].update_stmt); } -#line 3092 "parser.cpp" +#line 3094 "parser.cpp" break; case 12: /* statement: insert_statement */ -#line 497 "parser.y" +#line 500 "parser.y" { (yyval.base_stmt) = (yyvsp[0].insert_stmt); } -#line 3098 "parser.cpp" +#line 3100 "parser.cpp" break; case 13: /* statement: explain_statement */ -#line 498 "parser.y" +#line 501 "parser.y" { (yyval.base_stmt) = (yyvsp[0].explain_stmt); } -#line 3104 "parser.cpp" +#line 3106 "parser.cpp" break; case 14: /* statement: flush_statement */ -#line 499 "parser.y" +#line 502 "parser.y" { (yyval.base_stmt) = (yyvsp[0].flush_stmt); } -#line 3110 "parser.cpp" +#line 3112 "parser.cpp" break; case 15: /* statement: optimize_statement */ -#line 500 "parser.y" +#line 503 "parser.y" { (yyval.base_stmt) = (yyvsp[0].optimize_stmt); } -#line 3116 "parser.cpp" +#line 3118 "parser.cpp" break; case 16: /* statement: command_statement */ -#line 501 "parser.y" +#line 504 "parser.y" { (yyval.base_stmt) = (yyvsp[0].command_stmt); } -#line 3122 "parser.cpp" +#line 3124 "parser.cpp" break; - case 17: /* explainable_statement: create_statement */ -#line 503 "parser.y" + case 17: /* statement: compact_statement */ +#line 505 "parser.y" + { (yyval.base_stmt) = (yyvsp[0].compact_stmt); } +#line 3130 "parser.cpp" + break; + + case 18: /* explainable_statement: create_statement */ +#line 507 "parser.y" { (yyval.base_stmt) = (yyvsp[0].create_stmt); } -#line 3128 "parser.cpp" +#line 3136 "parser.cpp" break; - case 18: /* explainable_statement: drop_statement */ -#line 504 "parser.y" + case 19: /* explainable_statement: drop_statement */ +#line 508 "parser.y" { (yyval.base_stmt) = (yyvsp[0].drop_stmt); } -#line 3134 "parser.cpp" +#line 3142 "parser.cpp" break; - case 19: /* explainable_statement: copy_statement */ -#line 505 "parser.y" + case 20: /* explainable_statement: copy_statement */ +#line 509 "parser.y" { (yyval.base_stmt) = (yyvsp[0].copy_stmt); } -#line 3140 "parser.cpp" +#line 3148 "parser.cpp" break; - case 20: /* explainable_statement: show_statement */ -#line 506 "parser.y" + case 21: /* explainable_statement: show_statement */ +#line 510 "parser.y" { (yyval.base_stmt) = (yyvsp[0].show_stmt); } -#line 3146 "parser.cpp" +#line 3154 "parser.cpp" break; - case 21: /* explainable_statement: select_statement */ -#line 507 "parser.y" + case 22: /* explainable_statement: select_statement */ +#line 511 "parser.y" { (yyval.base_stmt) = (yyvsp[0].select_stmt); } -#line 3152 "parser.cpp" +#line 3160 "parser.cpp" break; - case 22: /* explainable_statement: delete_statement */ -#line 508 "parser.y" + case 23: /* explainable_statement: delete_statement */ +#line 512 "parser.y" { (yyval.base_stmt) = (yyvsp[0].delete_stmt); } -#line 3158 "parser.cpp" +#line 3166 "parser.cpp" break; - case 23: /* explainable_statement: update_statement */ -#line 509 "parser.y" + case 24: /* explainable_statement: update_statement */ +#line 513 "parser.y" { (yyval.base_stmt) = (yyvsp[0].update_stmt); } -#line 3164 "parser.cpp" +#line 3172 "parser.cpp" break; - case 24: /* explainable_statement: insert_statement */ -#line 510 "parser.y" + case 25: /* explainable_statement: insert_statement */ +#line 514 "parser.y" { (yyval.base_stmt) = (yyvsp[0].insert_stmt); } -#line 3170 "parser.cpp" +#line 3178 "parser.cpp" break; - case 25: /* explainable_statement: flush_statement */ -#line 511 "parser.y" + case 26: /* explainable_statement: flush_statement */ +#line 515 "parser.y" { (yyval.base_stmt) = (yyvsp[0].flush_stmt); } -#line 3176 "parser.cpp" +#line 3184 "parser.cpp" break; - case 26: /* explainable_statement: optimize_statement */ -#line 512 "parser.y" + case 27: /* explainable_statement: optimize_statement */ +#line 516 "parser.y" { (yyval.base_stmt) = (yyvsp[0].optimize_stmt); } -#line 3182 "parser.cpp" +#line 3190 "parser.cpp" break; - case 27: /* explainable_statement: command_statement */ -#line 513 "parser.y" + case 28: /* explainable_statement: command_statement */ +#line 517 "parser.y" { (yyval.base_stmt) = (yyvsp[0].command_stmt); } -#line 3188 "parser.cpp" +#line 3196 "parser.cpp" + break; + + case 29: /* explainable_statement: compact_statement */ +#line 518 "parser.y" + { (yyval.base_stmt) = (yyvsp[0].compact_stmt); } +#line 3202 "parser.cpp" break; - case 28: /* create_statement: CREATE DATABASE if_not_exists IDENTIFIER */ -#line 520 "parser.y" + case 30: /* create_statement: CREATE DATABASE if_not_exists IDENTIFIER */ +#line 525 "parser.y" { (yyval.create_stmt) = new infinity::CreateStatement(); std::shared_ptr create_schema_info = std::make_shared(); @@ -3204,11 +3218,11 @@ 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 3208 "parser.cpp" +#line 3222 "parser.cpp" break; - case 29: /* create_statement: CREATE COLLECTION if_not_exists table_name */ -#line 537 "parser.y" + case 31: /* create_statement: CREATE COLLECTION if_not_exists table_name */ +#line 542 "parser.y" { (yyval.create_stmt) = new infinity::CreateStatement(); std::shared_ptr create_collection_info = std::make_shared(); @@ -3222,11 +3236,11 @@ 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 3226 "parser.cpp" +#line 3240 "parser.cpp" break; - case 30: /* create_statement: CREATE TABLE if_not_exists table_name '(' table_element_array ')' optional_table_properties_list */ -#line 553 "parser.y" + case 32: /* create_statement: CREATE TABLE if_not_exists table_name '(' table_element_array ')' optional_table_properties_list */ +#line 558 "parser.y" { (yyval.create_stmt) = new infinity::CreateStatement(); std::shared_ptr create_table_info = std::make_shared(); @@ -3255,11 +3269,11 @@ 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 3259 "parser.cpp" +#line 3273 "parser.cpp" break; - case 31: /* create_statement: CREATE TABLE if_not_exists table_name AS select_statement */ -#line 582 "parser.y" + case 33: /* create_statement: CREATE TABLE if_not_exists table_name AS select_statement */ +#line 587 "parser.y" { (yyval.create_stmt) = new infinity::CreateStatement(); std::shared_ptr create_table_info = std::make_shared(); @@ -3275,11 +3289,11 @@ YYLTYPE yylloc = yyloc_default; create_table_info->select_ = (yyvsp[0].select_stmt); (yyval.create_stmt)->create_info_ = create_table_info; } -#line 3279 "parser.cpp" +#line 3293 "parser.cpp" break; - case 32: /* create_statement: CREATE VIEW if_not_exists table_name optional_identifier_array AS select_statement */ -#line 598 "parser.y" + case 34: /* create_statement: CREATE VIEW if_not_exists table_name optional_identifier_array AS select_statement */ +#line 603 "parser.y" { (yyval.create_stmt) = new infinity::CreateStatement(); std::shared_ptr create_view_info = std::make_shared(); @@ -3296,11 +3310,11 @@ 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 3300 "parser.cpp" +#line 3314 "parser.cpp" break; - case 33: /* create_statement: CREATE INDEX if_not_exists_info ON table_name index_info_list */ -#line 616 "parser.y" + case 35: /* create_statement: CREATE INDEX if_not_exists_info ON table_name index_info_list */ +#line 621 "parser.y" { std::shared_ptr create_index_info = std::make_shared(); if((yyvsp[-1].table_name_t)->schema_name_ptr_ != nullptr) { @@ -3329,45 +3343,45 @@ YYLTYPE yylloc = yyloc_default; (yyval.create_stmt) = new infinity::CreateStatement(); (yyval.create_stmt)->create_info_ = create_index_info; } -#line 3333 "parser.cpp" +#line 3347 "parser.cpp" break; - case 34: /* table_element_array: table_element */ -#line 645 "parser.y" + case 36: /* table_element_array: table_element */ +#line 650 "parser.y" { (yyval.table_element_array_t) = new std::vector(); (yyval.table_element_array_t)->push_back((yyvsp[0].table_element_t)); } -#line 3342 "parser.cpp" +#line 3356 "parser.cpp" break; - case 35: /* table_element_array: table_element_array ',' table_element */ -#line 649 "parser.y" + case 37: /* table_element_array: table_element_array ',' table_element */ +#line 654 "parser.y" { (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 3351 "parser.cpp" +#line 3365 "parser.cpp" break; - case 36: /* table_element: table_column */ -#line 655 "parser.y" + case 38: /* table_element: table_column */ +#line 660 "parser.y" { (yyval.table_element_t) = (yyvsp[0].table_column_t); } -#line 3359 "parser.cpp" +#line 3373 "parser.cpp" break; - case 37: /* table_element: table_constraint */ -#line 658 "parser.y" + case 39: /* table_element: table_constraint */ +#line 663 "parser.y" { (yyval.table_element_t) = (yyvsp[0].table_constraint_t); } -#line 3367 "parser.cpp" +#line 3381 "parser.cpp" break; - case 38: /* table_column: IDENTIFIER column_type default_expr */ -#line 664 "parser.y" + case 40: /* table_column: IDENTIFIER column_type default_expr */ +#line 669 "parser.y" { std::shared_ptr type_info_ptr{nullptr}; switch((yyvsp[-1].column_type_t).logical_type_) { @@ -3406,11 +3420,11 @@ YYLTYPE yylloc = yyloc_default; } */ } -#line 3410 "parser.cpp" +#line 3424 "parser.cpp" break; - case 39: /* table_column: IDENTIFIER column_type column_constraints default_expr */ -#line 702 "parser.y" + case 41: /* table_column: IDENTIFIER column_type column_constraints default_expr */ +#line 707 "parser.y" { std::shared_ptr type_info_ptr{nullptr}; switch((yyvsp[-2].column_type_t).logical_type_) { @@ -3445,308 +3459,308 @@ YYLTYPE yylloc = yyloc_default; } */ } -#line 3449 "parser.cpp" +#line 3463 "parser.cpp" break; - case 40: /* column_type: BOOLEAN */ -#line 738 "parser.y" + case 42: /* column_type: BOOLEAN */ +#line 743 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kBoolean, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3455 "parser.cpp" +#line 3469 "parser.cpp" break; - case 41: /* column_type: TINYINT */ -#line 739 "parser.y" + case 43: /* column_type: TINYINT */ +#line 744 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTinyInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3461 "parser.cpp" +#line 3475 "parser.cpp" break; - case 42: /* column_type: SMALLINT */ -#line 740 "parser.y" + case 44: /* column_type: SMALLINT */ +#line 745 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kSmallInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3467 "parser.cpp" +#line 3481 "parser.cpp" break; - case 43: /* column_type: INTEGER */ -#line 741 "parser.y" + case 45: /* column_type: INTEGER */ +#line 746 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kInteger, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3473 "parser.cpp" +#line 3487 "parser.cpp" break; - case 44: /* column_type: INT */ -#line 742 "parser.y" + case 46: /* column_type: INT */ +#line 747 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kInteger, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3479 "parser.cpp" +#line 3493 "parser.cpp" break; - case 45: /* column_type: BIGINT */ -#line 743 "parser.y" + case 47: /* column_type: BIGINT */ +#line 748 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kBigInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3485 "parser.cpp" +#line 3499 "parser.cpp" break; - case 46: /* column_type: HUGEINT */ -#line 744 "parser.y" + case 48: /* column_type: HUGEINT */ +#line 749 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kHugeInt, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3491 "parser.cpp" +#line 3505 "parser.cpp" break; - case 47: /* column_type: FLOAT */ -#line 745 "parser.y" + case 49: /* column_type: FLOAT */ +#line 750 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kFloat, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3497 "parser.cpp" +#line 3511 "parser.cpp" break; - case 48: /* column_type: REAL */ -#line 746 "parser.y" + case 50: /* column_type: REAL */ +#line 751 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kFloat, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3503 "parser.cpp" +#line 3517 "parser.cpp" break; - case 49: /* column_type: DOUBLE */ -#line 747 "parser.y" + case 51: /* column_type: DOUBLE */ +#line 752 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDouble, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3509 "parser.cpp" +#line 3523 "parser.cpp" break; - case 50: /* column_type: DATE */ -#line 748 "parser.y" + case 52: /* column_type: DATE */ +#line 753 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDate, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3515 "parser.cpp" +#line 3529 "parser.cpp" break; - case 51: /* column_type: TIME */ -#line 749 "parser.y" + case 53: /* column_type: TIME */ +#line 754 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTime, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3521 "parser.cpp" +#line 3535 "parser.cpp" break; - case 52: /* column_type: DATETIME */ -#line 750 "parser.y" + case 54: /* column_type: DATETIME */ +#line 755 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDateTime, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3527 "parser.cpp" +#line 3541 "parser.cpp" break; - case 53: /* column_type: TIMESTAMP */ -#line 751 "parser.y" + case 55: /* column_type: TIMESTAMP */ +#line 756 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTimestamp, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3533 "parser.cpp" +#line 3547 "parser.cpp" break; - case 54: /* column_type: UUID */ -#line 752 "parser.y" + case 56: /* column_type: UUID */ +#line 757 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kUuid, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3539 "parser.cpp" +#line 3553 "parser.cpp" break; - case 55: /* column_type: POINT */ -#line 753 "parser.y" + case 57: /* column_type: POINT */ +#line 758 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kPoint, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3545 "parser.cpp" +#line 3559 "parser.cpp" break; - case 56: /* column_type: LINE */ -#line 754 "parser.y" + case 58: /* column_type: LINE */ +#line 759 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kLine, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3551 "parser.cpp" +#line 3565 "parser.cpp" break; - case 57: /* column_type: LSEG */ -#line 755 "parser.y" + case 59: /* column_type: LSEG */ +#line 760 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kLineSeg, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3557 "parser.cpp" +#line 3571 "parser.cpp" break; - case 58: /* column_type: BOX */ -#line 756 "parser.y" + case 60: /* column_type: BOX */ +#line 761 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kBox, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3563 "parser.cpp" +#line 3577 "parser.cpp" break; - case 59: /* column_type: CIRCLE */ -#line 759 "parser.y" + case 61: /* column_type: CIRCLE */ +#line 764 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kCircle, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3569 "parser.cpp" +#line 3583 "parser.cpp" break; - case 60: /* column_type: VARCHAR */ -#line 761 "parser.y" + case 62: /* column_type: VARCHAR */ +#line 766 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kVarchar, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3575 "parser.cpp" +#line 3589 "parser.cpp" break; - case 61: /* column_type: DECIMAL '(' LONG_VALUE ',' LONG_VALUE ')' */ -#line 762 "parser.y" + case 63: /* column_type: DECIMAL '(' LONG_VALUE ',' LONG_VALUE ')' */ +#line 767 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDecimal, 0, (yyvsp[-3].long_value), (yyvsp[-1].long_value), infinity::EmbeddingDataType::kElemInvalid}; } -#line 3581 "parser.cpp" +#line 3595 "parser.cpp" break; - case 62: /* column_type: DECIMAL '(' LONG_VALUE ')' */ -#line 763 "parser.y" + case 64: /* column_type: DECIMAL '(' LONG_VALUE ')' */ +#line 768 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDecimal, 0, (yyvsp[-1].long_value), 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3587 "parser.cpp" +#line 3601 "parser.cpp" break; - case 63: /* column_type: DECIMAL */ -#line 764 "parser.y" + case 65: /* column_type: DECIMAL */ +#line 769 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kDecimal, 0, 0, 0, infinity::EmbeddingDataType::kElemInvalid}; } -#line 3593 "parser.cpp" +#line 3607 "parser.cpp" break; - case 64: /* column_type: EMBEDDING '(' BIT ',' LONG_VALUE ')' */ -#line 767 "parser.y" + case 66: /* column_type: EMBEDDING '(' BIT ',' LONG_VALUE ')' */ +#line 772 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 3599 "parser.cpp" +#line 3613 "parser.cpp" break; - case 65: /* column_type: EMBEDDING '(' TINYINT ',' LONG_VALUE ')' */ -#line 768 "parser.y" + case 67: /* column_type: EMBEDDING '(' TINYINT ',' LONG_VALUE ')' */ +#line 773 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 3605 "parser.cpp" +#line 3619 "parser.cpp" break; - case 66: /* column_type: EMBEDDING '(' SMALLINT ',' LONG_VALUE ')' */ -#line 769 "parser.y" + case 68: /* column_type: EMBEDDING '(' SMALLINT ',' LONG_VALUE ')' */ +#line 774 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 3611 "parser.cpp" +#line 3625 "parser.cpp" break; - case 67: /* column_type: EMBEDDING '(' INTEGER ',' LONG_VALUE ')' */ -#line 770 "parser.y" + case 69: /* column_type: EMBEDDING '(' INTEGER ',' LONG_VALUE ')' */ +#line 775 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3617 "parser.cpp" +#line 3631 "parser.cpp" break; - case 68: /* column_type: EMBEDDING '(' INT ',' LONG_VALUE ')' */ -#line 771 "parser.y" + case 70: /* column_type: EMBEDDING '(' INT ',' LONG_VALUE ')' */ +#line 776 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3623 "parser.cpp" +#line 3637 "parser.cpp" break; - case 69: /* column_type: EMBEDDING '(' BIGINT ',' LONG_VALUE ')' */ -#line 772 "parser.y" + case 71: /* column_type: EMBEDDING '(' BIGINT ',' LONG_VALUE ')' */ +#line 777 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 3629 "parser.cpp" +#line 3643 "parser.cpp" break; - case 70: /* column_type: EMBEDDING '(' FLOAT ',' LONG_VALUE ')' */ -#line 773 "parser.y" + case 72: /* column_type: EMBEDDING '(' FLOAT ',' LONG_VALUE ')' */ +#line 778 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 3635 "parser.cpp" +#line 3649 "parser.cpp" break; - case 71: /* column_type: EMBEDDING '(' DOUBLE ',' LONG_VALUE ')' */ -#line 774 "parser.y" + case 73: /* column_type: EMBEDDING '(' DOUBLE ',' LONG_VALUE ')' */ +#line 779 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 3641 "parser.cpp" +#line 3655 "parser.cpp" break; - case 72: /* column_type: TENSOR '(' BIT ',' LONG_VALUE ')' */ -#line 775 "parser.y" + case 74: /* column_type: TENSOR '(' BIT ',' LONG_VALUE ')' */ +#line 780 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 3647 "parser.cpp" +#line 3661 "parser.cpp" break; - case 73: /* column_type: TENSOR '(' TINYINT ',' LONG_VALUE ')' */ -#line 776 "parser.y" + case 75: /* column_type: TENSOR '(' TINYINT ',' LONG_VALUE ')' */ +#line 781 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 3653 "parser.cpp" +#line 3667 "parser.cpp" break; - case 74: /* column_type: TENSOR '(' SMALLINT ',' LONG_VALUE ')' */ -#line 777 "parser.y" + case 76: /* column_type: TENSOR '(' SMALLINT ',' LONG_VALUE ')' */ +#line 782 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 3659 "parser.cpp" +#line 3673 "parser.cpp" break; - case 75: /* column_type: TENSOR '(' INTEGER ',' LONG_VALUE ')' */ -#line 778 "parser.y" + case 77: /* column_type: TENSOR '(' INTEGER ',' LONG_VALUE ')' */ +#line 783 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3665 "parser.cpp" +#line 3679 "parser.cpp" break; - case 76: /* column_type: TENSOR '(' INT ',' LONG_VALUE ')' */ -#line 779 "parser.y" + case 78: /* column_type: TENSOR '(' INT ',' LONG_VALUE ')' */ +#line 784 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3671 "parser.cpp" +#line 3685 "parser.cpp" break; - case 77: /* column_type: TENSOR '(' BIGINT ',' LONG_VALUE ')' */ -#line 780 "parser.y" + case 79: /* column_type: TENSOR '(' BIGINT ',' LONG_VALUE ')' */ +#line 785 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 3677 "parser.cpp" +#line 3691 "parser.cpp" break; - case 78: /* column_type: TENSOR '(' FLOAT ',' LONG_VALUE ')' */ -#line 781 "parser.y" + case 80: /* column_type: TENSOR '(' FLOAT ',' LONG_VALUE ')' */ +#line 786 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 3683 "parser.cpp" +#line 3697 "parser.cpp" break; - case 79: /* column_type: TENSOR '(' DOUBLE ',' LONG_VALUE ')' */ -#line 782 "parser.y" + case 81: /* column_type: TENSOR '(' DOUBLE ',' LONG_VALUE ')' */ +#line 787 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kTensor, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 3689 "parser.cpp" +#line 3703 "parser.cpp" break; - case 80: /* column_type: VECTOR '(' BIT ',' LONG_VALUE ')' */ -#line 783 "parser.y" + case 82: /* column_type: VECTOR '(' BIT ',' LONG_VALUE ')' */ +#line 788 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemBit}; } -#line 3695 "parser.cpp" +#line 3709 "parser.cpp" break; - case 81: /* column_type: VECTOR '(' TINYINT ',' LONG_VALUE ')' */ -#line 784 "parser.y" + case 83: /* column_type: VECTOR '(' TINYINT ',' LONG_VALUE ')' */ +#line 789 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt8}; } -#line 3701 "parser.cpp" +#line 3715 "parser.cpp" break; - case 82: /* column_type: VECTOR '(' SMALLINT ',' LONG_VALUE ')' */ -#line 785 "parser.y" + case 84: /* column_type: VECTOR '(' SMALLINT ',' LONG_VALUE ')' */ +#line 790 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt16}; } -#line 3707 "parser.cpp" +#line 3721 "parser.cpp" break; - case 83: /* column_type: VECTOR '(' INTEGER ',' LONG_VALUE ')' */ -#line 786 "parser.y" + case 85: /* column_type: VECTOR '(' INTEGER ',' LONG_VALUE ')' */ +#line 791 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3713 "parser.cpp" +#line 3727 "parser.cpp" break; - case 84: /* column_type: VECTOR '(' INT ',' LONG_VALUE ')' */ -#line 787 "parser.y" + case 86: /* column_type: VECTOR '(' INT ',' LONG_VALUE ')' */ +#line 792 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt32}; } -#line 3719 "parser.cpp" +#line 3733 "parser.cpp" break; - case 85: /* column_type: VECTOR '(' BIGINT ',' LONG_VALUE ')' */ -#line 788 "parser.y" + case 87: /* column_type: VECTOR '(' BIGINT ',' LONG_VALUE ')' */ +#line 793 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemInt64}; } -#line 3725 "parser.cpp" +#line 3739 "parser.cpp" break; - case 86: /* column_type: VECTOR '(' FLOAT ',' LONG_VALUE ')' */ -#line 789 "parser.y" + case 88: /* column_type: VECTOR '(' FLOAT ',' LONG_VALUE ')' */ +#line 794 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemFloat}; } -#line 3731 "parser.cpp" +#line 3745 "parser.cpp" break; - case 87: /* column_type: VECTOR '(' DOUBLE ',' LONG_VALUE ')' */ -#line 790 "parser.y" + case 89: /* column_type: VECTOR '(' DOUBLE ',' LONG_VALUE ')' */ +#line 795 "parser.y" { (yyval.column_type_t) = infinity::ColumnType{infinity::LogicalType::kEmbedding, (yyvsp[-1].long_value), 0, 0, infinity::kElemDouble}; } -#line 3737 "parser.cpp" +#line 3751 "parser.cpp" break; - case 88: /* column_constraints: column_constraint */ -#line 809 "parser.y" + case 90: /* column_constraints: column_constraint */ +#line 814 "parser.y" { (yyval.column_constraints_t) = new std::unordered_set(); (yyval.column_constraints_t)->insert((yyvsp[0].column_constraint_t)); } -#line 3746 "parser.cpp" +#line 3760 "parser.cpp" break; - case 89: /* column_constraints: column_constraints column_constraint */ -#line 813 "parser.y" + case 91: /* column_constraints: column_constraints column_constraint */ +#line 818 "parser.y" { if((yyvsp[-1].column_constraints_t)->contains((yyvsp[0].column_constraint_t))) { yyerror(&yyloc, scanner, result, "Duplicate column constraint."); @@ -3756,101 +3770,101 @@ 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 3760 "parser.cpp" +#line 3774 "parser.cpp" break; - case 90: /* column_constraint: PRIMARY KEY */ -#line 823 "parser.y" + case 92: /* column_constraint: PRIMARY KEY */ +#line 828 "parser.y" { (yyval.column_constraint_t) = infinity::ConstraintType::kPrimaryKey; } -#line 3768 "parser.cpp" +#line 3782 "parser.cpp" break; - case 91: /* column_constraint: UNIQUE */ -#line 826 "parser.y" + case 93: /* column_constraint: UNIQUE */ +#line 831 "parser.y" { (yyval.column_constraint_t) = infinity::ConstraintType::kUnique; } -#line 3776 "parser.cpp" +#line 3790 "parser.cpp" break; - case 92: /* column_constraint: NULLABLE */ -#line 829 "parser.y" + case 94: /* column_constraint: NULLABLE */ +#line 834 "parser.y" { (yyval.column_constraint_t) = infinity::ConstraintType::kNull; } -#line 3784 "parser.cpp" +#line 3798 "parser.cpp" break; - case 93: /* column_constraint: NOT NULLABLE */ -#line 832 "parser.y" + case 95: /* column_constraint: NOT NULLABLE */ +#line 837 "parser.y" { (yyval.column_constraint_t) = infinity::ConstraintType::kNotNull; } -#line 3792 "parser.cpp" +#line 3806 "parser.cpp" break; - case 94: /* default_expr: DEFAULT constant_expr */ -#line 836 "parser.y" + case 96: /* default_expr: DEFAULT constant_expr */ +#line 841 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 3800 "parser.cpp" +#line 3814 "parser.cpp" break; - case 95: /* default_expr: %empty */ -#line 839 "parser.y" + case 97: /* default_expr: %empty */ +#line 844 "parser.y" { (yyval.const_expr_t) = nullptr; } -#line 3808 "parser.cpp" +#line 3822 "parser.cpp" break; - case 96: /* table_constraint: PRIMARY KEY '(' identifier_array ')' */ -#line 844 "parser.y" + case 98: /* table_constraint: PRIMARY KEY '(' identifier_array ')' */ +#line 849 "parser.y" { (yyval.table_constraint_t) = new infinity::TableConstraint(); (yyval.table_constraint_t)->names_ptr_ = (yyvsp[-1].identifier_array_t); (yyval.table_constraint_t)->constraint_ = infinity::ConstraintType::kPrimaryKey; } -#line 3818 "parser.cpp" +#line 3832 "parser.cpp" break; - case 97: /* table_constraint: UNIQUE '(' identifier_array ')' */ -#line 849 "parser.y" + case 99: /* table_constraint: UNIQUE '(' identifier_array ')' */ +#line 854 "parser.y" { (yyval.table_constraint_t) = new infinity::TableConstraint(); (yyval.table_constraint_t)->names_ptr_ = (yyvsp[-1].identifier_array_t); (yyval.table_constraint_t)->constraint_ = infinity::ConstraintType::kUnique; } -#line 3828 "parser.cpp" +#line 3842 "parser.cpp" break; - case 98: /* identifier_array: IDENTIFIER */ -#line 856 "parser.y" + case 100: /* identifier_array: IDENTIFIER */ +#line 861 "parser.y" { (yyval.identifier_array_t) = new std::vector(); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.identifier_array_t)->emplace_back((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 3839 "parser.cpp" +#line 3853 "parser.cpp" break; - case 99: /* identifier_array: identifier_array ',' IDENTIFIER */ -#line 862 "parser.y" + case 101: /* identifier_array: identifier_array ',' IDENTIFIER */ +#line 867 "parser.y" { ParserHelper::ToLower((yyvsp[0].str_value)); (yyvsp[-2].identifier_array_t)->emplace_back((yyvsp[0].str_value)); free((yyvsp[0].str_value)); (yyval.identifier_array_t) = (yyvsp[-2].identifier_array_t); } -#line 3850 "parser.cpp" +#line 3864 "parser.cpp" break; - case 100: /* delete_statement: DELETE FROM table_name where_clause */ -#line 872 "parser.y" + case 102: /* delete_statement: DELETE FROM table_name where_clause */ +#line 877 "parser.y" { (yyval.delete_stmt) = new infinity::DeleteStatement(); @@ -3863,11 +3877,11 @@ YYLTYPE yylloc = yyloc_default; delete (yyvsp[-1].table_name_t); (yyval.delete_stmt)->where_expr_ = (yyvsp[0].expr_t); } -#line 3867 "parser.cpp" +#line 3881 "parser.cpp" break; - case 101: /* insert_statement: INSERT INTO table_name optional_identifier_array VALUES expr_array_list */ -#line 888 "parser.y" + case 103: /* insert_statement: INSERT INTO table_name optional_identifier_array VALUES expr_array_list */ +#line 893 "parser.y" { bool is_error{false}; for (auto expr_array : *(yyvsp[0].expr_array_list_t)) { @@ -3902,11 +3916,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.insert_stmt)->columns_ = (yyvsp[-2].identifier_array_t); (yyval.insert_stmt)->values_ = (yyvsp[0].expr_array_list_t); } -#line 3906 "parser.cpp" +#line 3920 "parser.cpp" break; - case 102: /* insert_statement: INSERT INTO table_name optional_identifier_array select_without_paren */ -#line 922 "parser.y" + case 104: /* insert_statement: INSERT INTO table_name optional_identifier_array select_without_paren */ +#line 927 "parser.y" { (yyval.insert_stmt) = new infinity::InsertStatement(); if((yyvsp[-2].table_name_t)->schema_name_ptr_ != nullptr) { @@ -3919,101 +3933,101 @@ YYLTYPE yylloc = yyloc_default; (yyval.insert_stmt)->columns_ = (yyvsp[-1].identifier_array_t); (yyval.insert_stmt)->select_ = (yyvsp[0].select_stmt); } -#line 3923 "parser.cpp" +#line 3937 "parser.cpp" break; - case 103: /* optional_identifier_array: '(' identifier_array ')' */ -#line 935 "parser.y" + case 105: /* optional_identifier_array: '(' identifier_array ')' */ +#line 940 "parser.y" { (yyval.identifier_array_t) = (yyvsp[-1].identifier_array_t); } -#line 3931 "parser.cpp" +#line 3945 "parser.cpp" break; - case 104: /* optional_identifier_array: %empty */ -#line 938 "parser.y" + case 106: /* optional_identifier_array: %empty */ +#line 943 "parser.y" { (yyval.identifier_array_t) = nullptr; } -#line 3939 "parser.cpp" +#line 3953 "parser.cpp" break; - case 105: /* explain_statement: EXPLAIN explain_type explainable_statement */ -#line 945 "parser.y" + case 107: /* explain_statement: EXPLAIN explain_type explainable_statement */ +#line 950 "parser.y" { (yyval.explain_stmt) = new infinity::ExplainStatement(); (yyval.explain_stmt)->type_ = (yyvsp[-1].explain_type_t); (yyval.explain_stmt)->statement_ = (yyvsp[0].base_stmt); } -#line 3949 "parser.cpp" +#line 3963 "parser.cpp" break; - case 106: /* explain_type: ANALYZE */ -#line 951 "parser.y" + case 108: /* explain_type: ANALYZE */ +#line 956 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kAnalyze; } -#line 3957 "parser.cpp" +#line 3971 "parser.cpp" break; - case 107: /* explain_type: AST */ -#line 954 "parser.y" + case 109: /* explain_type: AST */ +#line 959 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kAst; } -#line 3965 "parser.cpp" +#line 3979 "parser.cpp" break; - case 108: /* explain_type: RAW */ -#line 957 "parser.y" + case 110: /* explain_type: RAW */ +#line 962 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kUnOpt; } -#line 3973 "parser.cpp" +#line 3987 "parser.cpp" break; - case 109: /* explain_type: LOGICAL */ -#line 960 "parser.y" + case 111: /* explain_type: LOGICAL */ +#line 965 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kOpt; } -#line 3981 "parser.cpp" +#line 3995 "parser.cpp" break; - case 110: /* explain_type: PHYSICAL */ -#line 963 "parser.y" + case 112: /* explain_type: PHYSICAL */ +#line 968 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kPhysical; } -#line 3989 "parser.cpp" +#line 4003 "parser.cpp" break; - case 111: /* explain_type: PIPELINE */ -#line 966 "parser.y" + case 113: /* explain_type: PIPELINE */ +#line 971 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kPipeline; } -#line 3997 "parser.cpp" +#line 4011 "parser.cpp" break; - case 112: /* explain_type: FRAGMENT */ -#line 969 "parser.y" + case 114: /* explain_type: FRAGMENT */ +#line 974 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kFragment; } -#line 4005 "parser.cpp" +#line 4019 "parser.cpp" break; - case 113: /* explain_type: %empty */ -#line 972 "parser.y" + case 115: /* explain_type: %empty */ +#line 977 "parser.y" { (yyval.explain_type_t) = infinity::ExplainType::kPhysical; } -#line 4013 "parser.cpp" +#line 4027 "parser.cpp" break; - case 114: /* update_statement: UPDATE table_name SET update_expr_array where_clause */ -#line 979 "parser.y" + case 116: /* update_statement: UPDATE table_name SET update_expr_array where_clause */ +#line 984 "parser.y" { (yyval.update_stmt) = new infinity::UpdateStatement(); if((yyvsp[-3].table_name_t)->schema_name_ptr_ != nullptr) { @@ -4026,29 +4040,29 @@ 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 4030 "parser.cpp" +#line 4044 "parser.cpp" break; - case 115: /* update_expr_array: update_expr */ -#line 992 "parser.y" + case 117: /* update_expr_array: update_expr */ +#line 997 "parser.y" { (yyval.update_expr_array_t) = new std::vector(); (yyval.update_expr_array_t)->emplace_back((yyvsp[0].update_expr_t)); } -#line 4039 "parser.cpp" +#line 4053 "parser.cpp" break; - case 116: /* update_expr_array: update_expr_array ',' update_expr */ -#line 996 "parser.y" + case 118: /* update_expr_array: update_expr_array ',' update_expr */ +#line 1001 "parser.y" { (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 4048 "parser.cpp" +#line 4062 "parser.cpp" break; - case 117: /* update_expr: IDENTIFIER '=' expr */ -#line 1001 "parser.y" + case 119: /* update_expr: IDENTIFIER '=' expr */ +#line 1006 "parser.y" { (yyval.update_expr_t) = new infinity::UpdateExpr(); ParserHelper::ToLower((yyvsp[-2].str_value)); @@ -4056,11 +4070,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-2].str_value)); (yyval.update_expr_t)->value = (yyvsp[0].expr_t); } -#line 4060 "parser.cpp" +#line 4074 "parser.cpp" break; - case 118: /* drop_statement: DROP DATABASE if_exists IDENTIFIER */ -#line 1014 "parser.y" + case 120: /* drop_statement: DROP DATABASE if_exists IDENTIFIER */ +#line 1019 "parser.y" { (yyval.drop_stmt) = new infinity::DropStatement(); std::shared_ptr drop_schema_info = std::make_shared(); @@ -4072,11 +4086,11 @@ 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 4076 "parser.cpp" +#line 4090 "parser.cpp" break; - case 119: /* drop_statement: DROP COLLECTION if_exists table_name */ -#line 1027 "parser.y" + case 121: /* drop_statement: DROP COLLECTION if_exists table_name */ +#line 1032 "parser.y" { (yyval.drop_stmt) = new infinity::DropStatement(); std::shared_ptr drop_collection_info = std::make_unique(); @@ -4090,11 +4104,11 @@ 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 4094 "parser.cpp" +#line 4108 "parser.cpp" break; - case 120: /* drop_statement: DROP TABLE if_exists table_name */ -#line 1042 "parser.y" + case 122: /* drop_statement: DROP TABLE if_exists table_name */ +#line 1047 "parser.y" { (yyval.drop_stmt) = new infinity::DropStatement(); std::shared_ptr drop_table_info = std::make_unique(); @@ -4108,11 +4122,11 @@ 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 4112 "parser.cpp" +#line 4126 "parser.cpp" break; - case 121: /* drop_statement: DROP VIEW if_exists table_name */ -#line 1057 "parser.y" + case 123: /* drop_statement: DROP VIEW if_exists table_name */ +#line 1062 "parser.y" { (yyval.drop_stmt) = new infinity::DropStatement(); std::shared_ptr drop_view_info = std::make_unique(); @@ -4126,11 +4140,11 @@ 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 4130 "parser.cpp" +#line 4144 "parser.cpp" break; - case 122: /* drop_statement: DROP INDEX if_exists IDENTIFIER ON table_name */ -#line 1072 "parser.y" + case 124: /* drop_statement: DROP INDEX if_exists IDENTIFIER ON table_name */ +#line 1077 "parser.y" { (yyval.drop_stmt) = new infinity::DropStatement(); std::shared_ptr drop_index_info = std::make_shared(); @@ -4149,11 +4163,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].table_name_t)->table_name_ptr_); delete (yyvsp[0].table_name_t); } -#line 4153 "parser.cpp" +#line 4167 "parser.cpp" break; - case 123: /* copy_statement: COPY table_name TO file_path WITH '(' copy_option_list ')' */ -#line 1095 "parser.y" + case 125: /* copy_statement: COPY table_name TO file_path WITH '(' copy_option_list ')' */ +#line 1100 "parser.y" { (yyval.copy_stmt) = new infinity::CopyStatement(); @@ -4195,11 +4209,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].copy_option_array); } -#line 4199 "parser.cpp" +#line 4213 "parser.cpp" break; - case 124: /* copy_statement: COPY table_name FROM file_path WITH '(' copy_option_list ')' */ -#line 1136 "parser.y" + case 126: /* copy_statement: COPY table_name FROM file_path WITH '(' copy_option_list ')' */ +#line 1141 "parser.y" { (yyval.copy_stmt) = new infinity::CopyStatement(); @@ -4241,27 +4255,27 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].copy_option_array); } -#line 4245 "parser.cpp" +#line 4259 "parser.cpp" break; - case 125: /* select_statement: select_without_paren */ -#line 1181 "parser.y" + case 127: /* select_statement: select_without_paren */ +#line 1186 "parser.y" { (yyval.select_stmt) = (yyvsp[0].select_stmt); } -#line 4253 "parser.cpp" +#line 4267 "parser.cpp" break; - case 126: /* select_statement: select_with_paren */ -#line 1184 "parser.y" + case 128: /* select_statement: select_with_paren */ +#line 1189 "parser.y" { (yyval.select_stmt) = (yyvsp[0].select_stmt); } -#line 4261 "parser.cpp" +#line 4275 "parser.cpp" break; - case 127: /* select_statement: select_statement set_operator select_clause_without_modifier_paren */ -#line 1187 "parser.y" + case 129: /* select_statement: select_statement set_operator select_clause_without_modifier_paren */ +#line 1192 "parser.y" { infinity::SelectStatement* node = (yyvsp[-2].select_stmt); while(node->nested_select_ != nullptr) { @@ -4271,11 +4285,11 @@ YYLTYPE yylloc = yyloc_default; node->nested_select_ = (yyvsp[0].select_stmt); (yyval.select_stmt) = (yyvsp[-2].select_stmt); } -#line 4275 "parser.cpp" +#line 4289 "parser.cpp" break; - case 128: /* select_statement: select_statement set_operator select_clause_without_modifier */ -#line 1196 "parser.y" + case 130: /* select_statement: select_statement set_operator select_clause_without_modifier */ +#line 1201 "parser.y" { infinity::SelectStatement* node = (yyvsp[-2].select_stmt); while(node->nested_select_ != nullptr) { @@ -4285,36 +4299,36 @@ YYLTYPE yylloc = yyloc_default; node->nested_select_ = (yyvsp[0].select_stmt); (yyval.select_stmt) = (yyvsp[-2].select_stmt); } -#line 4289 "parser.cpp" +#line 4303 "parser.cpp" break; - case 129: /* select_with_paren: '(' select_without_paren ')' */ -#line 1206 "parser.y" + case 131: /* select_with_paren: '(' select_without_paren ')' */ +#line 1211 "parser.y" { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4297 "parser.cpp" +#line 4311 "parser.cpp" break; - case 130: /* select_with_paren: '(' select_with_paren ')' */ -#line 1209 "parser.y" + case 132: /* select_with_paren: '(' select_with_paren ')' */ +#line 1214 "parser.y" { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4305 "parser.cpp" +#line 4319 "parser.cpp" break; - case 131: /* select_without_paren: with_clause select_clause_with_modifier */ -#line 1213 "parser.y" + case 133: /* select_without_paren: with_clause select_clause_with_modifier */ +#line 1218 "parser.y" { (yyvsp[0].select_stmt)->with_exprs_ = (yyvsp[-1].with_expr_list_t); (yyval.select_stmt) = (yyvsp[0].select_stmt); } -#line 4314 "parser.cpp" +#line 4328 "parser.cpp" break; - case 132: /* select_clause_with_modifier: select_clause_without_modifier order_by_clause limit_expr offset_expr */ -#line 1218 "parser.y" + case 134: /* select_clause_with_modifier: select_clause_without_modifier order_by_clause limit_expr offset_expr */ +#line 1223 "parser.y" { if((yyvsp[-1].expr_t) == nullptr and (yyvsp[0].expr_t) != nullptr) { delete (yyvsp[-3].select_stmt); @@ -4336,27 +4350,27 @@ YYLTYPE yylloc = yyloc_default; (yyvsp[-3].select_stmt)->offset_expr_ = (yyvsp[0].expr_t); (yyval.select_stmt) = (yyvsp[-3].select_stmt); } -#line 4340 "parser.cpp" +#line 4354 "parser.cpp" break; - case 133: /* select_clause_without_modifier_paren: '(' select_clause_without_modifier ')' */ -#line 1240 "parser.y" + case 135: /* select_clause_without_modifier_paren: '(' select_clause_without_modifier ')' */ +#line 1245 "parser.y" { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4348 "parser.cpp" +#line 4362 "parser.cpp" break; - case 134: /* select_clause_without_modifier_paren: '(' select_clause_without_modifier_paren ')' */ -#line 1243 "parser.y" + case 136: /* select_clause_without_modifier_paren: '(' select_clause_without_modifier_paren ')' */ +#line 1248 "parser.y" { (yyval.select_stmt) = (yyvsp[-1].select_stmt); } -#line 4356 "parser.cpp" +#line 4370 "parser.cpp" break; - case 135: /* select_clause_without_modifier: SELECT distinct expr_array from_clause search_clause where_clause group_by_clause having_clause */ -#line 1248 "parser.y" + case 137: /* select_clause_without_modifier: SELECT distinct expr_array from_clause search_clause where_clause group_by_clause having_clause */ +#line 1253 "parser.y" { (yyval.select_stmt) = new infinity::SelectStatement(); (yyval.select_stmt)->select_list_ = (yyvsp[-5].expr_array_t); @@ -4372,245 +4386,245 @@ YYLTYPE yylloc = yyloc_default; YYERROR; } } -#line 4376 "parser.cpp" +#line 4390 "parser.cpp" break; - case 136: /* order_by_clause: ORDER BY order_by_expr_list */ -#line 1264 "parser.y" + case 138: /* order_by_clause: ORDER BY order_by_expr_list */ +#line 1269 "parser.y" { (yyval.order_by_expr_list_t) = (yyvsp[0].order_by_expr_list_t); } -#line 4384 "parser.cpp" +#line 4398 "parser.cpp" break; - case 137: /* order_by_clause: %empty */ -#line 1267 "parser.y" + case 139: /* order_by_clause: %empty */ +#line 1272 "parser.y" { (yyval.order_by_expr_list_t) = nullptr; } -#line 4392 "parser.cpp" +#line 4406 "parser.cpp" break; - case 138: /* order_by_expr_list: order_by_expr */ -#line 1271 "parser.y" + case 140: /* order_by_expr_list: order_by_expr */ +#line 1276 "parser.y" { (yyval.order_by_expr_list_t) = new std::vector(); (yyval.order_by_expr_list_t)->emplace_back((yyvsp[0].order_by_expr_t)); } -#line 4401 "parser.cpp" +#line 4415 "parser.cpp" break; - case 139: /* order_by_expr_list: order_by_expr_list ',' order_by_expr */ -#line 1275 "parser.y" + case 141: /* order_by_expr_list: order_by_expr_list ',' order_by_expr */ +#line 1280 "parser.y" { (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 4410 "parser.cpp" +#line 4424 "parser.cpp" break; - case 140: /* order_by_expr: expr order_by_type */ -#line 1280 "parser.y" + case 142: /* order_by_expr: expr order_by_type */ +#line 1285 "parser.y" { (yyval.order_by_expr_t) = new infinity::OrderByExpr(); (yyval.order_by_expr_t)->expr_ = (yyvsp[-1].expr_t); (yyval.order_by_expr_t)->type_ = (yyvsp[0].order_by_type_t); } -#line 4420 "parser.cpp" +#line 4434 "parser.cpp" break; - case 141: /* order_by_type: ASC */ -#line 1286 "parser.y" + case 143: /* order_by_type: ASC */ +#line 1291 "parser.y" { (yyval.order_by_type_t) = infinity::kAsc; } -#line 4428 "parser.cpp" +#line 4442 "parser.cpp" break; - case 142: /* order_by_type: DESC */ -#line 1289 "parser.y" + case 144: /* order_by_type: DESC */ +#line 1294 "parser.y" { (yyval.order_by_type_t) = infinity::kDesc; } -#line 4436 "parser.cpp" +#line 4450 "parser.cpp" break; - case 143: /* order_by_type: %empty */ -#line 1292 "parser.y" + case 145: /* order_by_type: %empty */ +#line 1297 "parser.y" { (yyval.order_by_type_t) = infinity::kAsc; } -#line 4444 "parser.cpp" +#line 4458 "parser.cpp" break; - case 144: /* limit_expr: LIMIT expr */ -#line 1296 "parser.y" + case 146: /* limit_expr: LIMIT expr */ +#line 1301 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4452 "parser.cpp" +#line 4466 "parser.cpp" break; - case 145: /* limit_expr: %empty */ -#line 1300 "parser.y" + case 147: /* limit_expr: %empty */ +#line 1305 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4458 "parser.cpp" +#line 4472 "parser.cpp" break; - case 146: /* offset_expr: OFFSET expr */ -#line 1302 "parser.y" + case 148: /* offset_expr: OFFSET expr */ +#line 1307 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4466 "parser.cpp" +#line 4480 "parser.cpp" break; - case 147: /* offset_expr: %empty */ -#line 1306 "parser.y" + case 149: /* offset_expr: %empty */ +#line 1311 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4472 "parser.cpp" +#line 4486 "parser.cpp" break; - case 148: /* distinct: DISTINCT */ -#line 1308 "parser.y" + case 150: /* distinct: DISTINCT */ +#line 1313 "parser.y" { (yyval.bool_value) = true; } -#line 4480 "parser.cpp" +#line 4494 "parser.cpp" break; - case 149: /* distinct: %empty */ -#line 1311 "parser.y" + case 151: /* distinct: %empty */ +#line 1316 "parser.y" { (yyval.bool_value) = false; } -#line 4488 "parser.cpp" +#line 4502 "parser.cpp" break; - case 150: /* from_clause: FROM table_reference */ -#line 1315 "parser.y" + case 152: /* from_clause: FROM table_reference */ +#line 1320 "parser.y" { (yyval.table_reference_t) = (yyvsp[0].table_reference_t); } -#line 4496 "parser.cpp" +#line 4510 "parser.cpp" break; - case 151: /* from_clause: %empty */ -#line 1318 "parser.y" + case 153: /* from_clause: %empty */ +#line 1323 "parser.y" { (yyval.table_reference_t) = nullptr; } -#line 4504 "parser.cpp" +#line 4518 "parser.cpp" break; - case 152: /* search_clause: SEARCH sub_search_array */ -#line 1322 "parser.y" + case 154: /* search_clause: SEARCH sub_search_array */ +#line 1327 "parser.y" { infinity::SearchExpr* search_expr = new infinity::SearchExpr(); search_expr->SetExprs((yyvsp[0].expr_array_t)); (yyval.expr_t) = search_expr; } -#line 4514 "parser.cpp" +#line 4528 "parser.cpp" break; - case 153: /* search_clause: %empty */ -#line 1327 "parser.y" + case 155: /* search_clause: %empty */ +#line 1332 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4522 "parser.cpp" +#line 4536 "parser.cpp" break; - case 154: /* where_clause: WHERE expr */ -#line 1331 "parser.y" + case 156: /* where_clause: WHERE expr */ +#line 1336 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4530 "parser.cpp" +#line 4544 "parser.cpp" break; - case 155: /* where_clause: %empty */ -#line 1334 "parser.y" + case 157: /* where_clause: %empty */ +#line 1339 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4538 "parser.cpp" +#line 4552 "parser.cpp" break; - case 156: /* having_clause: HAVING expr */ -#line 1338 "parser.y" + case 158: /* having_clause: HAVING expr */ +#line 1343 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 4546 "parser.cpp" +#line 4560 "parser.cpp" break; - case 157: /* having_clause: %empty */ -#line 1341 "parser.y" + case 159: /* having_clause: %empty */ +#line 1346 "parser.y" { (yyval.expr_t) = nullptr; } -#line 4554 "parser.cpp" +#line 4568 "parser.cpp" break; - case 158: /* group_by_clause: GROUP BY expr_array */ -#line 1345 "parser.y" + case 160: /* group_by_clause: GROUP BY expr_array */ +#line 1350 "parser.y" { (yyval.expr_array_t) = (yyvsp[0].expr_array_t); } -#line 4562 "parser.cpp" +#line 4576 "parser.cpp" break; - case 159: /* group_by_clause: %empty */ -#line 1348 "parser.y" + case 161: /* group_by_clause: %empty */ +#line 1353 "parser.y" { (yyval.expr_array_t) = nullptr; } -#line 4570 "parser.cpp" +#line 4584 "parser.cpp" break; - case 160: /* set_operator: UNION */ -#line 1352 "parser.y" + case 162: /* set_operator: UNION */ +#line 1357 "parser.y" { (yyval.set_operator_t) = infinity::SetOperatorType::kUnion; } -#line 4578 "parser.cpp" +#line 4592 "parser.cpp" break; - case 161: /* set_operator: UNION ALL */ -#line 1355 "parser.y" + case 163: /* set_operator: UNION ALL */ +#line 1360 "parser.y" { (yyval.set_operator_t) = infinity::SetOperatorType::kUnionAll; } -#line 4586 "parser.cpp" +#line 4600 "parser.cpp" break; - case 162: /* set_operator: INTERSECT */ -#line 1358 "parser.y" + case 164: /* set_operator: INTERSECT */ +#line 1363 "parser.y" { (yyval.set_operator_t) = infinity::SetOperatorType::kIntersect; } -#line 4594 "parser.cpp" +#line 4608 "parser.cpp" break; - case 163: /* set_operator: EXCEPT */ -#line 1361 "parser.y" + case 165: /* set_operator: EXCEPT */ +#line 1366 "parser.y" { (yyval.set_operator_t) = infinity::SetOperatorType::kExcept; } -#line 4602 "parser.cpp" +#line 4616 "parser.cpp" break; - case 164: /* table_reference: table_reference_unit */ -#line 1369 "parser.y" + case 166: /* table_reference: table_reference_unit */ +#line 1374 "parser.y" { (yyval.table_reference_t) = (yyvsp[0].table_reference_t); } -#line 4610 "parser.cpp" +#line 4624 "parser.cpp" break; - case 165: /* table_reference: table_reference ',' table_reference_unit */ -#line 1372 "parser.y" + case 167: /* table_reference: table_reference ',' table_reference_unit */ +#line 1377 "parser.y" { infinity::CrossProductReference* cross_product_ref = nullptr; if((yyvsp[-2].table_reference_t)->type_ == infinity::TableRefType::kCrossProduct) { @@ -4624,11 +4638,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.table_reference_t) = cross_product_ref; } -#line 4628 "parser.cpp" +#line 4642 "parser.cpp" break; - case 168: /* table_reference_name: table_name table_alias */ -#line 1389 "parser.y" + case 170: /* table_reference_name: table_name table_alias */ +#line 1394 "parser.y" { infinity::TableReference* table_ref = new infinity::TableReference(); if((yyvsp[-1].table_name_t)->schema_name_ptr_ != nullptr) { @@ -4642,32 +4656,32 @@ YYLTYPE yylloc = yyloc_default; table_ref->alias_ = (yyvsp[0].table_alias_t); (yyval.table_reference_t) = table_ref; } -#line 4646 "parser.cpp" +#line 4660 "parser.cpp" break; - case 169: /* table_reference_name: '(' select_statement ')' table_alias */ -#line 1403 "parser.y" + case 171: /* table_reference_name: '(' select_statement ')' table_alias */ +#line 1408 "parser.y" { infinity::SubqueryReference* subquery_reference = new infinity::SubqueryReference(); subquery_reference->select_statement_ = (yyvsp[-2].select_stmt); subquery_reference->alias_ = (yyvsp[0].table_alias_t); (yyval.table_reference_t) = subquery_reference; } -#line 4657 "parser.cpp" +#line 4671 "parser.cpp" break; - case 170: /* table_name: IDENTIFIER */ -#line 1412 "parser.y" + case 172: /* table_name: IDENTIFIER */ +#line 1417 "parser.y" { (yyval.table_name_t) = new infinity::TableName(); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.table_name_t)->table_name_ptr_ = (yyvsp[0].str_value); } -#line 4667 "parser.cpp" +#line 4681 "parser.cpp" break; - case 171: /* table_name: IDENTIFIER '.' IDENTIFIER */ -#line 1417 "parser.y" + case 173: /* table_name: IDENTIFIER '.' IDENTIFIER */ +#line 1422 "parser.y" { (yyval.table_name_t) = new infinity::TableName(); ParserHelper::ToLower((yyvsp[-2].str_value)); @@ -4675,84 +4689,84 @@ 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 4679 "parser.cpp" +#line 4693 "parser.cpp" break; - case 172: /* table_alias: AS IDENTIFIER */ -#line 1426 "parser.y" + case 174: /* table_alias: AS IDENTIFIER */ +#line 1431 "parser.y" { (yyval.table_alias_t) = new infinity::TableAlias(); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.table_alias_t)->alias_ = (yyvsp[0].str_value); } -#line 4689 "parser.cpp" +#line 4703 "parser.cpp" break; - case 173: /* table_alias: IDENTIFIER */ -#line 1431 "parser.y" + case 175: /* table_alias: IDENTIFIER */ +#line 1436 "parser.y" { (yyval.table_alias_t) = new infinity::TableAlias(); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.table_alias_t)->alias_ = (yyvsp[0].str_value); } -#line 4699 "parser.cpp" +#line 4713 "parser.cpp" break; - case 174: /* table_alias: AS IDENTIFIER '(' identifier_array ')' */ -#line 1436 "parser.y" + case 176: /* table_alias: AS IDENTIFIER '(' identifier_array ')' */ +#line 1441 "parser.y" { (yyval.table_alias_t) = new infinity::TableAlias(); ParserHelper::ToLower((yyvsp[-3].str_value)); (yyval.table_alias_t)->alias_ = (yyvsp[-3].str_value); (yyval.table_alias_t)->column_alias_array_ = (yyvsp[-1].identifier_array_t); } -#line 4710 "parser.cpp" +#line 4724 "parser.cpp" break; - case 175: /* table_alias: %empty */ -#line 1442 "parser.y" + case 177: /* table_alias: %empty */ +#line 1447 "parser.y" { (yyval.table_alias_t) = nullptr; } -#line 4718 "parser.cpp" +#line 4732 "parser.cpp" break; - case 176: /* with_clause: WITH with_expr_list */ -#line 1449 "parser.y" + case 178: /* with_clause: WITH with_expr_list */ +#line 1454 "parser.y" { (yyval.with_expr_list_t) = (yyvsp[0].with_expr_list_t); } -#line 4726 "parser.cpp" +#line 4740 "parser.cpp" break; - case 177: /* with_clause: %empty */ -#line 1452 "parser.y" + case 179: /* with_clause: %empty */ +#line 1457 "parser.y" { (yyval.with_expr_list_t) = nullptr; } -#line 4734 "parser.cpp" +#line 4748 "parser.cpp" break; - case 178: /* with_expr_list: with_expr */ -#line 1456 "parser.y" + case 180: /* with_expr_list: with_expr */ +#line 1461 "parser.y" { (yyval.with_expr_list_t) = new std::vector(); (yyval.with_expr_list_t)->emplace_back((yyvsp[0].with_expr_t)); } -#line 4743 "parser.cpp" +#line 4757 "parser.cpp" break; - case 179: /* with_expr_list: with_expr_list ',' with_expr */ -#line 1459 "parser.y" + case 181: /* with_expr_list: with_expr_list ',' with_expr */ +#line 1464 "parser.y" { (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 4752 "parser.cpp" +#line 4766 "parser.cpp" break; - case 180: /* with_expr: IDENTIFIER AS '(' select_clause_with_modifier ')' */ -#line 1464 "parser.y" + case 182: /* with_expr: IDENTIFIER AS '(' select_clause_with_modifier ')' */ +#line 1469 "parser.y" { (yyval.with_expr_t) = new infinity::WithExpr(); ParserHelper::ToLower((yyvsp[-4].str_value)); @@ -4760,11 +4774,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-4].str_value)); (yyval.with_expr_t)->select_ = (yyvsp[-1].select_stmt); } -#line 4764 "parser.cpp" +#line 4778 "parser.cpp" break; - case 181: /* join_clause: table_reference_unit NATURAL JOIN table_reference_name */ -#line 1476 "parser.y" + case 183: /* join_clause: table_reference_unit NATURAL JOIN table_reference_name */ +#line 1481 "parser.y" { infinity::JoinReference* join_reference = new infinity::JoinReference(); join_reference->left_ = (yyvsp[-3].table_reference_t); @@ -4772,11 +4786,11 @@ YYLTYPE yylloc = yyloc_default; join_reference->join_type_ = infinity::JoinType::kNatural; (yyval.table_reference_t) = join_reference; } -#line 4776 "parser.cpp" +#line 4790 "parser.cpp" break; - case 182: /* join_clause: table_reference_unit join_type JOIN table_reference_name ON expr */ -#line 1483 "parser.y" + case 184: /* join_clause: table_reference_unit join_type JOIN table_reference_name ON expr */ +#line 1488 "parser.y" { infinity::JoinReference* join_reference = new infinity::JoinReference(); join_reference->left_ = (yyvsp[-5].table_reference_t); @@ -4785,102 +4799,102 @@ YYLTYPE yylloc = yyloc_default; join_reference->condition_ = (yyvsp[0].expr_t); (yyval.table_reference_t) = join_reference; } -#line 4789 "parser.cpp" +#line 4803 "parser.cpp" break; - case 183: /* join_type: INNER */ -#line 1497 "parser.y" + case 185: /* join_type: INNER */ +#line 1502 "parser.y" { (yyval.join_type_t) = infinity::JoinType::kInner; } -#line 4797 "parser.cpp" +#line 4811 "parser.cpp" break; - case 184: /* join_type: LEFT */ -#line 1500 "parser.y" + case 186: /* join_type: LEFT */ +#line 1505 "parser.y" { (yyval.join_type_t) = infinity::JoinType::kLeft; } -#line 4805 "parser.cpp" +#line 4819 "parser.cpp" break; - case 185: /* join_type: RIGHT */ -#line 1503 "parser.y" + case 187: /* join_type: RIGHT */ +#line 1508 "parser.y" { (yyval.join_type_t) = infinity::JoinType::kRight; } -#line 4813 "parser.cpp" +#line 4827 "parser.cpp" break; - case 186: /* join_type: OUTER */ -#line 1506 "parser.y" + case 188: /* join_type: OUTER */ +#line 1511 "parser.y" { (yyval.join_type_t) = infinity::JoinType::kFull; } -#line 4821 "parser.cpp" +#line 4835 "parser.cpp" break; - case 187: /* join_type: FULL */ -#line 1509 "parser.y" + case 189: /* join_type: FULL */ +#line 1514 "parser.y" { (yyval.join_type_t) = infinity::JoinType::kFull; } -#line 4829 "parser.cpp" +#line 4843 "parser.cpp" break; - case 188: /* join_type: CROSS */ -#line 1512 "parser.y" + case 190: /* join_type: CROSS */ +#line 1517 "parser.y" { (yyval.join_type_t) = infinity::JoinType::kCross; } -#line 4837 "parser.cpp" +#line 4851 "parser.cpp" break; - case 189: /* join_type: %empty */ -#line 1515 "parser.y" + case 191: /* join_type: %empty */ +#line 1520 "parser.y" { } -#line 4844 "parser.cpp" +#line 4858 "parser.cpp" break; - case 190: /* show_statement: SHOW DATABASES */ -#line 1521 "parser.y" + case 192: /* show_statement: SHOW DATABASES */ +#line 1526 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kDatabases; } -#line 4853 "parser.cpp" +#line 4867 "parser.cpp" break; - case 191: /* show_statement: SHOW TABLES */ -#line 1525 "parser.y" + case 193: /* show_statement: SHOW TABLES */ +#line 1530 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kTables; } -#line 4862 "parser.cpp" +#line 4876 "parser.cpp" break; - case 192: /* show_statement: SHOW VIEWS */ -#line 1529 "parser.y" + case 194: /* show_statement: SHOW VIEWS */ +#line 1534 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kViews; } -#line 4871 "parser.cpp" +#line 4885 "parser.cpp" break; - case 193: /* show_statement: SHOW CONFIGS */ -#line 1533 "parser.y" + case 195: /* show_statement: SHOW CONFIGS */ +#line 1538 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kConfigs; } -#line 4880 "parser.cpp" +#line 4894 "parser.cpp" break; - case 194: /* show_statement: SHOW CONFIG IDENTIFIER */ -#line 1537 "parser.y" + case 196: /* show_statement: SHOW CONFIG IDENTIFIER */ +#line 1542 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kConfig; @@ -4888,71 +4902,71 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->var_name_ = std::string((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 4892 "parser.cpp" +#line 4906 "parser.cpp" break; - case 195: /* show_statement: SHOW PROFILES */ -#line 1544 "parser.y" + case 197: /* show_statement: SHOW PROFILES */ +#line 1549 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kProfiles; } -#line 4901 "parser.cpp" +#line 4915 "parser.cpp" break; - case 196: /* show_statement: SHOW SESSION VARIABLES */ -#line 1548 "parser.y" + case 198: /* show_statement: SHOW SESSION VARIABLES */ +#line 1553 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kSessionVariables; } -#line 4910 "parser.cpp" +#line 4924 "parser.cpp" break; - case 197: /* show_statement: SHOW GLOBAL VARIABLES */ -#line 1552 "parser.y" + case 199: /* show_statement: SHOW GLOBAL VARIABLES */ +#line 1557 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kGlobalVariables; } -#line 4919 "parser.cpp" +#line 4933 "parser.cpp" break; - case 198: /* show_statement: SHOW SESSION VARIABLE IDENTIFIER */ -#line 1556 "parser.y" + case 200: /* show_statement: SHOW SESSION VARIABLE IDENTIFIER */ +#line 1561 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kSessionVariable; (yyval.show_stmt)->var_name_ = std::string((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 4930 "parser.cpp" +#line 4944 "parser.cpp" break; - case 199: /* show_statement: SHOW GLOBAL VARIABLE IDENTIFIER */ -#line 1562 "parser.y" + case 201: /* show_statement: SHOW GLOBAL VARIABLE IDENTIFIER */ +#line 1567 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kGlobalVariable; (yyval.show_stmt)->var_name_ = std::string((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 4941 "parser.cpp" +#line 4955 "parser.cpp" break; - case 200: /* show_statement: SHOW DATABASE IDENTIFIER */ -#line 1568 "parser.y" + case 202: /* show_statement: SHOW DATABASE IDENTIFIER */ +#line 1573 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kDatabase; (yyval.show_stmt)->schema_name_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 4952 "parser.cpp" +#line 4966 "parser.cpp" break; - case 201: /* show_statement: SHOW TABLE table_name */ -#line 1574 "parser.y" + case 203: /* show_statement: SHOW TABLE table_name */ +#line 1579 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kTable; @@ -4964,11 +4978,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].table_name_t)->table_name_ptr_); delete (yyvsp[0].table_name_t); } -#line 4968 "parser.cpp" +#line 4982 "parser.cpp" break; - case 202: /* show_statement: SHOW TABLE table_name COLUMNS */ -#line 1585 "parser.y" + case 204: /* show_statement: SHOW TABLE table_name COLUMNS */ +#line 1590 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kColumns; @@ -4980,11 +4994,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].table_name_t)->table_name_ptr_); delete (yyvsp[-1].table_name_t); } -#line 4984 "parser.cpp" +#line 4998 "parser.cpp" break; - case 203: /* show_statement: SHOW TABLE table_name SEGMENTS */ -#line 1596 "parser.y" + case 205: /* show_statement: SHOW TABLE table_name SEGMENTS */ +#line 1601 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kSegments; @@ -4996,11 +5010,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].table_name_t)->table_name_ptr_); delete (yyvsp[-1].table_name_t); } -#line 5000 "parser.cpp" +#line 5014 "parser.cpp" break; - case 204: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE */ -#line 1607 "parser.y" + case 206: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE */ +#line 1612 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kSegment; @@ -5013,11 +5027,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->segment_id_ = (yyvsp[0].long_value); delete (yyvsp[-2].table_name_t); } -#line 5017 "parser.cpp" +#line 5031 "parser.cpp" break; - case 205: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCKS */ -#line 1619 "parser.y" + case 207: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCKS */ +#line 1624 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kBlocks; @@ -5030,11 +5044,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->segment_id_ = (yyvsp[-1].long_value); delete (yyvsp[-3].table_name_t); } -#line 5034 "parser.cpp" +#line 5048 "parser.cpp" break; - case 206: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCK LONG_VALUE */ -#line 1631 "parser.y" + case 208: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCK LONG_VALUE */ +#line 1636 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kBlock; @@ -5048,11 +5062,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->block_id_ = (yyvsp[0].long_value); delete (yyvsp[-4].table_name_t); } -#line 5052 "parser.cpp" +#line 5066 "parser.cpp" break; - case 207: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCK LONG_VALUE COLUMN LONG_VALUE */ -#line 1644 "parser.y" + case 209: /* show_statement: SHOW TABLE table_name SEGMENT LONG_VALUE BLOCK LONG_VALUE COLUMN LONG_VALUE */ +#line 1649 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kBlockColumn; @@ -5067,11 +5081,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->column_id_ = (yyvsp[0].long_value); delete (yyvsp[-6].table_name_t); } -#line 5071 "parser.cpp" +#line 5085 "parser.cpp" break; - case 208: /* show_statement: SHOW TABLE table_name INDEXES */ -#line 1658 "parser.y" + case 210: /* show_statement: SHOW TABLE table_name INDEXES */ +#line 1663 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kIndexes; @@ -5083,11 +5097,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].table_name_t)->table_name_ptr_); delete (yyvsp[-1].table_name_t); } -#line 5087 "parser.cpp" +#line 5101 "parser.cpp" break; - case 209: /* show_statement: SHOW TABLE table_name INDEX IDENTIFIER */ -#line 1669 "parser.y" + case 211: /* show_statement: SHOW TABLE table_name INDEX IDENTIFIER */ +#line 1674 "parser.y" { (yyval.show_stmt) = new infinity::ShowStatement(); (yyval.show_stmt)->show_type_ = infinity::ShowStmtType::kIndex; @@ -5102,38 +5116,38 @@ YYLTYPE yylloc = yyloc_default; (yyval.show_stmt)->index_name_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 5106 "parser.cpp" +#line 5120 "parser.cpp" break; - case 210: /* flush_statement: FLUSH DATA */ -#line 1687 "parser.y" + case 212: /* flush_statement: FLUSH DATA */ +#line 1692 "parser.y" { (yyval.flush_stmt) = new infinity::FlushStatement(); (yyval.flush_stmt)->type_ = infinity::FlushType::kData; } -#line 5115 "parser.cpp" +#line 5129 "parser.cpp" break; - case 211: /* flush_statement: FLUSH LOG */ -#line 1691 "parser.y" + case 213: /* flush_statement: FLUSH LOG */ +#line 1696 "parser.y" { (yyval.flush_stmt) = new infinity::FlushStatement(); (yyval.flush_stmt)->type_ = infinity::FlushType::kLog; } -#line 5124 "parser.cpp" +#line 5138 "parser.cpp" break; - case 212: /* flush_statement: FLUSH BUFFER */ -#line 1695 "parser.y" + case 214: /* flush_statement: FLUSH BUFFER */ +#line 1700 "parser.y" { (yyval.flush_stmt) = new infinity::FlushStatement(); (yyval.flush_stmt)->type_ = infinity::FlushType::kBuffer; } -#line 5133 "parser.cpp" +#line 5147 "parser.cpp" break; - case 213: /* optimize_statement: OPTIMIZE table_name */ -#line 1703 "parser.y" + case 215: /* optimize_statement: OPTIMIZE table_name */ +#line 1708 "parser.y" { (yyval.optimize_stmt) = new infinity::OptimizeStatement(); if((yyvsp[0].table_name_t)->schema_name_ptr_ != nullptr) { @@ -5144,54 +5158,54 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].table_name_t)->table_name_ptr_); delete (yyvsp[0].table_name_t); } -#line 5148 "parser.cpp" +#line 5162 "parser.cpp" break; - case 214: /* command_statement: USE IDENTIFIER */ -#line 1717 "parser.y" + case 216: /* command_statement: USE IDENTIFIER */ +#line 1722 "parser.y" { (yyval.command_stmt) = new infinity::CommandStatement(); ParserHelper::ToLower((yyvsp[0].str_value)); (yyval.command_stmt)->command_info_ = std::make_unique((yyvsp[0].str_value)); free((yyvsp[0].str_value)); } -#line 5159 "parser.cpp" +#line 5173 "parser.cpp" break; - case 215: /* command_statement: EXPORT PROFILE LONG_VALUE file_path */ -#line 1723 "parser.y" + case 217: /* command_statement: EXPORT PROFILE LONG_VALUE file_path */ +#line 1728 "parser.y" { (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique((yyvsp[0].str_value), infinity::ExportType::kProfileRecord, (yyvsp[-1].long_value)); free((yyvsp[0].str_value)); } -#line 5169 "parser.cpp" +#line 5183 "parser.cpp" break; - case 216: /* command_statement: SET SESSION IDENTIFIER ON */ -#line 1728 "parser.y" + case 218: /* command_statement: SET SESSION IDENTIFIER ON */ +#line 1733 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kSession, infinity::SetVarType::kBool, (yyvsp[-1].str_value), true); free((yyvsp[-1].str_value)); } -#line 5180 "parser.cpp" +#line 5194 "parser.cpp" break; - case 217: /* command_statement: SET SESSION IDENTIFIER OFF */ -#line 1734 "parser.y" + case 219: /* command_statement: SET SESSION IDENTIFIER OFF */ +#line 1739 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kSession, infinity::SetVarType::kBool, (yyvsp[-1].str_value), false); free((yyvsp[-1].str_value)); } -#line 5191 "parser.cpp" +#line 5205 "parser.cpp" break; - case 218: /* command_statement: SET SESSION IDENTIFIER IDENTIFIER */ -#line 1740 "parser.y" + case 220: /* command_statement: SET SESSION IDENTIFIER IDENTIFIER */ +#line 1745 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -5200,55 +5214,55 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); free((yyvsp[0].str_value)); } -#line 5204 "parser.cpp" +#line 5218 "parser.cpp" break; - case 219: /* command_statement: SET SESSION IDENTIFIER LONG_VALUE */ -#line 1748 "parser.y" + case 221: /* command_statement: SET SESSION IDENTIFIER LONG_VALUE */ +#line 1753 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kSession, infinity::SetVarType::kInteger, (yyvsp[-1].str_value), (yyvsp[0].long_value)); free((yyvsp[-1].str_value)); } -#line 5215 "parser.cpp" +#line 5229 "parser.cpp" break; - case 220: /* command_statement: SET SESSION IDENTIFIER DOUBLE_VALUE */ -#line 1754 "parser.y" + case 222: /* command_statement: SET SESSION IDENTIFIER DOUBLE_VALUE */ +#line 1759 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kSession, infinity::SetVarType::kDouble, (yyvsp[-1].str_value), (yyvsp[0].double_value)); free((yyvsp[-1].str_value)); } -#line 5226 "parser.cpp" +#line 5240 "parser.cpp" break; - case 221: /* command_statement: SET GLOBAL IDENTIFIER ON */ -#line 1760 "parser.y" + case 223: /* command_statement: SET GLOBAL IDENTIFIER ON */ +#line 1765 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kGlobal, infinity::SetVarType::kBool, (yyvsp[-1].str_value), true); free((yyvsp[-1].str_value)); } -#line 5237 "parser.cpp" +#line 5251 "parser.cpp" break; - case 222: /* command_statement: SET GLOBAL IDENTIFIER OFF */ -#line 1766 "parser.y" + case 224: /* command_statement: SET GLOBAL IDENTIFIER OFF */ +#line 1771 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kGlobal, infinity::SetVarType::kBool, (yyvsp[-1].str_value), false); free((yyvsp[-1].str_value)); } -#line 5248 "parser.cpp" +#line 5262 "parser.cpp" break; - case 223: /* command_statement: SET GLOBAL IDENTIFIER IDENTIFIER */ -#line 1772 "parser.y" + case 225: /* command_statement: SET GLOBAL IDENTIFIER IDENTIFIER */ +#line 1777 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -5257,55 +5271,55 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); free((yyvsp[0].str_value)); } -#line 5261 "parser.cpp" +#line 5275 "parser.cpp" break; - case 224: /* command_statement: SET GLOBAL IDENTIFIER LONG_VALUE */ -#line 1780 "parser.y" + case 226: /* command_statement: SET GLOBAL IDENTIFIER LONG_VALUE */ +#line 1785 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kGlobal, infinity::SetVarType::kInteger, (yyvsp[-1].str_value), (yyvsp[0].long_value)); free((yyvsp[-1].str_value)); } -#line 5272 "parser.cpp" +#line 5286 "parser.cpp" break; - case 225: /* command_statement: SET GLOBAL IDENTIFIER DOUBLE_VALUE */ -#line 1786 "parser.y" + case 227: /* command_statement: SET GLOBAL IDENTIFIER DOUBLE_VALUE */ +#line 1791 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kGlobal, infinity::SetVarType::kDouble, (yyvsp[-1].str_value), (yyvsp[0].double_value)); free((yyvsp[-1].str_value)); } -#line 5283 "parser.cpp" +#line 5297 "parser.cpp" break; - case 226: /* command_statement: SET CONFIG IDENTIFIER ON */ -#line 1792 "parser.y" + case 228: /* command_statement: SET CONFIG IDENTIFIER ON */ +#line 1797 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kConfig, infinity::SetVarType::kBool, (yyvsp[-1].str_value), true); free((yyvsp[-1].str_value)); } -#line 5294 "parser.cpp" +#line 5308 "parser.cpp" break; - case 227: /* command_statement: SET CONFIG IDENTIFIER OFF */ -#line 1798 "parser.y" + case 229: /* command_statement: SET CONFIG IDENTIFIER OFF */ +#line 1803 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kConfig, infinity::SetVarType::kBool, (yyvsp[-1].str_value), false); free((yyvsp[-1].str_value)); } -#line 5305 "parser.cpp" +#line 5319 "parser.cpp" break; - case 228: /* command_statement: SET CONFIG IDENTIFIER IDENTIFIER */ -#line 1804 "parser.y" + case 230: /* command_statement: SET CONFIG IDENTIFIER IDENTIFIER */ +#line 1809 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -5314,76 +5328,77 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); free((yyvsp[0].str_value)); } -#line 5318 "parser.cpp" +#line 5332 "parser.cpp" break; - case 229: /* command_statement: SET CONFIG IDENTIFIER LONG_VALUE */ -#line 1812 "parser.y" + case 231: /* command_statement: SET CONFIG IDENTIFIER LONG_VALUE */ +#line 1817 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kConfig, infinity::SetVarType::kInteger, (yyvsp[-1].str_value), (yyvsp[0].long_value)); free((yyvsp[-1].str_value)); } -#line 5329 "parser.cpp" +#line 5343 "parser.cpp" break; - case 230: /* command_statement: SET CONFIG IDENTIFIER DOUBLE_VALUE */ -#line 1818 "parser.y" + case 232: /* command_statement: SET CONFIG IDENTIFIER DOUBLE_VALUE */ +#line 1823 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); (yyval.command_stmt) = new infinity::CommandStatement(); (yyval.command_stmt)->command_info_ = std::make_unique(infinity::SetScope::kConfig, infinity::SetVarType::kDouble, (yyvsp[-1].str_value), (yyvsp[0].double_value)); free((yyvsp[-1].str_value)); } -#line 5340 "parser.cpp" +#line 5354 "parser.cpp" break; - case 231: /* command_statement: COMPACT TABLE table_name */ -#line 1824 "parser.y" - { - (yyval.command_stmt) = new infinity::CommandStatement(); + case 233: /* compact_statement: COMPACT TABLE table_name */ +#line 1830 "parser.y" + { + std::string schema_name; if ((yyvsp[0].table_name_t)->schema_name_ptr_ != nullptr) { - (yyval.command_stmt)->command_info_ = std::make_unique(std::string((yyvsp[0].table_name_t)->schema_name_ptr_), std::string((yyvsp[0].table_name_t)->table_name_ptr_)); + schema_name = std::string((yyvsp[0].table_name_t)->schema_name_ptr_); free((yyvsp[0].table_name_t)->schema_name_ptr_); - free((yyvsp[0].table_name_t)->table_name_ptr_); - } else { - (yyval.command_stmt)->command_info_ = std::make_unique(std::string((yyvsp[0].table_name_t)->table_name_ptr_)); - free((yyvsp[0].table_name_t)->table_name_ptr_); - } delete (yyvsp[0].table_name_t); + } + std::string table_name = std::string((yyvsp[0].table_name_t)->table_name_ptr_); + free((yyvsp[0].table_name_t)->table_name_ptr_); + + (yyval.compact_stmt) = new infinity::ManualCompactStatement(std::move(schema_name), std::move(table_name)); + delete (yyvsp[0].table_name_t); } -#line 5356 "parser.cpp" +#line 5371 "parser.cpp" break; - case 232: /* expr_array: expr_alias */ -#line 1840 "parser.y" + case 234: /* expr_array: expr_alias */ +#line 1847 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 5365 "parser.cpp" +#line 5380 "parser.cpp" break; - case 233: /* expr_array: expr_array ',' expr_alias */ -#line 1844 "parser.y" + case 235: /* expr_array: expr_array ',' expr_alias */ +#line 1851 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 5374 "parser.cpp" +#line 5389 "parser.cpp" break; - case 234: /* expr_array_list: '(' expr_array ')' */ -#line 1849 "parser.y" + case 236: /* expr_array_list: '(' expr_array ')' */ +#line 1856 "parser.y" { (yyval.expr_array_list_t) = new std::vector*>(); (yyval.expr_array_list_t)->push_back((yyvsp[-1].expr_array_t)); } -#line 5383 "parser.cpp" +#line 5398 "parser.cpp" break; - case 235: /* expr_array_list: expr_array_list ',' '(' expr_array ')' */ -#line 1853 "parser.y" + case 237: /* expr_array_list: expr_array_list ',' '(' expr_array ')' */ +#line 1860 "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."); @@ -5399,57 +5414,57 @@ 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 5403 "parser.cpp" +#line 5418 "parser.cpp" break; - case 236: /* expr_alias: expr AS IDENTIFIER */ -#line 1880 "parser.y" + case 238: /* expr_alias: expr AS IDENTIFIER */ +#line 1887 "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 5414 "parser.cpp" +#line 5429 "parser.cpp" break; - case 237: /* expr_alias: expr */ -#line 1886 "parser.y" + case 239: /* expr_alias: expr */ +#line 1893 "parser.y" { (yyval.expr_t) = (yyvsp[0].expr_t); } -#line 5422 "parser.cpp" +#line 5437 "parser.cpp" break; - case 243: /* operand: '(' expr ')' */ -#line 1896 "parser.y" + case 245: /* operand: '(' expr ')' */ +#line 1903 "parser.y" { (yyval.expr_t) = (yyvsp[-1].expr_t); } -#line 5430 "parser.cpp" +#line 5445 "parser.cpp" break; - case 244: /* operand: '(' select_without_paren ')' */ -#line 1899 "parser.y" + case 246: /* operand: '(' select_without_paren ')' */ +#line 1906 "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 5441 "parser.cpp" +#line 5456 "parser.cpp" break; - case 245: /* operand: constant_expr */ -#line 1905 "parser.y" + case 247: /* operand: constant_expr */ +#line 1912 "parser.y" { (yyval.expr_t) = (yyvsp[0].const_expr_t); } -#line 5449 "parser.cpp" +#line 5464 "parser.cpp" break; - case 254: /* knn_expr: KNN '(' expr ',' array_expr ',' STRING ',' STRING ',' LONG_VALUE ')' with_index_param_list */ -#line 1917 "parser.y" + case 256: /* knn_expr: KNN '(' expr ',' array_expr ',' STRING ',' STRING ',' LONG_VALUE ')' with_index_param_list */ +#line 1924 "parser.y" { infinity::KnnExpr* knn_expr = new infinity::KnnExpr(); (yyval.expr_t) = knn_expr; @@ -5616,11 +5631,11 @@ YYLTYPE yylloc = yyloc_default; knn_expr->topn_ = (yyvsp[-2].long_value); knn_expr->opt_params_ = (yyvsp[0].with_index_param_list_t); } -#line 5620 "parser.cpp" +#line 5635 "parser.cpp" break; - case 255: /* match_expr: MATCH '(' STRING ',' STRING ')' */ -#line 2084 "parser.y" + case 257: /* match_expr: MATCH '(' STRING ',' STRING ')' */ +#line 2091 "parser.y" { infinity::MatchExpr* match_expr = new infinity::MatchExpr(); match_expr->fields_ = std::string((yyvsp[-3].str_value)); @@ -5629,11 +5644,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = match_expr; } -#line 5633 "parser.cpp" +#line 5648 "parser.cpp" break; - case 256: /* match_expr: MATCH '(' STRING ',' STRING ',' STRING ')' */ -#line 2092 "parser.y" + case 258: /* match_expr: MATCH '(' STRING ',' STRING ',' STRING ')' */ +#line 2099 "parser.y" { infinity::MatchExpr* match_expr = new infinity::MatchExpr(); match_expr->fields_ = std::string((yyvsp[-5].str_value)); @@ -5644,22 +5659,22 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = match_expr; } -#line 5648 "parser.cpp" +#line 5663 "parser.cpp" break; - case 257: /* query_expr: QUERY '(' STRING ')' */ -#line 2103 "parser.y" + case 259: /* query_expr: QUERY '(' STRING ')' */ +#line 2110 "parser.y" { infinity::MatchExpr* match_expr = new infinity::MatchExpr(); match_expr->matching_text_ = std::string((yyvsp[-1].str_value)); free((yyvsp[-1].str_value)); (yyval.expr_t) = match_expr; } -#line 5659 "parser.cpp" +#line 5674 "parser.cpp" break; - case 258: /* query_expr: QUERY '(' STRING ',' STRING ')' */ -#line 2109 "parser.y" + case 260: /* query_expr: QUERY '(' STRING ',' STRING ')' */ +#line 2116 "parser.y" { infinity::MatchExpr* match_expr = new infinity::MatchExpr(); match_expr->matching_text_ = std::string((yyvsp[-3].str_value)); @@ -5668,22 +5683,22 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = match_expr; } -#line 5672 "parser.cpp" +#line 5687 "parser.cpp" break; - case 259: /* fusion_expr: FUSION '(' STRING ')' */ -#line 2118 "parser.y" + case 261: /* fusion_expr: FUSION '(' STRING ')' */ +#line 2125 "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 5683 "parser.cpp" +#line 5698 "parser.cpp" break; - case 260: /* fusion_expr: FUSION '(' STRING ',' STRING ')' */ -#line 2124 "parser.y" + case 262: /* fusion_expr: FUSION '(' STRING ',' STRING ')' */ +#line 2131 "parser.y" { infinity::FusionExpr* fusion_expr = new infinity::FusionExpr(); fusion_expr->method_ = std::string((yyvsp[-3].str_value)); @@ -5692,83 +5707,83 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[-1].str_value)); (yyval.expr_t) = fusion_expr; } -#line 5696 "parser.cpp" +#line 5711 "parser.cpp" break; - case 261: /* sub_search_array: knn_expr */ -#line 2134 "parser.y" + case 263: /* sub_search_array: knn_expr */ +#line 2141 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 5705 "parser.cpp" +#line 5720 "parser.cpp" break; - case 262: /* sub_search_array: match_expr */ -#line 2138 "parser.y" + case 264: /* sub_search_array: match_expr */ +#line 2145 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 5714 "parser.cpp" +#line 5729 "parser.cpp" break; - case 263: /* sub_search_array: query_expr */ -#line 2142 "parser.y" + case 265: /* sub_search_array: query_expr */ +#line 2149 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 5723 "parser.cpp" +#line 5738 "parser.cpp" break; - case 264: /* sub_search_array: fusion_expr */ -#line 2146 "parser.y" + case 266: /* sub_search_array: fusion_expr */ +#line 2153 "parser.y" { (yyval.expr_array_t) = new std::vector(); (yyval.expr_array_t)->emplace_back((yyvsp[0].expr_t)); } -#line 5732 "parser.cpp" +#line 5747 "parser.cpp" break; - case 265: /* sub_search_array: sub_search_array ',' knn_expr */ -#line 2150 "parser.y" + case 267: /* sub_search_array: sub_search_array ',' knn_expr */ +#line 2157 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 5741 "parser.cpp" +#line 5756 "parser.cpp" break; - case 266: /* sub_search_array: sub_search_array ',' match_expr */ -#line 2154 "parser.y" + case 268: /* sub_search_array: sub_search_array ',' match_expr */ +#line 2161 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 5750 "parser.cpp" +#line 5765 "parser.cpp" break; - case 267: /* sub_search_array: sub_search_array ',' query_expr */ -#line 2158 "parser.y" + case 269: /* sub_search_array: sub_search_array ',' query_expr */ +#line 2165 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 5759 "parser.cpp" +#line 5774 "parser.cpp" break; - case 268: /* sub_search_array: sub_search_array ',' fusion_expr */ -#line 2162 "parser.y" + case 270: /* sub_search_array: sub_search_array ',' fusion_expr */ +#line 2169 "parser.y" { (yyvsp[-2].expr_array_t)->emplace_back((yyvsp[0].expr_t)); (yyval.expr_array_t) = (yyvsp[-2].expr_array_t); } -#line 5768 "parser.cpp" +#line 5783 "parser.cpp" break; - case 269: /* function_expr: IDENTIFIER '(' ')' */ -#line 2167 "parser.y" + case 271: /* function_expr: IDENTIFIER '(' ')' */ +#line 2174 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-2].str_value)); @@ -5777,11 +5792,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_ = nullptr; (yyval.expr_t) = func_expr; } -#line 5781 "parser.cpp" +#line 5796 "parser.cpp" break; - case 270: /* function_expr: IDENTIFIER '(' expr_array ')' */ -#line 2175 "parser.y" + case 272: /* function_expr: IDENTIFIER '(' expr_array ')' */ +#line 2182 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-3].str_value)); @@ -5790,11 +5805,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_ = (yyvsp[-1].expr_array_t); (yyval.expr_t) = func_expr; } -#line 5794 "parser.cpp" +#line 5809 "parser.cpp" break; - case 271: /* function_expr: IDENTIFIER '(' DISTINCT expr_array ')' */ -#line 2183 "parser.y" + case 273: /* function_expr: IDENTIFIER '(' DISTINCT expr_array ')' */ +#line 2190 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-4].str_value)); @@ -5804,11 +5819,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->distinct_ = true; (yyval.expr_t) = func_expr; } -#line 5808 "parser.cpp" +#line 5823 "parser.cpp" break; - case 272: /* function_expr: operand IS NOT NULLABLE */ -#line 2192 "parser.y" + case 274: /* function_expr: operand IS NOT NULLABLE */ +#line 2199 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "is_not_null"; @@ -5816,11 +5831,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[-3].expr_t)); (yyval.expr_t) = func_expr; } -#line 5820 "parser.cpp" +#line 5835 "parser.cpp" break; - case 273: /* function_expr: operand IS NULLABLE */ -#line 2199 "parser.y" + case 275: /* function_expr: operand IS NULLABLE */ +#line 2206 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "is_null"; @@ -5828,11 +5843,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[-2].expr_t)); (yyval.expr_t) = func_expr; } -#line 5832 "parser.cpp" +#line 5847 "parser.cpp" break; - case 274: /* function_expr: NOT operand */ -#line 2206 "parser.y" + case 276: /* function_expr: NOT operand */ +#line 2213 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "not"; @@ -5840,11 +5855,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5844 "parser.cpp" +#line 5859 "parser.cpp" break; - case 275: /* function_expr: '-' operand */ -#line 2213 "parser.y" + case 277: /* function_expr: '-' operand */ +#line 2220 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "-"; @@ -5852,11 +5867,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5856 "parser.cpp" +#line 5871 "parser.cpp" break; - case 276: /* function_expr: '+' operand */ -#line 2220 "parser.y" + case 278: /* function_expr: '+' operand */ +#line 2227 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "+"; @@ -5864,11 +5879,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5868 "parser.cpp" +#line 5883 "parser.cpp" break; - case 277: /* function_expr: operand '-' operand */ -#line 2227 "parser.y" + case 279: /* function_expr: operand '-' operand */ +#line 2234 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "-"; @@ -5877,11 +5892,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5881 "parser.cpp" +#line 5896 "parser.cpp" break; - case 278: /* function_expr: operand '+' operand */ -#line 2235 "parser.y" + case 280: /* function_expr: operand '+' operand */ +#line 2242 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "+"; @@ -5890,11 +5905,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5894 "parser.cpp" +#line 5909 "parser.cpp" break; - case 279: /* function_expr: operand '*' operand */ -#line 2243 "parser.y" + case 281: /* function_expr: operand '*' operand */ +#line 2250 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "*"; @@ -5903,11 +5918,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5907 "parser.cpp" +#line 5922 "parser.cpp" break; - case 280: /* function_expr: operand '/' operand */ -#line 2251 "parser.y" + case 282: /* function_expr: operand '/' operand */ +#line 2258 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "/"; @@ -5916,11 +5931,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5920 "parser.cpp" +#line 5935 "parser.cpp" break; - case 281: /* function_expr: operand '%' operand */ -#line 2259 "parser.y" + case 283: /* function_expr: operand '%' operand */ +#line 2266 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "%"; @@ -5929,11 +5944,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5933 "parser.cpp" +#line 5948 "parser.cpp" break; - case 282: /* function_expr: operand '=' operand */ -#line 2267 "parser.y" + case 284: /* function_expr: operand '=' operand */ +#line 2274 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "="; @@ -5942,11 +5957,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5946 "parser.cpp" +#line 5961 "parser.cpp" break; - case 283: /* function_expr: operand EQUAL operand */ -#line 2275 "parser.y" + case 285: /* function_expr: operand EQUAL operand */ +#line 2282 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "="; @@ -5955,11 +5970,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5959 "parser.cpp" +#line 5974 "parser.cpp" break; - case 284: /* function_expr: operand NOT_EQ operand */ -#line 2283 "parser.y" + case 286: /* function_expr: operand NOT_EQ operand */ +#line 2290 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "<>"; @@ -5968,11 +5983,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5972 "parser.cpp" +#line 5987 "parser.cpp" break; - case 285: /* function_expr: operand '<' operand */ -#line 2291 "parser.y" + case 287: /* function_expr: operand '<' operand */ +#line 2298 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "<"; @@ -5981,11 +5996,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5985 "parser.cpp" +#line 6000 "parser.cpp" break; - case 286: /* function_expr: operand '>' operand */ -#line 2299 "parser.y" + case 288: /* function_expr: operand '>' operand */ +#line 2306 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = ">"; @@ -5994,11 +6009,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 5998 "parser.cpp" +#line 6013 "parser.cpp" break; - case 287: /* function_expr: operand LESS_EQ operand */ -#line 2307 "parser.y" + case 289: /* function_expr: operand LESS_EQ operand */ +#line 2314 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "<="; @@ -6007,11 +6022,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6011 "parser.cpp" +#line 6026 "parser.cpp" break; - case 288: /* function_expr: operand GREATER_EQ operand */ -#line 2315 "parser.y" + case 290: /* function_expr: operand GREATER_EQ operand */ +#line 2322 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = ">="; @@ -6020,11 +6035,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6024 "parser.cpp" +#line 6039 "parser.cpp" break; - case 289: /* function_expr: EXTRACT '(' STRING FROM operand ')' */ -#line 2323 "parser.y" + case 291: /* function_expr: EXTRACT '(' STRING FROM operand ')' */ +#line 2330 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); ParserHelper::ToLower((yyvsp[-3].str_value)); @@ -6055,11 +6070,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[-1].expr_t)); (yyval.expr_t) = func_expr; } -#line 6059 "parser.cpp" +#line 6074 "parser.cpp" break; - case 290: /* function_expr: operand LIKE operand */ -#line 2353 "parser.y" + case 292: /* function_expr: operand LIKE operand */ +#line 2360 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "like"; @@ -6068,11 +6083,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6072 "parser.cpp" +#line 6087 "parser.cpp" break; - case 291: /* function_expr: operand NOT LIKE operand */ -#line 2361 "parser.y" + case 293: /* function_expr: operand NOT LIKE operand */ +#line 2368 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "not_like"; @@ -6081,11 +6096,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6085 "parser.cpp" +#line 6100 "parser.cpp" break; - case 292: /* conjunction_expr: expr AND expr */ -#line 2370 "parser.y" + case 294: /* conjunction_expr: expr AND expr */ +#line 2377 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "and"; @@ -6094,11 +6109,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6098 "parser.cpp" +#line 6113 "parser.cpp" break; - case 293: /* conjunction_expr: expr OR expr */ -#line 2378 "parser.y" + case 295: /* conjunction_expr: expr OR expr */ +#line 2385 "parser.y" { infinity::FunctionExpr* func_expr = new infinity::FunctionExpr(); func_expr->func_name_ = "or"; @@ -6107,11 +6122,11 @@ YYLTYPE yylloc = yyloc_default; func_expr->arguments_->emplace_back((yyvsp[0].expr_t)); (yyval.expr_t) = func_expr; } -#line 6111 "parser.cpp" +#line 6126 "parser.cpp" break; - case 294: /* between_expr: operand BETWEEN operand AND operand */ -#line 2387 "parser.y" + case 296: /* between_expr: operand BETWEEN operand AND operand */ +#line 2394 "parser.y" { infinity::BetweenExpr* between_expr = new infinity::BetweenExpr(); between_expr->value_ = (yyvsp[-4].expr_t); @@ -6119,44 +6134,44 @@ YYLTYPE yylloc = yyloc_default; between_expr->upper_bound_ = (yyvsp[0].expr_t); (yyval.expr_t) = between_expr; } -#line 6123 "parser.cpp" +#line 6138 "parser.cpp" break; - case 295: /* in_expr: operand IN '(' expr_array ')' */ -#line 2395 "parser.y" + case 297: /* in_expr: operand IN '(' expr_array ')' */ +#line 2402 "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 6134 "parser.cpp" +#line 6149 "parser.cpp" break; - case 296: /* in_expr: operand NOT IN '(' expr_array ')' */ -#line 2401 "parser.y" + case 298: /* in_expr: operand NOT IN '(' expr_array ')' */ +#line 2408 "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 6145 "parser.cpp" +#line 6160 "parser.cpp" break; - case 297: /* case_expr: CASE expr case_check_array END */ -#line 2408 "parser.y" + case 299: /* case_expr: CASE expr case_check_array END */ +#line 2415 "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 6156 "parser.cpp" +#line 6171 "parser.cpp" break; - case 298: /* case_expr: CASE expr case_check_array ELSE expr END */ -#line 2414 "parser.y" + case 300: /* case_expr: CASE expr case_check_array ELSE expr END */ +#line 2421 "parser.y" { infinity::CaseExpr* case_expr = new infinity::CaseExpr(); case_expr->expr_ = (yyvsp[-4].expr_t); @@ -6164,32 +6179,32 @@ YYLTYPE yylloc = yyloc_default; case_expr->else_expr_ = (yyvsp[-1].expr_t); (yyval.expr_t) = case_expr; } -#line 6168 "parser.cpp" +#line 6183 "parser.cpp" break; - case 299: /* case_expr: CASE case_check_array END */ -#line 2421 "parser.y" + case 301: /* case_expr: CASE case_check_array END */ +#line 2428 "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 6178 "parser.cpp" +#line 6193 "parser.cpp" break; - case 300: /* case_expr: CASE case_check_array ELSE expr END */ -#line 2426 "parser.y" + case 302: /* case_expr: CASE case_check_array ELSE expr END */ +#line 2433 "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 6189 "parser.cpp" +#line 6204 "parser.cpp" break; - case 301: /* case_check_array: WHEN expr THEN expr */ -#line 2433 "parser.y" + case 303: /* case_check_array: WHEN expr THEN expr */ +#line 2440 "parser.y" { (yyval.case_check_array_t) = new std::vector(); infinity::WhenThen* when_then_ptr = new infinity::WhenThen(); @@ -6197,11 +6212,11 @@ YYLTYPE yylloc = yyloc_default; when_then_ptr->then_ = (yyvsp[0].expr_t); (yyval.case_check_array_t)->emplace_back(when_then_ptr); } -#line 6201 "parser.cpp" +#line 6216 "parser.cpp" break; - case 302: /* case_check_array: case_check_array WHEN expr THEN expr */ -#line 2440 "parser.y" + case 304: /* case_check_array: case_check_array WHEN expr THEN expr */ +#line 2447 "parser.y" { infinity::WhenThen* when_then_ptr = new infinity::WhenThen(); when_then_ptr->when_ = (yyvsp[-2].expr_t); @@ -6209,11 +6224,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 6213 "parser.cpp" +#line 6228 "parser.cpp" break; - case 303: /* cast_expr: CAST '(' expr AS column_type ')' */ -#line 2448 "parser.y" + case 305: /* cast_expr: CAST '(' expr AS column_type ')' */ +#line 2455 "parser.y" { std::shared_ptr type_info_ptr{nullptr}; switch((yyvsp[-1].column_type_t).logical_type_) { @@ -6237,33 +6252,33 @@ YYLTYPE yylloc = yyloc_default; cast_expr->expr_ = (yyvsp[-3].expr_t); (yyval.expr_t) = cast_expr; } -#line 6241 "parser.cpp" +#line 6256 "parser.cpp" break; - case 304: /* subquery_expr: EXISTS '(' select_without_paren ')' */ -#line 2472 "parser.y" + case 306: /* subquery_expr: EXISTS '(' select_without_paren ')' */ +#line 2479 "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 6252 "parser.cpp" +#line 6267 "parser.cpp" break; - case 305: /* subquery_expr: NOT EXISTS '(' select_without_paren ')' */ -#line 2478 "parser.y" + case 307: /* subquery_expr: NOT EXISTS '(' select_without_paren ')' */ +#line 2485 "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 6263 "parser.cpp" +#line 6278 "parser.cpp" break; - case 306: /* subquery_expr: operand IN '(' select_without_paren ')' */ -#line 2484 "parser.y" + case 308: /* subquery_expr: operand IN '(' select_without_paren ')' */ +#line 2491 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kIn; @@ -6271,11 +6286,11 @@ YYLTYPE yylloc = yyloc_default; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 6275 "parser.cpp" +#line 6290 "parser.cpp" break; - case 307: /* subquery_expr: operand NOT IN '(' select_without_paren ')' */ -#line 2491 "parser.y" + case 309: /* subquery_expr: operand NOT IN '(' select_without_paren ')' */ +#line 2498 "parser.y" { infinity::SubqueryExpr* subquery_expr = new infinity::SubqueryExpr(); subquery_expr->subquery_type_ = infinity::SubqueryType::kNotIn; @@ -6283,11 +6298,11 @@ YYLTYPE yylloc = yyloc_default; subquery_expr->select_ = (yyvsp[-1].select_stmt); (yyval.expr_t) = subquery_expr; } -#line 6287 "parser.cpp" +#line 6302 "parser.cpp" break; - case 308: /* column_expr: IDENTIFIER */ -#line 2499 "parser.y" + case 310: /* column_expr: IDENTIFIER */ +#line 2506 "parser.y" { infinity::ColumnExpr* column_expr = new infinity::ColumnExpr(); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -6295,11 +6310,11 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].str_value)); (yyval.expr_t) = column_expr; } -#line 6299 "parser.cpp" +#line 6314 "parser.cpp" break; - case 309: /* column_expr: column_expr '.' IDENTIFIER */ -#line 2506 "parser.y" + case 311: /* column_expr: column_expr '.' IDENTIFIER */ +#line 2513 "parser.y" { infinity::ColumnExpr* column_expr = (infinity::ColumnExpr*)(yyvsp[-2].expr_t); ParserHelper::ToLower((yyvsp[0].str_value)); @@ -6307,21 +6322,21 @@ YYLTYPE yylloc = yyloc_default; free((yyvsp[0].str_value)); (yyval.expr_t) = column_expr; } -#line 6311 "parser.cpp" +#line 6326 "parser.cpp" break; - case 310: /* column_expr: '*' */ -#line 2513 "parser.y" + case 312: /* column_expr: '*' */ +#line 2520 "parser.y" { infinity::ColumnExpr* column_expr = new infinity::ColumnExpr(); column_expr->star_ = true; (yyval.expr_t) = column_expr; } -#line 6321 "parser.cpp" +#line 6336 "parser.cpp" break; - case 311: /* column_expr: column_expr '.' '*' */ -#line 2518 "parser.y" + case 313: /* column_expr: column_expr '.' '*' */ +#line 2525 "parser.y" { infinity::ColumnExpr* column_expr = (infinity::ColumnExpr*)(yyvsp[-2].expr_t); if(column_expr->star_) { @@ -6331,353 +6346,353 @@ YYLTYPE yylloc = yyloc_default; column_expr->star_ = true; (yyval.expr_t) = column_expr; } -#line 6335 "parser.cpp" +#line 6350 "parser.cpp" break; - case 312: /* constant_expr: STRING */ -#line 2528 "parser.y" + case 314: /* constant_expr: STRING */ +#line 2535 "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 6345 "parser.cpp" +#line 6360 "parser.cpp" break; - case 313: /* constant_expr: TRUE */ -#line 2533 "parser.y" + case 315: /* constant_expr: TRUE */ +#line 2540 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kBoolean); const_expr->bool_value_ = true; (yyval.const_expr_t) = const_expr; } -#line 6355 "parser.cpp" +#line 6370 "parser.cpp" break; - case 314: /* constant_expr: FALSE */ -#line 2538 "parser.y" + case 316: /* constant_expr: FALSE */ +#line 2545 "parser.y" { infinity::ConstantExpr* const_expr = new infinity::ConstantExpr(infinity::LiteralType::kBoolean); const_expr->bool_value_ = false; (yyval.const_expr_t) = const_expr; } -#line 6365 "parser.cpp" +#line 6380 "parser.cpp" break; - case 315: /* constant_expr: DOUBLE_VALUE */ -#line 2543 "parser.y" + case 317: /* constant_expr: DOUBLE_VALUE */ +#line 2550 "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 6375 "parser.cpp" +#line 6390 "parser.cpp" break; - case 316: /* constant_expr: LONG_VALUE */ -#line 2548 "parser.y" + case 318: /* constant_expr: LONG_VALUE */ +#line 2555 "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 6385 "parser.cpp" +#line 6400 "parser.cpp" break; - case 317: /* constant_expr: DATE STRING */ -#line 2553 "parser.y" + case 319: /* constant_expr: DATE STRING */ +#line 2560 "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 6395 "parser.cpp" +#line 6410 "parser.cpp" break; - case 318: /* constant_expr: TIME STRING */ -#line 2558 "parser.y" + case 320: /* constant_expr: TIME STRING */ +#line 2565 "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 6405 "parser.cpp" +#line 6420 "parser.cpp" break; - case 319: /* constant_expr: DATETIME STRING */ -#line 2563 "parser.y" + case 321: /* constant_expr: DATETIME STRING */ +#line 2570 "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 6415 "parser.cpp" +#line 6430 "parser.cpp" break; - case 320: /* constant_expr: TIMESTAMP STRING */ -#line 2568 "parser.y" + case 322: /* constant_expr: TIMESTAMP STRING */ +#line 2575 "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 6425 "parser.cpp" +#line 6440 "parser.cpp" break; - case 321: /* constant_expr: INTERVAL interval_expr */ -#line 2573 "parser.y" + case 323: /* constant_expr: INTERVAL interval_expr */ +#line 2580 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6433 "parser.cpp" +#line 6448 "parser.cpp" break; - case 322: /* constant_expr: interval_expr */ -#line 2576 "parser.y" + case 324: /* constant_expr: interval_expr */ +#line 2583 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6441 "parser.cpp" +#line 6456 "parser.cpp" break; - case 323: /* constant_expr: long_array_expr */ -#line 2579 "parser.y" + case 325: /* constant_expr: long_array_expr */ +#line 2586 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6449 "parser.cpp" +#line 6464 "parser.cpp" break; - case 324: /* constant_expr: double_array_expr */ -#line 2582 "parser.y" + case 326: /* constant_expr: double_array_expr */ +#line 2589 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6457 "parser.cpp" +#line 6472 "parser.cpp" break; - case 325: /* array_expr: long_array_expr */ -#line 2586 "parser.y" + case 327: /* array_expr: long_array_expr */ +#line 2593 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6465 "parser.cpp" +#line 6480 "parser.cpp" break; - case 326: /* array_expr: double_array_expr */ -#line 2589 "parser.y" + case 328: /* array_expr: double_array_expr */ +#line 2596 "parser.y" { (yyval.const_expr_t) = (yyvsp[0].const_expr_t); } -#line 6473 "parser.cpp" +#line 6488 "parser.cpp" break; - case 327: /* long_array_expr: unclosed_long_array_expr ']' */ -#line 2593 "parser.y" + case 329: /* long_array_expr: unclosed_long_array_expr ']' */ +#line 2600 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 6481 "parser.cpp" +#line 6496 "parser.cpp" break; - case 328: /* unclosed_long_array_expr: '[' LONG_VALUE */ -#line 2597 "parser.y" + case 330: /* unclosed_long_array_expr: '[' LONG_VALUE */ +#line 2604 "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 6491 "parser.cpp" +#line 6506 "parser.cpp" break; - case 329: /* unclosed_long_array_expr: unclosed_long_array_expr ',' LONG_VALUE */ -#line 2602 "parser.y" + case 331: /* unclosed_long_array_expr: unclosed_long_array_expr ',' LONG_VALUE */ +#line 2609 "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 6500 "parser.cpp" +#line 6515 "parser.cpp" break; - case 330: /* double_array_expr: unclosed_double_array_expr ']' */ -#line 2607 "parser.y" + case 332: /* double_array_expr: unclosed_double_array_expr ']' */ +#line 2614 "parser.y" { (yyval.const_expr_t) = (yyvsp[-1].const_expr_t); } -#line 6508 "parser.cpp" +#line 6523 "parser.cpp" break; - case 331: /* unclosed_double_array_expr: '[' DOUBLE_VALUE */ -#line 2611 "parser.y" + case 333: /* unclosed_double_array_expr: '[' DOUBLE_VALUE */ +#line 2618 "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 6518 "parser.cpp" +#line 6533 "parser.cpp" break; - case 332: /* unclosed_double_array_expr: unclosed_double_array_expr ',' DOUBLE_VALUE */ -#line 2616 "parser.y" + case 334: /* unclosed_double_array_expr: unclosed_double_array_expr ',' DOUBLE_VALUE */ +#line 2623 "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 6527 "parser.cpp" +#line 6542 "parser.cpp" break; - case 333: /* interval_expr: LONG_VALUE SECONDS */ -#line 2621 "parser.y" + case 335: /* interval_expr: LONG_VALUE SECONDS */ +#line 2628 "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 6538 "parser.cpp" +#line 6553 "parser.cpp" break; - case 334: /* interval_expr: LONG_VALUE SECOND */ -#line 2627 "parser.y" + case 336: /* interval_expr: LONG_VALUE SECOND */ +#line 2634 "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 6549 "parser.cpp" +#line 6564 "parser.cpp" break; - case 335: /* interval_expr: LONG_VALUE MINUTES */ -#line 2633 "parser.y" + case 337: /* interval_expr: LONG_VALUE MINUTES */ +#line 2640 "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 6560 "parser.cpp" +#line 6575 "parser.cpp" break; - case 336: /* interval_expr: LONG_VALUE MINUTE */ -#line 2639 "parser.y" + case 338: /* interval_expr: LONG_VALUE MINUTE */ +#line 2646 "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 6571 "parser.cpp" +#line 6586 "parser.cpp" break; - case 337: /* interval_expr: LONG_VALUE HOURS */ -#line 2645 "parser.y" + case 339: /* interval_expr: LONG_VALUE HOURS */ +#line 2652 "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 6582 "parser.cpp" +#line 6597 "parser.cpp" break; - case 338: /* interval_expr: LONG_VALUE HOUR */ -#line 2651 "parser.y" + case 340: /* interval_expr: LONG_VALUE HOUR */ +#line 2658 "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 6593 "parser.cpp" +#line 6608 "parser.cpp" break; - case 339: /* interval_expr: LONG_VALUE DAYS */ -#line 2657 "parser.y" + case 341: /* interval_expr: LONG_VALUE DAYS */ +#line 2664 "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 6604 "parser.cpp" +#line 6619 "parser.cpp" break; - case 340: /* interval_expr: LONG_VALUE DAY */ -#line 2663 "parser.y" + case 342: /* interval_expr: LONG_VALUE DAY */ +#line 2670 "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 6615 "parser.cpp" +#line 6630 "parser.cpp" break; - case 341: /* interval_expr: LONG_VALUE MONTHS */ -#line 2669 "parser.y" + case 343: /* interval_expr: LONG_VALUE MONTHS */ +#line 2676 "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 6626 "parser.cpp" +#line 6641 "parser.cpp" break; - case 342: /* interval_expr: LONG_VALUE MONTH */ -#line 2675 "parser.y" + case 344: /* interval_expr: LONG_VALUE MONTH */ +#line 2682 "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 6637 "parser.cpp" +#line 6652 "parser.cpp" break; - case 343: /* interval_expr: LONG_VALUE YEARS */ -#line 2681 "parser.y" + case 345: /* interval_expr: LONG_VALUE YEARS */ +#line 2688 "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 6648 "parser.cpp" +#line 6663 "parser.cpp" break; - case 344: /* interval_expr: LONG_VALUE YEAR */ -#line 2687 "parser.y" + case 346: /* interval_expr: LONG_VALUE YEAR */ +#line 2694 "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 6659 "parser.cpp" +#line 6674 "parser.cpp" break; - case 345: /* copy_option_list: copy_option */ -#line 2698 "parser.y" + case 347: /* copy_option_list: copy_option */ +#line 2705 "parser.y" { (yyval.copy_option_array) = new std::vector(); (yyval.copy_option_array)->push_back((yyvsp[0].copy_option_t)); } -#line 6668 "parser.cpp" +#line 6683 "parser.cpp" break; - case 346: /* copy_option_list: copy_option_list ',' copy_option */ -#line 2702 "parser.y" + case 348: /* copy_option_list: copy_option_list ',' copy_option */ +#line 2709 "parser.y" { (yyvsp[-2].copy_option_array)->push_back((yyvsp[0].copy_option_t)); (yyval.copy_option_array) = (yyvsp[-2].copy_option_array); } -#line 6677 "parser.cpp" +#line 6692 "parser.cpp" break; - case 347: /* copy_option: FORMAT IDENTIFIER */ -#line 2707 "parser.y" + case 349: /* copy_option: FORMAT IDENTIFIER */ +#line 2714 "parser.y" { (yyval.copy_option_t) = new infinity::CopyOption(); (yyval.copy_option_t)->option_type_ = infinity::CopyOptionType::kFormat; @@ -6700,11 +6715,11 @@ YYLTYPE yylloc = yyloc_default; YYERROR; } } -#line 6704 "parser.cpp" +#line 6719 "parser.cpp" break; - case 348: /* copy_option: DELIMITER STRING */ -#line 2729 "parser.y" + case 350: /* copy_option: DELIMITER STRING */ +#line 2736 "parser.y" { (yyval.copy_option_t) = new infinity::CopyOption(); (yyval.copy_option_t)->option_type_ = infinity::CopyOptionType::kDelimiter; @@ -6715,53 +6730,53 @@ YYLTYPE yylloc = yyloc_default; } free((yyvsp[0].str_value)); } -#line 6719 "parser.cpp" +#line 6734 "parser.cpp" break; - case 349: /* copy_option: HEADER */ -#line 2739 "parser.y" + case 351: /* copy_option: HEADER */ +#line 2746 "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 6729 "parser.cpp" +#line 6744 "parser.cpp" break; - case 350: /* file_path: STRING */ -#line 2745 "parser.y" + case 352: /* file_path: STRING */ +#line 2752 "parser.y" { (yyval.str_value) = (yyvsp[0].str_value); } -#line 6737 "parser.cpp" +#line 6752 "parser.cpp" break; - case 351: /* if_exists: IF EXISTS */ -#line 2749 "parser.y" + case 353: /* if_exists: IF EXISTS */ +#line 2756 "parser.y" { (yyval.bool_value) = true; } -#line 6743 "parser.cpp" +#line 6758 "parser.cpp" break; - case 352: /* if_exists: %empty */ -#line 2750 "parser.y" + case 354: /* if_exists: %empty */ +#line 2757 "parser.y" { (yyval.bool_value) = false; } -#line 6749 "parser.cpp" +#line 6764 "parser.cpp" break; - case 353: /* if_not_exists: IF NOT EXISTS */ -#line 2752 "parser.y" + case 355: /* if_not_exists: IF NOT EXISTS */ +#line 2759 "parser.y" { (yyval.bool_value) = true; } -#line 6755 "parser.cpp" +#line 6770 "parser.cpp" break; - case 354: /* if_not_exists: %empty */ -#line 2753 "parser.y" + case 356: /* if_not_exists: %empty */ +#line 2760 "parser.y" { (yyval.bool_value) = false; } -#line 6761 "parser.cpp" +#line 6776 "parser.cpp" break; - case 357: /* if_not_exists_info: if_not_exists IDENTIFIER */ -#line 2768 "parser.y" + case 359: /* if_not_exists_info: if_not_exists IDENTIFIER */ +#line 2775 "parser.y" { (yyval.if_not_exists_info_t) = new infinity::IfNotExistsInfo(); (yyval.if_not_exists_info_t)->exists_ = true; @@ -6770,79 +6785,79 @@ YYLTYPE yylloc = yyloc_default; (yyval.if_not_exists_info_t)->info_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 6774 "parser.cpp" +#line 6789 "parser.cpp" break; - case 358: /* if_not_exists_info: %empty */ -#line 2776 "parser.y" + case 360: /* if_not_exists_info: %empty */ +#line 2783 "parser.y" { (yyval.if_not_exists_info_t) = new infinity::IfNotExistsInfo(); } -#line 6782 "parser.cpp" +#line 6797 "parser.cpp" break; - case 359: /* with_index_param_list: WITH '(' index_param_list ')' */ -#line 2780 "parser.y" + case 361: /* with_index_param_list: WITH '(' index_param_list ')' */ +#line 2787 "parser.y" { (yyval.with_index_param_list_t) = std::move((yyvsp[-1].index_param_list_t)); } -#line 6790 "parser.cpp" +#line 6805 "parser.cpp" break; - case 360: /* with_index_param_list: %empty */ -#line 2783 "parser.y" + case 362: /* with_index_param_list: %empty */ +#line 2790 "parser.y" { (yyval.with_index_param_list_t) = new std::vector(); } -#line 6798 "parser.cpp" +#line 6813 "parser.cpp" break; - case 361: /* optional_table_properties_list: PROPERTIES '(' index_param_list ')' */ -#line 2787 "parser.y" + case 363: /* optional_table_properties_list: PROPERTIES '(' index_param_list ')' */ +#line 2794 "parser.y" { (yyval.with_index_param_list_t) = (yyvsp[-1].index_param_list_t); } -#line 6806 "parser.cpp" +#line 6821 "parser.cpp" break; - case 362: /* optional_table_properties_list: %empty */ -#line 2790 "parser.y" + case 364: /* optional_table_properties_list: %empty */ +#line 2797 "parser.y" { (yyval.with_index_param_list_t) = nullptr; } -#line 6814 "parser.cpp" +#line 6829 "parser.cpp" break; - case 363: /* index_param_list: index_param */ -#line 2794 "parser.y" + case 365: /* index_param_list: index_param */ +#line 2801 "parser.y" { (yyval.index_param_list_t) = new std::vector(); (yyval.index_param_list_t)->push_back((yyvsp[0].index_param_t)); } -#line 6823 "parser.cpp" +#line 6838 "parser.cpp" break; - case 364: /* index_param_list: index_param_list ',' index_param */ -#line 2798 "parser.y" + case 366: /* index_param_list: index_param_list ',' index_param */ +#line 2805 "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 6832 "parser.cpp" +#line 6847 "parser.cpp" break; - case 365: /* index_param: IDENTIFIER */ -#line 2803 "parser.y" + case 367: /* index_param: IDENTIFIER */ +#line 2810 "parser.y" { (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 6842 "parser.cpp" +#line 6857 "parser.cpp" break; - case 366: /* index_param: IDENTIFIER '=' IDENTIFIER */ -#line 2808 "parser.y" + case 368: /* index_param: IDENTIFIER '=' IDENTIFIER */ +#line 2815 "parser.y" { (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[-2].str_value); @@ -6851,11 +6866,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.index_param_t)->param_value_ = (yyvsp[0].str_value); free((yyvsp[0].str_value)); } -#line 6855 "parser.cpp" +#line 6870 "parser.cpp" break; - case 367: /* index_param: IDENTIFIER '=' LONG_VALUE */ -#line 2816 "parser.y" + case 369: /* index_param: IDENTIFIER '=' LONG_VALUE */ +#line 2823 "parser.y" { (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[-2].str_value); @@ -6863,11 +6878,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.index_param_t)->param_value_ = std::to_string((yyvsp[0].long_value)); } -#line 6867 "parser.cpp" +#line 6882 "parser.cpp" break; - case 368: /* index_param: IDENTIFIER '=' DOUBLE_VALUE */ -#line 2823 "parser.y" + case 370: /* index_param: IDENTIFIER '=' DOUBLE_VALUE */ +#line 2830 "parser.y" { (yyval.index_param_t) = new infinity::InitParameter(); (yyval.index_param_t)->param_name_ = (yyvsp[-2].str_value); @@ -6875,11 +6890,11 @@ YYLTYPE yylloc = yyloc_default; (yyval.index_param_t)->param_value_ = std::to_string((yyvsp[0].double_value)); } -#line 6879 "parser.cpp" +#line 6894 "parser.cpp" break; - case 369: /* index_info_list: '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ -#line 2834 "parser.y" + case 371: /* index_info_list: '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ +#line 2841 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); infinity::IndexType index_type = infinity::IndexType::kInvalid; @@ -6928,11 +6943,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-4].identifier_array_t); } -#line 6932 "parser.cpp" +#line 6947 "parser.cpp" break; - case 370: /* index_info_list: index_info_list '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ -#line 2882 "parser.y" + case 372: /* index_info_list: index_info_list '(' identifier_array ')' USING IDENTIFIER with_index_param_list */ +#line 2889 "parser.y" { ParserHelper::ToLower((yyvsp[-1].str_value)); infinity::IndexType index_type = infinity::IndexType::kInvalid; @@ -6982,11 +6997,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-4].identifier_array_t); } -#line 6986 "parser.cpp" +#line 7001 "parser.cpp" break; - case 371: /* index_info_list: '(' identifier_array ')' */ -#line 2931 "parser.y" + case 373: /* index_info_list: '(' identifier_array ')' */ +#line 2938 "parser.y" { infinity::IndexType index_type = infinity::IndexType::kSecondary; size_t index_count = (yyvsp[-1].identifier_array_t)->size(); @@ -7000,11 +7015,11 @@ YYLTYPE yylloc = yyloc_default; } delete (yyvsp[-1].identifier_array_t); } -#line 7004 "parser.cpp" +#line 7019 "parser.cpp" break; -#line 7008 "parser.cpp" +#line 7023 "parser.cpp" default: break; } @@ -7233,7 +7248,7 @@ YYLTYPE yylloc = yyloc_default; return yyresult; } -#line 2945 "parser.y" +#line 2952 "parser.y" void diff --git a/src/parser/parser.h b/src/parser/parser.h index aa62533e98..5011616a6a 100644 --- a/src/parser/parser.h +++ b/src/parser/parser.h @@ -74,6 +74,7 @@ extern int sqldebug; #include "statement/show_statement.h" #include "statement/update_statement.h" #include "statement/command_statement.h" +#include "statement/compact_statement.h" #include "table_reference/base_table_reference.h" #include "table_reference/join_reference.h" #include "table_reference/cross_product_reference.h" @@ -115,7 +116,7 @@ struct SQL_LTYPE { } \ } -#line 119 "parser.h" +#line 120 "parser.h" /* Token kinds. */ #ifndef SQLTOKENTYPE @@ -300,7 +301,7 @@ struct SQL_LTYPE { #if ! defined SQLSTYPE && ! defined SQLSTYPE_IS_DECLARED union SQLSTYPE { -#line 102 "parser.y" +#line 103 "parser.y" bool bool_value; char* str_value; @@ -323,6 +324,7 @@ union SQLSTYPE infinity::FlushStatement* flush_stmt; infinity::OptimizeStatement* optimize_stmt; infinity::CommandStatement* command_stmt; + infinity::CompactStatement* compact_stmt; std::vector* stmt_array; @@ -373,7 +375,7 @@ union SQLSTYPE // infinity::IfExistsInfo* if_exists_info_t; infinity::IfNotExistsInfo* if_not_exists_info_t; -#line 377 "parser.h" +#line 379 "parser.h" }; typedef union SQLSTYPE SQLSTYPE; diff --git a/src/parser/parser.y b/src/parser/parser.y index 6eabdf4425..dc28c03c13 100644 --- a/src/parser/parser.y +++ b/src/parser/parser.y @@ -28,6 +28,7 @@ void yyerror(YYLTYPE * llocp, void* lexer, infinity::ParserResult* result, const #include "statement/show_statement.h" #include "statement/update_statement.h" #include "statement/command_statement.h" +#include "statement/compact_statement.h" #include "table_reference/base_table_reference.h" #include "table_reference/join_reference.h" #include "table_reference/cross_product_reference.h" @@ -121,6 +122,7 @@ struct SQL_LTYPE { infinity::FlushStatement* flush_stmt; infinity::OptimizeStatement* optimize_stmt; infinity::CommandStatement* command_stmt; + infinity::CompactStatement* compact_stmt; std::vector* stmt_array; @@ -394,6 +396,7 @@ struct SQL_LTYPE { %type flush_statement %type optimize_statement %type command_statement +%type compact_statement %type statement_list @@ -499,6 +502,7 @@ statement : create_statement { $$ = $1; } | flush_statement { $$ = $1; } | optimize_statement { $$ = $1; } | command_statement { $$ = $1; } +| compact_statement { $$ = $1; } explainable_statement : create_statement { $$ = $1; } | drop_statement { $$ = $1; } @@ -511,6 +515,7 @@ explainable_statement : create_statement { $$ = $1; } | flush_statement { $$ = $1; } | optimize_statement { $$ = $1; } | command_statement { $$ = $1; } +| compact_statement { $$ = $1; } /* * CREATE STATEMENT @@ -1821,16 +1826,18 @@ command_statement: USE IDENTIFIER { $$->command_info_ = std::make_unique(infinity::SetScope::kConfig, infinity::SetVarType::kDouble, $3, $4); free($3); } -| COMPACT TABLE table_name { - $$ = new infinity::CommandStatement(); + +compact_statement: COMPACT TABLE table_name { + std::string schema_name; if ($3->schema_name_ptr_ != nullptr) { - $$->command_info_ = std::make_unique(std::string($3->schema_name_ptr_), std::string($3->table_name_ptr_)); + schema_name = std::string($3->schema_name_ptr_); free($3->schema_name_ptr_); - free($3->table_name_ptr_); - } else { - $$->command_info_ = std::make_unique(std::string($3->table_name_ptr_)); - free($3->table_name_ptr_); - } delete $3; + } + std::string table_name = std::string($3->table_name_ptr_); + free($3->table_name_ptr_); + + $$ = new infinity::ManualCompactStatement(std::move(schema_name), std::move(table_name)); + delete $3; } /* diff --git a/src/parser/statement/command_statement.cpp b/src/parser/statement/command_statement.cpp index 6e4d132364..7e5e037246 100644 --- a/src/parser/statement/command_statement.cpp +++ b/src/parser/statement/command_statement.cpp @@ -20,7 +20,6 @@ std::string UseCmd::ToString() const { return "Use Command"; } std::string SetCmd::ToString() const { return "Set Command"; } std::string ExportCmd::ToString() const { return "Export Command"; } std::string CheckTable::ToString() const { return "Check Table Command"; } -std::string CompactTable::ToString() const { return "Compact Table Command"; } std::string CommandStatement::ToString() const { return command_info_->ToString(); } } // namespace infinity \ No newline at end of file diff --git a/src/parser/statement/command_statement.cppm b/src/parser/statement/command_statement.cppm index c11d31de3d..29cf20b31b 100644 --- a/src/parser/statement/command_statement.cppm +++ b/src/parser/statement/command_statement.cppm @@ -30,6 +30,5 @@ export using infinity::SetCmd; export using infinity::UseCmd; export using infinity::CheckTable; export using infinity::ExportCmd; -export using infinity::CompactTable; } diff --git a/src/parser/statement/command_statement.h b/src/parser/statement/command_statement.h index ff732fc31d..9a0af1e52a 100644 --- a/src/parser/statement/command_statement.h +++ b/src/parser/statement/command_statement.h @@ -27,7 +27,6 @@ enum class CommandType { kSet, kExport, kCheckTable, - kCompactTable, }; class CommandInfo { @@ -172,21 +171,6 @@ class CheckTable final : public CommandInfo { std::string table_name_; }; -class CompactTable final : public CommandInfo { -public: - explicit CompactTable(std::string &&schema_name, std::string &&table_name) - : CommandInfo(CommandType::kCompactTable), schema_name_(std::move(schema_name)), table_name_(std::move(table_name)) {} - - explicit CompactTable(std::string &&table_name) : CommandInfo(CommandType::kCompactTable), table_name_(std::move(table_name)) {} - - std::string ToString() const final; - -public: - std::string schema_name_{}; - - std::string table_name_; -}; - class CommandStatement final : public BaseStatement { public: CommandStatement() : BaseStatement(StatementType::kCommand) {} diff --git a/src/parser/statement/compact_statement.cppm b/src/parser/statement/compact_statement.cppm new file mode 100644 index 0000000000..cec6858d67 --- /dev/null +++ b/src/parser/statement/compact_statement.cppm @@ -0,0 +1,26 @@ +// 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 "compact_statement.h" + +export module compact_statement; + +namespace infinity { +export using infinity::CompactStatementType; +export using infinity::CompactStatement; +export using infinity::ManualCompactStatement; +export using infinity::AutoCompactStatement; +}; \ No newline at end of file diff --git a/src/parser/statement/compact_statement.h b/src/parser/statement/compact_statement.h new file mode 100644 index 0000000000..5a5eb01540 --- /dev/null +++ b/src/parser/statement/compact_statement.h @@ -0,0 +1,62 @@ +// 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. + +#pragma once + +#include "base_statement.h" +#include +#include + +namespace infinity { + +struct TableEntry; +struct SegmentEntry; + +enum class CompactStatementType { + kManual, + kAuto, + kInvalid +}; + +class CompactStatement : public BaseStatement { +public: + CompactStatement(CompactStatementType compact_type) : BaseStatement(StatementType::kCompact), compact_type_(compact_type) {} + + [[nodiscard]] std::string ToString() const final { return "Compact Table {}.{}"; } + +public: + CompactStatementType compact_type_; +}; + +class ManualCompactStatement : public CompactStatement { +public: + ManualCompactStatement(std::string schema_name, std::string table_name) + : CompactStatement{CompactStatementType::kManual}, schema_name_(std::move(schema_name)), table_name_(std::move(table_name)) {} + +public: + std::string schema_name_{}; + std::string table_name_{}; +}; + +struct AutoCompactStatement : public CompactStatement { +public: + AutoCompactStatement(TableEntry *table_entry, std::vector compactible_segments) + : CompactStatement{CompactStatementType::kAuto}, table_entry_(table_entry), compactible_segments_(std::move(compactible_segments)) {} + +public: + TableEntry *table_entry_; + std::vector compactible_segments_; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/planner/bound/base_table_ref.cppm b/src/planner/bound/base_table_ref.cppm index c3cdd6de12..2d597bc713 100644 --- a/src/planner/bound/base_table_ref.cppm +++ b/src/planner/bound/base_table_ref.cppm @@ -45,8 +45,8 @@ public: table_index_(table_index) {} // only use some fields - explicit BaseTableRef(TableEntry *table_entry, SharedPtr block_index) - : TableRef(TableRefType::kTable, ""), table_entry_ptr_(table_entry), block_index_(std::move(block_index)) {} + explicit BaseTableRef(TableEntry *table_entry, SharedPtr block_index, SharedPtr index_index = nullptr) + : TableRef(TableRefType::kTable, ""), table_entry_ptr_(table_entry), block_index_(block_index), index_index_(index_index) {} static SharedPtr FakeTableRef(TableEntry *table_entry, Txn *txn) { SharedPtr block_index = table_entry->GetBlockIndex(txn); @@ -66,6 +66,7 @@ public: TableEntry *table_entry_ptr_{}; Vector column_ids_{}; SharedPtr block_index_{}; + SharedPtr index_index_{}; SharedPtr> column_names_{}; SharedPtr>> column_types_{}; diff --git a/src/planner/bound_compact_statement.cppm b/src/planner/bound_compact_statement.cppm new file mode 100644 index 0000000000..a41b7c9d0a --- /dev/null +++ b/src/planner/bound_compact_statement.cppm @@ -0,0 +1,93 @@ +// 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; + +export module bound_compact_statement; + +import stl; +import logical_compact; +import logical_compact_finish; +import logical_compact_index; +import base_table_ref; +import block_index; +import logical_node; +import query_context; +import bind_context; +import logger; +import third_party; +import table_entry; +import compact_statement; + +namespace infinity { + +export struct BoundCompactStatement final { +public: + BoundCompactStatement(SharedPtr bind_context, SharedPtr base_table_ref, CompactStatementType compact_type) + : bind_context_(std::move(bind_context)), base_table_ref_(base_table_ref), compact_type_(compact_type) {} + + Vector> BuildPlans(QueryContext *query_context) { + Vector> res; + const SharedPtr &bind_context = this->bind_context_; + + auto compact_node = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref_, compact_type_); + auto &index_index = base_table_ref_->index_index_; + if (!index_index->IsEmpty()) { + if (index_index->index_snapshots_.size() == 1) { + auto compact_index_node = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref_); + compact_index_node->set_left_node(compact_node); + auto compact_finish_node = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref_, compact_type_); + compact_finish_node->set_left_node(compact_index_node); + res.emplace_back(compact_finish_node); + } else { + res.emplace_back(compact_node); + if (index_index->index_snapshots_.size() > 2) { + LOG_WARN(fmt::format("Table {} has more than 2 index, but logical plan is binary tree now", *base_table_ref_->table_name())); + } + auto index_index1 = MakeShared(); + auto index_index2 = MakeShared(); + SizeT idx = 0; + for (const auto &[index_name, index_snapshot] : index_index->index_snapshots_) { + if (idx == 0) { + index_index1->Insert(index_name, index_snapshot); + } else { + index_index2->Insert(index_name, index_snapshot); + } + ++idx; + } + auto base_table_ref1 = MakeShared(base_table_ref_->table_entry_ptr_, base_table_ref_->block_index_, index_index1); + auto base_table_ref2 = MakeShared(base_table_ref_->table_entry_ptr_, base_table_ref_->block_index_, index_index2); + auto compact_index_node1 = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref1); + auto compact_index_node2 = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref2); + auto compact_finish = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref_, compact_type_); + compact_finish->set_left_node(compact_index_node1); + compact_finish->set_right_node(compact_index_node2); + res.emplace_back(compact_finish); + } + } else { + auto compact_finish = MakeShared(bind_context->GetNewLogicalNodeId(), base_table_ref_, compact_type_); + compact_finish->set_left_node(compact_node); + res.emplace_back(compact_finish); + } + return res; + } + +private: + SharedPtr bind_context_{}; + + SharedPtr base_table_ref_{}; + CompactStatementType compact_type_; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/planner/logical_node_type.cppm b/src/planner/logical_node_type.cppm index 6db41f28fc..bd04463204 100644 --- a/src/planner/logical_node_type.cppm +++ b/src/planner/logical_node_type.cppm @@ -73,5 +73,10 @@ export enum class LogicalNodeType : u8 { kPrepare, kFlush, kOptimize, + + // Compact + kCompact, + kCompactIndex, + kCompactFinish, }; } diff --git a/src/planner/logical_planner.cpp b/src/planner/logical_planner.cpp index 01969d1ef4..14efdee449 100644 --- a/src/planner/logical_planner.cpp +++ b/src/planner/logical_planner.cpp @@ -29,6 +29,7 @@ import query_binder; import bound_delete_statement; import bound_update_statement; import bound_select_statement; +import bound_compact_statement; import insert_binder; import logical_insert; import logical_node; @@ -175,6 +176,9 @@ Status LogicalPlanner::Build(const BaseStatement *statement, SharedPtr(statement), bind_context_ptr); } + case StatementType::kCompact: { + return BuildCompact(static_cast(statement), bind_context_ptr); + } default: { UnrecoverableError("Invalid statement type."); } @@ -946,21 +950,6 @@ Status LogicalPlanner::BuildCommand(const CommandStatement *statement, SharedPtr } break; } - case CommandType::kCompactTable: { - auto *compact_table = static_cast(command_statement->command_info_.get()); - BindSchemaName(compact_table->schema_name_); - - Txn *txn = query_context_ptr_->GetTxn(); - auto [table_entry, status] = txn->GetTableByName(compact_table->schema_name_, compact_table->table_name_); - if (!status.ok()) { - RecoverableError(status); - } - auto logical_command = MakeShared(bind_context_ptr->GetNewLogicalNodeId(), std::move(command_statement->command_info_)); - logical_command->table_entry_ = table_entry; - - this->logical_plan_ = logical_command; - break; - } default: { UnrecoverableError("Invalid command type."); } @@ -968,6 +957,13 @@ Status LogicalPlanner::BuildCommand(const CommandStatement *statement, SharedPtr return Status::OK(); } +Status LogicalPlanner::BuildCompact(const CompactStatement *statement, SharedPtr &bind_context_ptr) { + UniquePtr query_binder_ptr = MakeUnique(this->query_context_ptr_, bind_context_ptr); + UniquePtr bound_statement_ptr = query_binder_ptr->BindCompact(*statement); + this->logical_plans_ = bound_statement_ptr->BuildPlans(query_context_ptr_); + return Status::OK(); +} + Status LogicalPlanner::BuildShow(ShowStatement *statement, SharedPtr &bind_context_ptr) { BindSchemaName(statement->schema_name_); switch (statement->show_type_) { diff --git a/src/planner/logical_planner.cppm b/src/planner/logical_planner.cppm index 56714af912..863df0e7ed 100644 --- a/src/planner/logical_planner.cppm +++ b/src/planner/logical_planner.cppm @@ -40,6 +40,7 @@ import execute_statement; import alter_statement; import explain_statement; import command_statement; +import compact_statement; import data_type; import extra_ddl_info; @@ -165,10 +166,17 @@ public: Status BuildCommand(const CommandStatement *statement, SharedPtr &bind_context_ptr); + Status BuildCompact(const CompactStatement *statement, SharedPtr &bind_context_ptr); + // Explain Status BuildExplain(const ExplainStatement *statement, SharedPtr &bind_context_ptr); - [[nodiscard]] SharedPtr LogicalPlan() const { return logical_plan_; } + [[nodiscard]] Vector> LogicalPlans() const { + if (logical_plans_.empty()) { + return {logical_plan_}; + } + return logical_plans_; + } private: void BindSchemaName(String &schema_name) const; @@ -181,6 +189,7 @@ private: SharedPtr> names_ptr_{}; SharedPtr> types_ptr_{}; SharedPtr logical_plan_{}; + Vector> logical_plans_{}; }; } // namespace infinity diff --git a/src/planner/node/logical_command.cpp b/src/planner/node/logical_command.cpp index b6d48a8d0d..c4cdac2ed1 100644 --- a/src/planner/node/logical_command.cpp +++ b/src/planner/node/logical_command.cpp @@ -102,11 +102,6 @@ String LogicalCommand::ToString(i64 &space) const { ss << String(space, ' ') << arrow_str << "Check table: " << check_table_info->table_name(); break; } - case CommandType::kCompactTable: { - CompactTable *compact_table_info = (CompactTable *)(command_info_.get()); - ss << String(space, ' ') << arrow_str << "Compact table: " << compact_table_info->schema_name_ << " " << compact_table_info->table_name_; - break; - } case CommandType::kInvalid: { UnrecoverableError("Invalid command type."); } diff --git a/src/planner/node/logical_command.cppm b/src/planner/node/logical_command.cppm index 63f33ccbe0..0bd585818d 100644 --- a/src/planner/node/logical_command.cppm +++ b/src/planner/node/logical_command.cppm @@ -46,9 +46,6 @@ public: inline String name() final { return "LogicalCommand"; } -public: - TableEntry *table_entry_ = nullptr; // only used for compact command - private: SharedPtr command_info_{}; }; diff --git a/src/planner/node/logical_compact.cpp b/src/planner/node/logical_compact.cpp new file mode 100644 index 0000000000..648962af84 --- /dev/null +++ b/src/planner/node/logical_compact.cpp @@ -0,0 +1,56 @@ +// 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 + +module logical_compact; + +import stl; +import column_binding; +import data_type; +import base_table_ref; +import logical_type; + +namespace infinity { + +Vector LogicalCompact::GetColumnBindings() const { return {}; } + +SharedPtr> LogicalCompact::GetOutputNames() const { + auto result = MakeShared>(); + result->push_back("OK"); + return result; +} + +SharedPtr>> LogicalCompact::GetOutputTypes() const { + auto result = MakeShared>>(); + result->push_back(MakeShared(LogicalType::kInteger)); + return result; +} + +String LogicalCompact::ToString(i64 &space) const { + std::stringstream ss; + String arrow_str; + if (space > 3) { + space -= 4; + arrow_str = "-> "; + } + ss << String(space, ' ') << arrow_str << "COMPACT "; + ss << *base_table_ref_->schema_name() << "." << *base_table_ref_->table_name(); + space += arrow_str.size(); + return ss.str(); +} + +} // namespace infinity \ No newline at end of file diff --git a/src/planner/node/logical_compact.cppm b/src/planner/node/logical_compact.cppm new file mode 100644 index 0000000000..cd526ef299 --- /dev/null +++ b/src/planner/node/logical_compact.cppm @@ -0,0 +1,50 @@ +// 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; + +export module logical_compact; + +import stl; +import logical_node; +import logical_node_type; +import column_binding; +import data_type; +import base_table_ref; +import query_context; +import compact_statement; + +namespace infinity { + +export class LogicalCompact : public LogicalNode { +public: + LogicalCompact(u64 node_id, SharedPtr base_table_ref, CompactStatementType compact_type) + : LogicalNode(node_id, LogicalNodeType::kCompact), base_table_ref_(base_table_ref), compact_type_(compact_type) {} + + [[nodiscard]] Vector GetColumnBindings() const final; + + [[nodiscard]] SharedPtr> GetOutputNames() const final; + + [[nodiscard]] SharedPtr>> GetOutputTypes() const final; + + String ToString(i64 &space) const final; + + inline String name() final { return "LogicalCompact"; } + +public: + SharedPtr base_table_ref_; + CompactStatementType compact_type_; +}; + +} // namespace infinity diff --git a/src/planner/node/logical_compact_finish.cpp b/src/planner/node/logical_compact_finish.cpp new file mode 100644 index 0000000000..1dc7dff61d --- /dev/null +++ b/src/planner/node/logical_compact_finish.cpp @@ -0,0 +1,56 @@ +// 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 + +module logical_compact_finish; + +import stl; +import column_binding; +import data_type; +import base_table_ref; +import logical_type; + +namespace infinity { + +Vector LogicalCompactFinish::GetColumnBindings() const { return {}; } + +SharedPtr> LogicalCompactFinish::GetOutputNames() const { + auto result = MakeShared>(); + result->push_back("OK"); + return result; +} + +SharedPtr>> LogicalCompactFinish::GetOutputTypes() const { + auto result = MakeShared>>(); + result->push_back(MakeShared(LogicalType::kInteger)); + return result; +} + +String LogicalCompactFinish::ToString(i64 &space) const { + std::stringstream ss; + String arrow_str; + if (space > 3) { + space -= 4; + arrow_str = "-> "; + } + ss << String(space, ' ') << arrow_str << "COMPACT FINISH"; + ss << *base_table_ref_->schema_name() << "." << *base_table_ref_->table_name(); + space += arrow_str.size(); + return ss.str(); +} + +} // namespace infinity \ No newline at end of file diff --git a/src/planner/node/logical_compact_finish.cppm b/src/planner/node/logical_compact_finish.cppm new file mode 100644 index 0000000000..c930a426fb --- /dev/null +++ b/src/planner/node/logical_compact_finish.cppm @@ -0,0 +1,49 @@ +// 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; + +export module logical_compact_finish; + +import stl; +import logical_node; +import logical_node_type; +import column_binding; +import data_type; +import base_table_ref; +import compact_statement; + +namespace infinity { + +export class LogicalCompactFinish : public LogicalNode { +public: + LogicalCompactFinish(u64 node_id, SharedPtr base_table_ref, CompactStatementType compact_type) + : LogicalNode(node_id, LogicalNodeType::kCompactFinish), base_table_ref_(base_table_ref), compact_type_(compact_type) {} + + [[nodiscard]] Vector GetColumnBindings() const final; + + [[nodiscard]] SharedPtr> GetOutputNames() const final; + + [[nodiscard]] SharedPtr>> GetOutputTypes() const final; + + String ToString(i64 &space) const final; + + inline String name() final { return "LogicalCompactFinish"; } + +public: + SharedPtr base_table_ref_; + CompactStatementType compact_type_; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/planner/node/logical_compact_index.cpp b/src/planner/node/logical_compact_index.cpp new file mode 100644 index 0000000000..745f052197 --- /dev/null +++ b/src/planner/node/logical_compact_index.cpp @@ -0,0 +1,57 @@ +// 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 + +module logical_compact_index; + +import stl; +import column_binding; +import data_type; +import table_entry; +import logical_type; + +namespace infinity { + +Vector LogicalCompactIndex::GetColumnBindings() const { return {}; } + +SharedPtr> LogicalCompactIndex::GetOutputNames() const { + auto result = MakeShared>(); + result->push_back("OK"); + return result; +} + +SharedPtr>> LogicalCompactIndex::GetOutputTypes() const { + auto result = MakeShared>>(); + result->push_back(MakeShared(LogicalType::kInteger)); + return result; +} + +String LogicalCompactIndex::ToString(i64 &space) const { + std::stringstream ss; + String arrow_str; + if (space > 3) { + space -= 4; + arrow_str = "-> "; + } + ss << String(space, ' ') << arrow_str << "COMPACT INDEX"; + auto *table_entry = base_table_ref_->table_entry_ptr_; + ss << *table_entry->GetDBName() << "." << *table_entry->GetTableName(); + space += arrow_str.size(); + return ss.str(); +} + +} // namespace infinity diff --git a/src/planner/node/logical_compact_index.cppm b/src/planner/node/logical_compact_index.cppm new file mode 100644 index 0000000000..b0540f3cd9 --- /dev/null +++ b/src/planner/node/logical_compact_index.cppm @@ -0,0 +1,47 @@ +// 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; + +export module logical_compact_index; + +import stl; +import logical_node; +import logical_node_type; +import column_binding; +import data_type; +import base_table_ref; + +namespace infinity { + +export class LogicalCompactIndex : public LogicalNode { +public: + LogicalCompactIndex(u64 node_id, SharedPtr base_table_ref) + : LogicalNode(node_id, LogicalNodeType::kCompactIndex), base_table_ref_(base_table_ref) {} + + [[nodiscard]] Vector GetColumnBindings() const final; + + [[nodiscard]] SharedPtr> GetOutputNames() const final; + + [[nodiscard]] SharedPtr>> GetOutputTypes() const final; + + String ToString(i64 &space) const final; + + inline String name() final { return "LogicalCompactIndex"; } + +public: + SharedPtr base_table_ref_{}; +}; + +} // namespace infinity \ No newline at end of file diff --git a/src/planner/query_binder.cpp b/src/planner/query_binder.cpp index 23a879bd82..8f9c634372 100644 --- a/src/planner/query_binder.cpp +++ b/src/planner/query_binder.cpp @@ -30,6 +30,7 @@ import binding; import bound_select_statement; import bound_delete_statement; import bound_update_statement; +import bound_compact_statement; import table_ref; import bind_alias_proxy; import base_expression; @@ -58,6 +59,7 @@ import base_statement; import select_statement; import delete_statement; import update_statement; +import compact_statement; import parsed_expr; import column_expr; import knn_expr; @@ -71,6 +73,7 @@ import data_type; import logical_type; import base_entry; import view_entry; +import table_entry; import txn; namespace infinity { @@ -185,7 +188,7 @@ UniquePtr QueryBinder::BindSelect(const SelectStatement &s if (statement.where_expr_) { auto where_binder = MakeShared(query_context_ptr_, bind_alias_proxy); SharedPtr where_expr = where_binder->Bind(*statement.where_expr_, this->bind_context_ptr_.get(), 0, true); - if(where_expr->Type().type() != LogicalType::kBoolean) { + if (where_expr->Type().type() != LogicalType::kBoolean) { RecoverableError(Status::InvalidFilterExpression(where_expr->Type().ToString())); } bound_select_statement->where_conditions_ = SplitExpressionByDelimiter(where_expr, ConjunctionType::kAnd); @@ -1004,4 +1007,24 @@ UniquePtr QueryBinder::BindUpdate(const UpdateStatement &s return bound_update_statement; } +UniquePtr QueryBinder::BindCompact(const CompactStatement &statement) { + Txn *txn = query_context_ptr_->GetTxn(); + SharedPtr base_table_ref = nullptr; + if (statement.compact_type_ == CompactStatementType::kManual) { + const auto &compact_statement = static_cast(statement); + base_table_ref = GetTableRef(compact_statement.schema_name_, compact_statement.table_name_); + } else { + const auto &compact_statement = static_cast(statement); + auto block_index = MakeShared(); + for (auto *compactible_segment : compact_statement.compactible_segments_) { + block_index->Insert(compactible_segment, txn); + } + base_table_ref = MakeShared(compact_statement.table_entry_, std::move(block_index)); + } + TableEntry *table_entry = base_table_ref->table_entry_ptr_; + base_table_ref->index_index_ = table_entry->GetIndexIndex(txn); + + return MakeUnique(bind_context_ptr_, base_table_ref, statement.compact_type_); +} + } // namespace infinity diff --git a/src/planner/query_binder.cppm b/src/planner/query_binder.cppm index 962d074d37..c127219eb5 100644 --- a/src/planner/query_binder.cppm +++ b/src/planner/query_binder.cppm @@ -26,9 +26,11 @@ import binding; import bind_alias_proxy; import bound_delete_statement; import bound_update_statement; +import bound_compact_statement; import select_statement; import delete_statement; import update_statement; +import compact_statement; import parsed_expr; import knn_expr; import table_reference; @@ -52,6 +54,8 @@ public: UniquePtr BindUpdate(const UpdateStatement &statement); + UniquePtr BindCompact(const CompactStatement &statement); + SharedPtr GetTableRef(const String &db_name, const String &table_name); QueryContext *query_context_ptr_; diff --git a/src/scheduler/fragment_context.cpp b/src/scheduler/fragment_context.cpp index c9a798d730..b3f6c6f8fb 100644 --- a/src/scheduler/fragment_context.cpp +++ b/src/scheduler/fragment_context.cpp @@ -39,6 +39,10 @@ import physical_create_index_do; import physical_sort; import physical_top; import physical_merge_top; +import physical_compact; +import physical_compact_index_prepare; +import physical_compact_index_do; +import physical_compact_finish; import global_block_id; import knn_expression; @@ -53,6 +57,7 @@ import data_block; import physical_merge_knn; import merge_knn_data; import create_index_data; +import compact_state_data; import logger; import task_scheduler; import plan_fragment; @@ -60,6 +65,8 @@ import aggregate_expression; import expression_state; import column_def; import explain_statement; +import table_entry; +import segment_entry; namespace infinity { @@ -132,6 +139,48 @@ UniquePtr MakeKnnScanState(PhysicalKnnScan *physical_knn_scan, Fr return operator_state; } +UniquePtr MakeCompactState(PhysicalCompact *physical_compact, FragmentTask *task, FragmentContext *fragment_ctx) { + if (fragment_ctx->ContextType() != FragmentType::kParallelMaterialize) { + UnrecoverableError("Compact operator should be in parallel materialized fragment."); + } + auto *parallel_materialize_fragment_ctx = static_cast(fragment_ctx); + auto compact_operator_state = MakeUnique(task->TaskID(), parallel_materialize_fragment_ctx->compact_state_data_); + return compact_operator_state; +} + +UniquePtr +MakeCompactIndexPrepareState(PhysicalCompactIndexPrepare *physical_compact_index_prepare, FragmentTask *task, FragmentContext *fragment_ctx) { + if (fragment_ctx->ContextType() != FragmentType::kSerialMaterialize) { + UnrecoverableError("Compact index prepare operator should be in parallel materialized fragment."); + } + auto *serial_materialize_fragment_ctx = static_cast(fragment_ctx); + auto compact_index_prepare_operator_state = + MakeUnique(serial_materialize_fragment_ctx->compact_state_data_, + serial_materialize_fragment_ctx->create_index_shared_data_array_); + return compact_index_prepare_operator_state; +} + +UniquePtr +MakeCompactIndexDoState(PhysicalCompactIndexDo *physical_compact_index_do, FragmentTask *task, FragmentContext *fragment_ctx) { + if (fragment_ctx->ContextType() != FragmentType::kParallelMaterialize) { + UnrecoverableError("Compact index do operator should be in parallel materialized fragment."); + } + auto *parallel_materialize_fragment_ctx = static_cast(fragment_ctx); + auto compact_index_do_operator_state = + MakeUnique(parallel_materialize_fragment_ctx->compact_state_data_, + parallel_materialize_fragment_ctx->create_index_shared_data_array_); + return compact_index_do_operator_state; +} + +UniquePtr MakeCompactFinishState(PhysicalCompactFinish *physical_compact_finish, FragmentContext *fragment_ctx) { + if (fragment_ctx->ContextType() != FragmentType::kSerialMaterialize) { + UnrecoverableError("Compact finish operator should be in serial materialized fragment."); + } + auto *serial_materialize_fragment_ctx = static_cast(fragment_ctx); + auto compact_finish_operator_state = MakeUnique(serial_materialize_fragment_ctx->compact_state_data_); + return compact_finish_operator_state; +} + UniquePtr MakeAggregateState(PhysicalAggregate *physical_aggregate, FragmentTask *task) { Vector> states; for (auto &expr : physical_aggregate->aggregates_) { @@ -329,6 +378,22 @@ MakeTaskState(SizeT operator_id, const Vector &physical_ops, case PhysicalOperatorType::kCommand: { return MakeTaskStateTemplate(physical_ops[operator_id]); } + case PhysicalOperatorType::kCompact: { + auto *physical_compact = static_cast(physical_ops[operator_id]); + return MakeCompactState(physical_compact, task, fragment_ctx); + } + case PhysicalOperatorType::kCompactIndexPrepare: { + auto *physical_compact_index_prepare = static_cast(physical_ops[operator_id]); + return MakeCompactIndexPrepareState(physical_compact_index_prepare, task, fragment_ctx); + } + case PhysicalOperatorType::kCompactIndexDo: { + auto *physical_compact_index_do = static_cast(physical_ops[operator_id]); + return MakeCompactIndexDoState(physical_compact_index_do, task, fragment_ctx); + } + case PhysicalOperatorType::kCompactFinish: { + auto *physical_compact_finish = static_cast(physical_ops[operator_id]); + return MakeCompactFinishState(physical_compact_finish, fragment_ctx); + } case PhysicalOperatorType::kExplain: { return MakeTaskStateTemplate(physical_ops[operator_id]); } @@ -408,7 +473,7 @@ void FragmentContext::BuildTask(QueryContext *query_context, FragmentContext *pa i64 parallel_size = static_cast(query_context->cpu_number_limit()); // i64 parallel_size = 1; - fragment_context->CreateTasks(parallel_size, operator_count); + fragment_context->CreateTasks(parallel_size, operator_count, parent_context); Vector> &tasks = fragment_context->Tasks(); i64 real_parallel_size = tasks.size(); @@ -495,12 +560,12 @@ FragmentContext::FragmentContext(PlanFragment *fragment_ptr, QueryContext *query bool FragmentContext::TryFinishFragment() { auto fragment_id = fragment_ptr_->FragmentID(); - auto *parent_plan_fragment = fragment_ptr_->GetParent(); + auto parent_plan_fragments = fragment_ptr_->GetParents(); if (!TryFinishFragmentInner()) { LOG_TRACE(fmt::format("{} tasks in fragment {} are not completed", unfinished_task_n_.load(), fragment_id)); if (fragment_type_ == FragmentType::kParallelStream) { - if (parent_plan_fragment) { + for (auto *parent_plan_fragment : parent_plan_fragments) { auto *scheduler = query_context_->scheduler(); LOG_TRACE(fmt::format("Schedule fragment: {} before fragment {} has finished.", parent_plan_fragment->FragmentID(), fragment_id)); scheduler->ScheduleFragment(parent_plan_fragment); @@ -510,7 +575,7 @@ bool FragmentContext::TryFinishFragment() { } else { LOG_TRACE(fmt::format("All tasks in fragment: {} are completed", fragment_id)); - if (parent_plan_fragment != nullptr) { + for (auto *parent_plan_fragment : parent_plan_fragments) { auto *parent_fragment_ctx = parent_plan_fragment->GetContext(); if (parent_fragment_ctx->TryStartFragment()) { // All child fragment are finished. @@ -578,13 +643,80 @@ SizeT InitKnnScanFragmentContext(PhysicalKnnScan *knn_scan_operator, FragmentCon SizeT InitCreateIndexDoFragmentContext(const PhysicalCreateIndexDo *create_index_do_operator, FragmentContext *fragment_ctx) { auto *table_ref = create_index_do_operator->base_table_ref_.get(); // FIXME: to create index on unsealed_segment - SizeT segment_cnt = table_ref->block_index_->segments_.size(); + SizeT segment_cnt = table_ref->block_index_->SegmentCount(); auto *parallel_materialize_fragment_ctx = static_cast(fragment_ctx); parallel_materialize_fragment_ctx->create_index_shared_data_ = MakeUnique(table_ref->block_index_.get()); return segment_cnt; } +SizeT InitCompactFragmentContext(PhysicalCompact *compact_operator, FragmentContext *fragment_context, FragmentContext *parent_context) { + SizeT task_n = compact_operator->TaskletCount(); + if (fragment_context->ContextType() != FragmentType::kParallelMaterialize) { + UnrecoverableError("Compact operator should be in parallel materialized fragment."); + } + auto *parallel_materialize_fragment_ctx = static_cast(fragment_context); + if (parent_context->ContextType() != FragmentType::kSerialMaterialize) { + UnrecoverableError("Compact operator parent should be in serial materialized fragment."); + } + auto *parent_serial_materialize_fragment_ctx = static_cast(parent_context); + + auto &compact_state_data = parent_serial_materialize_fragment_ctx->compact_state_data_; + compact_state_data->segment_data_list_.resize(task_n); + parallel_materialize_fragment_ctx->compact_state_data_ = compact_state_data; + return task_n; +} + +SizeT InitCompactIndexPrepareFragmentContext(PhysicalCompactIndexPrepare *compact_index_prepare_operator, + FragmentContext *fragment_context, + FragmentContext *parent_context) { + SizeT task_n = compact_index_prepare_operator->TaskletCount(); + if (fragment_context->ContextType() != FragmentType::kSerialMaterialize) { + UnrecoverableError("Compact index prepare operator should be in parallel materialized fragment."); + } + auto *serial_materialize_fragment_ctx = static_cast(fragment_context); + if (parent_context->ContextType() == FragmentType::kSerialMaterialize) { + auto *parent_serial_materialize_fragment_ctx = static_cast(parent_context); + serial_materialize_fragment_ctx->compact_state_data_ = parent_serial_materialize_fragment_ctx->compact_state_data_; + } else { + auto *parent_parallel_materialize_fragment_ctx = static_cast(parent_context); + serial_materialize_fragment_ctx->compact_state_data_ = parent_parallel_materialize_fragment_ctx->compact_state_data_; + serial_materialize_fragment_ctx->create_index_shared_data_array_ = parent_parallel_materialize_fragment_ctx->create_index_shared_data_array_; + } + + return task_n; +} + +void InitCompactIndexDoFragmentContext(PhysicalCompactIndexDo *compact_index_do_operator, + FragmentContext *fragment_context, + FragmentContext *parent_context) { + if (fragment_context->ContextType() != FragmentType::kParallelMaterialize) { + UnrecoverableError("Compact index do operator should be in parallel materialized fragment."); + } + auto *parallel_materialize_fragment_ctx = static_cast(fragment_context); + if (parent_context->ContextType() != FragmentType::kSerialMaterialize) { + UnrecoverableError("Compact index do operator parent should be in serial materialized fragment."); + } + auto *parent_serial_materialize_fragment_ctx = static_cast(parent_context); + parallel_materialize_fragment_ctx->compact_state_data_ = parent_serial_materialize_fragment_ctx->compact_state_data_; + + auto *table_ref = compact_index_do_operator->base_table_ref_.get(); + SizeT index_size = table_ref->index_index_->index_snapshots_vec_.size(); + parallel_materialize_fragment_ctx->create_index_shared_data_array_ = MakeShared>>(); + for (SizeT i = 0; i < index_size; ++i) { + parallel_materialize_fragment_ctx->create_index_shared_data_array_->emplace_back(MakeUnique()); + } +} + +void InitCompactFinishFragmentContext(PhysicalCompactFinish *compact_finish_operator, FragmentContext *fragment_context) { + if (fragment_context->ContextType() != FragmentType::kSerialMaterialize) { + UnrecoverableError("Compact finish operator should be in serial materialized fragment."); + } + auto *serial_materialize_fragment_ctx = static_cast(fragment_context); + TableEntry *table_entry = compact_finish_operator->base_table_ref_->table_entry_ptr_; + serial_materialize_fragment_ctx->compact_state_data_ = MakeShared(table_entry); +} + void FragmentContext::MakeSourceState(i64 parallel_count) { PhysicalOperator *first_operator = this->GetOperators().back(); switch (first_operator->operator_type()) { @@ -610,7 +742,7 @@ void FragmentContext::MakeSourceState(i64 parallel_count) { break; } case PhysicalOperatorType::kProjection: { - if(this->GetOperators().size() == 1) { + if (this->GetOperators().size() == 1) { // Only one operator and it's project tasks_[0]->source_state_ = MakeUnique(); } else { @@ -649,7 +781,9 @@ void FragmentContext::MakeSourceState(i64 parallel_count) { tasks_[0]->source_state_ = MakeUnique(); break; } - case PhysicalOperatorType::kCreateIndexDo: { + case PhysicalOperatorType::kCreateIndexDo: + case PhysicalOperatorType::kCompact: + case PhysicalOperatorType::kCompactIndexDo: { if (fragment_type_ != FragmentType::kParallelMaterialize) { UnrecoverableError( fmt::format("{} should in parallel materialized fragment", PhysicalOperatorToString(first_operator->operator_type()))); @@ -733,7 +867,9 @@ void FragmentContext::MakeSourceState(i64 parallel_count) { case PhysicalOperatorType::kShow: case PhysicalOperatorType::kMatch: case PhysicalOperatorType::kOptimize: - case PhysicalOperatorType::kFlush: { + case PhysicalOperatorType::kFlush: + case PhysicalOperatorType::kCompactFinish: + case PhysicalOperatorType::kCompactIndexPrepare: { if (fragment_type_ != FragmentType::kSerialMaterialize) { UnrecoverableError( fmt::format("{} should in serial materialized fragment", PhysicalOperatorToString(first_operator->operator_type()))); @@ -945,7 +1081,8 @@ void FragmentContext::MakeSinkState(i64 parallel_count) { } case PhysicalOperatorType::kInsert: case PhysicalOperatorType::kImport: - case PhysicalOperatorType::kExport: { + case PhysicalOperatorType::kExport: + case PhysicalOperatorType::kCompactIndexPrepare: { if (fragment_type_ != FragmentType::kSerialMaterialize) { UnrecoverableError( fmt::format("{} should in serial materialized fragment", PhysicalOperatorToString(last_operator->operator_type()))); @@ -958,7 +1095,9 @@ void FragmentContext::MakeSinkState(i64 parallel_count) { tasks_[0]->sink_state_ = MakeUnique(); break; } - case PhysicalOperatorType::kCreateIndexDo: { + case PhysicalOperatorType::kCreateIndexDo: + case PhysicalOperatorType::kCompact: + case PhysicalOperatorType::kCompactIndexDo: { if (fragment_type_ != FragmentType::kParallelMaterialize) { UnrecoverableError( fmt::format("{} should in parallel materialized fragment", PhysicalOperatorToString(last_operator->operator_type()))); @@ -980,7 +1119,8 @@ void FragmentContext::MakeSinkState(i64 parallel_count) { case PhysicalOperatorType::kDropDatabase: case PhysicalOperatorType::kDropView: case PhysicalOperatorType::kOptimize: - case PhysicalOperatorType::kFlush: { + case PhysicalOperatorType::kFlush: + case PhysicalOperatorType::kCompactFinish: { if (fragment_type_ != FragmentType::kSerialMaterialize) { UnrecoverableError( fmt::format("{} should in serial materialized fragment", PhysicalOperatorToString(last_operator->operator_type()))); @@ -1000,7 +1140,7 @@ void FragmentContext::MakeSinkState(i64 parallel_count) { } // Allocate tasks for the fragment and determine the sink and source -void FragmentContext::CreateTasks(i64 cpu_count, i64 operator_count) { +void FragmentContext::CreateTasks(i64 cpu_count, i64 operator_count, FragmentContext *parent_context) { i64 parallel_count = cpu_count; PhysicalOperator *first_operator = this->GetOperators().back(); switch (first_operator->operator_type()) { @@ -1042,6 +1182,35 @@ void FragmentContext::CreateTasks(i64 cpu_count, i64 operator_count) { parallel_count = std::max(parallel_count, 1l); break; } + case PhysicalOperatorType::kCompact: { + auto *compact_operator = static_cast(first_operator); + SizeT task_n = InitCompactFragmentContext(compact_operator, this, parent_context); + parallel_count = std::min(parallel_count, (i64)task_n); + if (parallel_count == 0) { + parallel_count = 1; + } + break; + } + case PhysicalOperatorType::kCompactIndexPrepare: { + auto *compact_index_prepare_operator = static_cast(first_operator); + SizeT task_n = InitCompactIndexPrepareFragmentContext(compact_index_prepare_operator, this, parent_context); + parallel_count = std::min(parallel_count, (i64)task_n); + if (parallel_count == 0) { + parallel_count = 1; + } + break; + } + case PhysicalOperatorType::kCompactIndexDo: { + auto *compact_index_do_operator = static_cast(first_operator); + InitCompactIndexDoFragmentContext(compact_index_do_operator, this, parent_context); + parallel_count = std::max(parallel_count, 1l); + break; + } + case PhysicalOperatorType::kCompactFinish: { + auto *compact_finish_operator = static_cast(first_operator); + InitCompactFinishFragmentContext(compact_finish_operator, this); + parallel_count = 1; + } default: { break; } diff --git a/src/scheduler/fragment_context.cppm b/src/scheduler/fragment_context.cppm index a39c42b31d..2d18c971a6 100644 --- a/src/scheduler/fragment_context.cppm +++ b/src/scheduler/fragment_context.cppm @@ -27,6 +27,7 @@ import knn_scan_data; import create_index_data; import logger; import third_party; +import compact_state_data; export module fragment_context; @@ -133,7 +134,7 @@ public: [[nodiscard]] PhysicalSource *GetSourceOperator() const; - void CreateTasks(i64 parallel_count, i64 operator_count); + void CreateTasks(i64 parallel_count, i64 operator_count, FragmentContext *parent_context); inline Vector> &Tasks() { return tasks_; } @@ -207,6 +208,10 @@ public: public: UniquePtr knn_scan_shared_data_{}; + + SharedPtr>> create_index_shared_data_array_{}; + + SharedPtr compact_state_data_{}; }; export class ParallelMaterializedFragmentCtx final : public FragmentContext { @@ -222,6 +227,9 @@ public: UniquePtr knn_scan_shared_data_{}; UniquePtr create_index_shared_data_{}; + SharedPtr>> create_index_shared_data_array_{}; + + SharedPtr compact_state_data_{}; protected: HashMap>> task_results_{}; diff --git a/src/scheduler/task_scheduler.cpp b/src/scheduler/task_scheduler.cpp index 82a8e08249..7e73fe3f5e 100644 --- a/src/scheduler/task_scheduler.cpp +++ b/src/scheduler/task_scheduler.cpp @@ -37,6 +37,7 @@ import physical_sink; import base_statement; import extra_ddl_info; import create_statement; +import command_statement; namespace infinity { @@ -97,24 +98,6 @@ u64 TaskScheduler::FindLeastWorkloadWorker() { return min_workload_worker_id; } -SizeT TaskScheduler::GetStartFragments(PlanFragment *plan_fragment, Vector &leaf_fragments) { - SizeT all_fragment_n = 0; - std::function TraversePlanFragmentTree = [&](PlanFragment *root) -> void { - all_fragment_n += root->GetContext()->Tasks().size(); - if (root->Children().empty()) { - leaf_fragments.emplace_back(root); - return; - } - for (auto &child : root->Children()) { - TraversePlanFragmentTree(child.get()); - } - }; - // Traverse the tree to get all leaf fragments - TraversePlanFragmentTree(plan_fragment); - - return all_fragment_n; -} - void TaskScheduler::Schedule(PlanFragment *plan_fragment, const BaseStatement *base_statement) { if (!initialized_) { UnrecoverableError("Scheduler isn't initialized"); @@ -125,8 +108,8 @@ void TaskScheduler::Schedule(PlanFragment *plan_fragment, const BaseStatement *b case StatementType::kSelect: case StatementType::kExplain: case StatementType::kDelete: - case StatementType::kUpdate: - { + case StatementType::kUpdate: + case StatementType::kCompact:{ use_scheduler = true; // continue; break; } @@ -158,7 +141,7 @@ void TaskScheduler::Schedule(PlanFragment *plan_fragment, const BaseStatement *b } Vector start_fragments; - SizeT task_n = GetStartFragments(plan_fragment, start_fragments); + SizeT task_n = plan_fragment->GetStartFragments(start_fragments); plan_fragment->GetContext()->notifier()->SetTaskN(task_n); for (auto *sub_fragment : start_fragments) { auto &tasks = sub_fragment->GetContext()->Tasks(); diff --git a/src/scheduler/task_scheduler.cppm b/src/scheduler/task_scheduler.cppm index 96b5bcb267..197ddb22de 100644 --- a/src/scheduler/task_scheduler.cppm +++ b/src/scheduler/task_scheduler.cppm @@ -58,8 +58,6 @@ public: private: u64 FindLeastWorkloadWorker(); - SizeT GetStartFragments(PlanFragment* plan_fragment, Vector& leaf_fragments); - void ScheduleTask(FragmentTask *task, u64 worker_id); void RunTask(FragmentTask *task); diff --git a/src/storage/background_process.cpp b/src/storage/background_process.cpp index 6c1e08bfc6..db6997b8f2 100644 --- a/src/storage/background_process.cpp +++ b/src/storage/background_process.cpp @@ -20,7 +20,6 @@ module background_process; import stl; import bg_task; -import compact_segments_task; import update_segment_bloom_filter_task; import logger; import blocking_queue; diff --git a/src/storage/bg_task/compact_segments_task.cpp b/src/storage/bg_task/compact_segments_task.cpp deleted file mode 100644 index 5626d93237..0000000000 --- a/src/storage/bg_task/compact_segments_task.cpp +++ /dev/null @@ -1,327 +0,0 @@ -// 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 -#include - -module compact_segments_task; - -import stl; -import catalog; -import third_party; -import default_values; -import infinity_exception; -import logger; -import data_access_state; -import column_vector; -import buffer_manager; -import txn; -import txn_state; -import txn_manager; -import infinity_exception; -import bg_task; -import wal_manager; -import wal_entry; -import global_block_id; -import block_index; -import segment_iter; -import segment_entry; -import table_index_entry; -import table_index_meta; -import block_entry; -import compaction_alg; -import status; -import build_fast_rough_filter_task; -import catalog_delta_entry; - -namespace infinity { - -RowID RowIDRemapper::GetNewRowID(SegmentID segment_id, BlockID block_id, BlockOffset block_offset) const { - auto &block_vec = row_id_map_.at(GlobalBlockID(segment_id, block_id)); - auto iter = std::upper_bound(block_vec.begin(), - block_vec.end(), - block_offset, - [](BlockOffset block_offset, const Pair &pair) { return block_offset < pair.first; } // NOLINT - ); - if (iter == block_vec.begin()) { - UnrecoverableError("RowID not found"); - } - --iter; - RowID rtn = iter->second; - rtn.segment_offset_ += iter->first - block_offset; - return rtn; -} - -class GreedyCompactableSegmentsGenerator { -public: - GreedyCompactableSegmentsGenerator(const Vector &segments, SizeT max_segment_size) : max_segment_size_(max_segment_size) { - for (auto *segment_entry : segments) { - segments_.emplace(segment_entry->actual_row_count(), segment_entry); - } - } - - // find the largest segment to fill the free space - Vector generate() { - Vector result; - do { - result.clear(); - SizeT segment_size = max_segment_size_; - - while (true) { - auto iter = segments_.upper_bound(segment_size); - if (iter == segments_.begin()) { - break; - } - --iter; - auto [row_count, segment_entry] = *iter; - segments_.erase(iter); - result.push_back(segment_entry); - segment_size -= row_count; - } - } while (result.size() == 1 && (result[0]->actual_row_count() == result[0]->row_count())); - // FIXME: compact single segment with too much delete row - return result; - } - -private: - MultiMap segments_; // TODO(opt): use Map replace MultiMap - - const SizeT max_segment_size_; -}; - -UniquePtr CompactSegmentsTask::MakeTaskWithPickedSegments(TableEntry *table_entry, Vector &&segments, Txn *txn) { - if (segments.empty()) { - UnrecoverableError("No segment to compact"); - } - { - HashSet segment_ids; - for (auto *segment : segments) { - if (!segment_ids.insert(segment->segment_id()).second) { - UnrecoverableError("Duplicate segment to compact"); - } - } - } - LOG_TRACE(fmt::format("Add compact task, picked, table dir: {}, begin ts: {}", *table_entry->TableEntryDir(), txn->BeginTS())); - return MakeUnique(table_entry, std::move(segments), txn, CompactSegmentsTaskType::kCompactPickedSegments); -} - -UniquePtr CompactSegmentsTask::MakeTaskWithWholeTable(TableEntry *table_entry, Txn *txn) { - Vector segments = table_entry->PickCompactSegments(); // wait auto compaction to finish and pick segments - LOG_TRACE(fmt::format("Add compact task, whole, table dir: {}, begin ts: {}", *table_entry->TableEntryDir(), txn->BeginTS())); - return MakeUnique(table_entry, std::move(segments), txn, CompactSegmentsTaskType::kCompactTable); -} - -CompactSegmentsTask::CompactSegmentsTask(TableEntry *table_entry, Vector &&segments, Txn *txn, CompactSegmentsTaskType type) - : task_type_(type), table_entry_(table_entry), commit_ts_(table_entry->commit_ts_), segments_(std::move(segments)), txn_(txn) {} - -void CompactSegmentsTask::Execute() { - CompactSegmentsTaskState state; - CompactSegments(state); - CreateNewIndex(state); - SaveSegmentsData(state); - ApplyDeletes(state); -} - -// generate new_table_ref_ to compact -void CompactSegmentsTask::CompactSegments(CompactSegmentsTaskState &state) { - auto &segment_data = state.segment_data_; - auto &old_segments = state.old_segments_; - - auto block_index = MakeShared(); - auto DoCompact = [&](const Vector &to_compact_segments) { - if (to_compact_segments.empty()) { - return; - } - - auto new_segment = CompactSegmentsToOne(state, to_compact_segments); - block_index->Insert(new_segment.get(), txn_); - - segment_data.emplace_back(new_segment, std::move(to_compact_segments)); - old_segments.insert(old_segments.end(), to_compact_segments.begin(), to_compact_segments.end()); - }; - - switch (task_type_) { - case CompactSegmentsTaskType::kCompactTable: { - Vector to_compact_segments; - for (auto *segment : segments_) { - if (segment->TrySetCompacting(this)) { - to_compact_segments.push_back(segment); - } - } - GreedyCompactableSegmentsGenerator generator(to_compact_segments, DEFAULT_SEGMENT_CAPACITY); - - while (true) { - Vector to_compact_segments = generator.generate(); - if (to_compact_segments.empty()) { - break; - } - DoCompact(to_compact_segments); - } - break; - } - case CompactSegmentsTaskType::kCompactPickedSegments: { - Vector to_compact_segments; - for (auto *segment : segments_) { - if (!segment->TrySetCompacting(this)) { - UnrecoverableError("Picked segment should be compactable"); - } - to_compact_segments.push_back(segment); - } - if (to_compact_segments.empty()) { - UnrecoverableError("No segment to compact"); - } - DoCompact(to_compact_segments); - break; - } - default: { - UnrecoverableError("Unknown compact segments task type"); - } - } - - // FIXME: fake table ref here - state.new_table_ref_ = MakeUnique(table_entry_, block_index); -} - -void CompactSegmentsTask::CreateNewIndex(CompactSegmentsTaskState &state) { - BaseTableRef *new_table_ref = state.new_table_ref_.get(); - auto *table_entry = new_table_ref->table_entry_ptr_; - TransactionID txn_id = txn_->TxnID(); - TxnTimeStamp begin_ts = txn_->BeginTS(); - - { - auto map_guard = table_entry->IndexMetaMap(); - for (auto &[index_name, table_index_meta] : *map_guard) { - auto [table_index_entry, status] = table_index_meta->GetEntryNolock(txn_id, begin_ts); - if (!status.ok()) { - if (status.code() == ErrorCode::kIndexNotExist) { - continue; // the index entry is not committed. - } else { - UnrecoverableError("Get index entry failed"); - } - } - status = txn_->CreateIndexPrepare(table_index_entry, new_table_ref, false /*prepare*/, false /*check_ts*/); - if (!status.ok()) { - UnrecoverableError("Create index prepare failed"); - } - } - } -} - -void CompactSegmentsTask::SaveSegmentsData(CompactSegmentsTaskState &state) { - auto *table_entry = table_entry_; - auto segment_data = std::move(state.segment_data_); - - Vector segment_infos; - Vector old_segment_ids; - - for (auto &[new_segment, old_segments] : segment_data) { - if (new_segment->row_count() > 0) { - new_segment->FlushNewData(); - segment_infos.push_back(WalSegmentInfo(new_segment.get())); - } - - for (auto *old_segment : old_segments) { - old_segment_ids.push_back(old_segment->segment_id()); - } - } - txn_->Compact(table_entry, std::move(segment_data), task_type_); - String db_name = *table_entry->GetDBName(); - String table_name = *table_entry->GetTableName(); - txn_->AddWalCmd(MakeShared(std::move(db_name), std::move(table_name), std::move(segment_infos), std::move(old_segment_ids))); -} - -void CompactSegmentsTask::ApplyDeletes(CompactSegmentsTaskState &state) { - const auto &remapper = state.remapper_; - const auto &old_segments = state.old_segments_; - - for (auto *old_segment : old_segments) { - old_segment->SetNoDelete(); - } - - Vector row_ids; - for (const auto &delete_info : to_deletes_) { - for (SegmentOffset offset : delete_info.delete_offsets_) { - RowID old_row_id(delete_info.segment_id_, offset); - RowID new_row_id = remapper.GetNewRowID(old_row_id); - row_ids.push_back(new_row_id); - } - } - txn_->Delete(table_entry_, row_ids, false); -} - -void CompactSegmentsTask::AddToDelete(SegmentID segment_id, Vector &&delete_offsets) { - std::unique_lock lock(mutex_); - to_deletes_.emplace_back(ToDeleteInfo{segment_id, std::move(delete_offsets)}); -} - -const String &CompactSegmentsTask::table_name() const { return *table_entry_->GetTableName(); } - -SharedPtr CompactSegmentsTask::CompactSegmentsToOne(CompactSegmentsTaskState &state, const Vector &segments) { - auto *table_entry = table_entry_; - auto &remapper = state.remapper_; - auto new_segment = SegmentEntry::NewSegmentEntry(table_entry, Catalog::GetNextSegmentID(table_entry), txn_); - - TxnTimeStamp begin_ts = txn_->BeginTS(); - SizeT column_count = table_entry->ColumnCount(); - BufferManager *buffer_mgr = txn_->buffer_mgr(); - - auto new_block = BlockEntry::NewBlockEntry(new_segment.get(), 0, 0, column_count, txn_); - for (auto *old_segment : segments) { - BlockEntryIter block_entry_iter(old_segment); // TODO: compact segment should be sealed. use better way to iterate block - for (auto *old_block = block_entry_iter.Next(); old_block != nullptr; old_block = block_entry_iter.Next()) { - Vector input_column_vectors; - for (ColumnID column_id = 0; column_id < column_count; ++column_id) { - auto *column_block_entry = old_block->GetColumnBlockEntry(column_id); - input_column_vectors.emplace_back(column_block_entry->GetColumnVector(buffer_mgr)); - } - SizeT read_offset = 0; - while (true) { - // The delete ops after begin_ts is not visible and must in to_delete - auto [row_begin, row_end] = old_block->GetVisibleRange(begin_ts, read_offset); - SizeT read_size = row_end - row_begin; - if (read_size == 0) { - break; - } - - auto block_entry_append = [&](SizeT row_begin, SizeT read_size) { - new_block->AppendBlock(input_column_vectors, row_begin, read_size, buffer_mgr); - RowID new_row_id(new_segment->segment_id(), new_block->block_id() * DEFAULT_BLOCK_CAPACITY + new_block->row_count()); - remapper.AddMap(old_segment->segment_id(), old_block->block_id(), row_begin, new_row_id); - read_offset = row_begin + read_size; - }; - - if (read_size + new_block->row_count() > new_block->row_capacity()) { - SizeT read_size1 = new_block->row_capacity() - new_block->row_count(); - block_entry_append(row_begin, read_size1); - row_begin += read_size1; - read_size -= read_size1; - new_segment->AppendBlockEntry(std::move(new_block)); - - new_block = BlockEntry::NewBlockEntry(new_segment.get(), new_segment->GetNextBlockID(), 0, column_count, txn_); - } - block_entry_append(row_begin, read_size); - } - } - } - if (new_block->row_count() > 0) { - new_segment->AppendBlockEntry(std::move(new_block)); - } - - return new_segment; -} - -} // namespace infinity \ No newline at end of file diff --git a/src/storage/bg_task/compact_segments_task.cppm b/src/storage/bg_task/compact_segments_task.cppm deleted file mode 100644 index ec90f3567e..0000000000 --- a/src/storage/bg_task/compact_segments_task.cppm +++ /dev/null @@ -1,138 +0,0 @@ -// 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; - -export module compact_segments_task; - -import stl; -import bg_task; -import default_values; -import infinity_exception; -import txn; -import global_block_id; -import base_table_ref; -import internal_types; - -namespace infinity { - -class TableEntry; -class SegmentEntry; - -class RowIDRemapper { -private: - using RowIDMap = HashMap>, GlobalBlockIDHash>; - -public: - RowIDRemapper(SizeT block_capacity = DEFAULT_BLOCK_CAPACITY) : block_capacity_(block_capacity) {} - - void AddMap(SegmentID segment_id, BlockID block_id, BlockOffset block_offset, RowID new_row_id) { - auto &block_vec = row_id_map_[GlobalBlockID(segment_id, block_id)]; - block_vec.emplace_back(block_offset, new_row_id); - } - - RowID GetNewRowID(SegmentID segment_id, BlockID block_id, BlockOffset block_offset) const; - - void AddMap(RowID old_row_id, RowID new_row_id) { - AddMap(old_row_id.segment_id_, old_row_id.segment_offset_ / block_capacity_, old_row_id.segment_offset_ % block_capacity_, new_row_id); - } - - RowID GetNewRowID(RowID old_row_id) const { - return GetNewRowID(old_row_id.segment_id_, old_row_id.segment_offset_ / block_capacity_, old_row_id.segment_offset_ % block_capacity_); - } - -private: - const SizeT block_capacity_; - - RowIDMap row_id_map_; -}; - -struct ToDeleteInfo { - const SegmentID segment_id_; - - const Vector delete_offsets_; -}; - -export struct CompactSegmentsTaskState { - RowIDRemapper remapper_; - - Vector, Vector>> segment_data_; - Vector old_segments_; - - UniquePtr new_table_ref_ = nullptr; -}; - -export enum class CompactSegmentsTaskType : i8 { - kCompactTable, - kCompactPickedSegments, - kInvalid, -}; - -export class CompactSegmentsTask final { -public: - static UniquePtr MakeTaskWithPickedSegments(TableEntry *table_entry, Vector &&segments, Txn *txn); - - static UniquePtr MakeTaskWithWholeTable(TableEntry *table_entry, Txn *txn); - - explicit CompactSegmentsTask(TableEntry *table_entry, Vector &&segments, Txn *txn, CompactSegmentsTaskType type); - -public: - bool TryCommitTxn() { - try { - txn_->txn_mgr()->CommitTxn(txn_); - return true; - } catch (const RecoverableException &e) { - txn_->txn_mgr()->RollBackTxn(txn_); - return false; - } - } - - void Execute(); - - // Called by `SegmentEntry::DeleteData` which is called by wal thread in - // So to_deletes_ is thread-safe. - // TODO: remove lock - void AddToDelete(SegmentID segment_id, Vector &&delete_offsets); - - // these functions are called by unit test. Do not use them directly. -public: - void CompactSegments(CompactSegmentsTaskState &state); - - void CreateNewIndex(CompactSegmentsTaskState &state); - - // Save new segment, set no_delete_ts, add compact wal cmd - void SaveSegmentsData(CompactSegmentsTaskState &state); - - // Apply the delete op commit in process of compacting - void ApplyDeletes(CompactSegmentsTaskState &state); - -public: - // Getter - const String &table_name() const; - -private: - SharedPtr CompactSegmentsToOne(CompactSegmentsTaskState &state, const Vector &segments); - -private: - const CompactSegmentsTaskType task_type_; - TableEntry *table_entry_; - TxnTimeStamp commit_ts_; - Vector segments_; - - Txn *const txn_; - - std::mutex mutex_; - Vector to_deletes_; -}; -} // namespace infinity \ No newline at end of file diff --git a/src/storage/bg_task/segment_sealing_tasks/update_segment_bloom_filter_task.cpp b/src/storage/bg_task/segment_sealing_tasks/update_segment_bloom_filter_task.cpp index bff74a9328..bfe49928fc 100644 --- a/src/storage/bg_task/segment_sealing_tasks/update_segment_bloom_filter_task.cpp +++ b/src/storage/bg_task/segment_sealing_tasks/update_segment_bloom_filter_task.cpp @@ -28,7 +28,6 @@ import logger; import wal_entry; import third_party; import build_fast_rough_filter_task; -import compact_segments_task; import catalog_delta_entry; namespace infinity { diff --git a/src/storage/common/block_index.cpp b/src/storage/common/block_index.cpp index 4f6a1b5870..4666c7e81b 100644 --- a/src/storage/common/block_index.cpp +++ b/src/storage/common/block_index.cpp @@ -21,40 +21,32 @@ module block_index; import stl; import segment_entry; import global_block_id; -import block_iter; import txn; +import table_index_entry; +import segment_index_entry; namespace infinity { void BlockIndex::Insert(SegmentEntry *segment_entry, Txn *txn) { if (segment_entry->CheckVisible(txn)) { - u32 segment_id = segment_entry->segment_id(); - segments_.emplace_back(segment_entry); - segment_index_.emplace(segment_id, segment_entry); - BlocksInfo blocks_info; - + SegmentSnapshot segment_info; + segment_info.segment_entry_ = segment_entry; { auto block_guard = segment_entry->GetBlocksGuard(); for (const auto &block_entry : block_guard.block_entries_) { if (block_entry->CheckVisible(txn)) { - blocks_info.block_map_.emplace(block_entry->block_id(), block_entry.get()); - global_blocks_.emplace_back(GlobalBlockID{segment_id, block_entry->block_id()}); + segment_info.block_map_.emplace_back(block_entry.get()); } } } TxnTimeStamp begin_ts = txn->BeginTS(); - blocks_info.segment_offset_ = segment_entry->row_count(begin_ts); + segment_info.segment_offset_ = segment_entry->row_count(begin_ts); - segment_block_index_.emplace(segment_id, std::move(blocks_info)); + SegmentID segment_id = segment_entry->segment_id(); + segment_block_index_.emplace(segment_id, std::move(segment_info)); } } -void BlockIndex::Reserve(SizeT n) { - segments_.reserve(n); - segment_index_.reserve(n); - segment_block_index_.reserve(n); -} - SegmentOffset BlockIndex::GetSegmentOffset(SegmentID segment_id) const { auto seg_it = segment_block_index_.find(segment_id); if (seg_it != segment_block_index_.end()) { @@ -66,14 +58,51 @@ SegmentOffset BlockIndex::GetSegmentOffset(SegmentID segment_id) const { BlockEntry *BlockIndex::GetBlockEntry(u32 segment_id, u16 block_id) const { auto seg_it = segment_block_index_.find(segment_id); - if (seg_it != segment_block_index_.end()) { - const auto &blocks_info = seg_it->second; - auto block_it = blocks_info.block_map_.find(block_id); - if (block_it != blocks_info.block_map_.end()) { - return block_it->second; + if (seg_it == segment_block_index_.end()) { + return nullptr; + } + const auto &blocks_info = seg_it->second; + if (blocks_info.block_map_.size() <= block_id) { + return nullptr; + } + return blocks_info.block_map_[block_id]; +} + +void IndexIndex::Insert(String index_name, SharedPtr index_snapshot) { + index_snapshots_vec_.push_back(index_snapshot.get()); + index_snapshots_.emplace(std::move(index_name), index_snapshot); +} + +void IndexIndex::Insert(TableIndexEntry *table_index_entry, Txn *txn) { + if (table_index_entry->CheckVisible(txn)) { + auto index_snapshot = MakeUnique(); + index_snapshot->table_index_entry_ = table_index_entry; + + SegmentIndexesGuard segment_index_guard = table_index_entry->GetSegmentIndexesGuard(); + for (const auto &[segment_id, segment_index_entry] : segment_index_guard.index_by_segment_) { + if (segment_index_entry->CheckVisible(txn)) { + index_snapshot->segment_index_entries_.emplace(segment_id, segment_index_entry.get()); + } } + + String index_name = *table_index_entry->GetIndexName(); + Insert(std::move(index_name), std::move(index_snapshot)); + } +} + +void IndexIndex::Insert(TableIndexEntry *table_index_entry, SegmentIndexEntry *segment_index_entry) { + auto index_snapshot_it = index_snapshots_.find(*table_index_entry->GetIndexName()); + IndexSnapshot *index_snapshot_ptr = nullptr; + if (index_snapshot_it == index_snapshots_.end()) { + auto index_snapshot = MakeUnique(); + index_snapshot_ptr = index_snapshot.get(); + index_snapshot->table_index_entry_ = table_index_entry; + String index_name = *table_index_entry->GetIndexName(); + Insert(std::move(index_name), std::move(index_snapshot)); + } else { + index_snapshot_ptr = index_snapshot_it->second.get(); } - return nullptr; + index_snapshot_ptr->segment_index_entries_.emplace(segment_index_entry->segment_id(), segment_index_entry); } } // namespace infinity diff --git a/src/storage/common/block_index.cppm b/src/storage/common/block_index.cppm index 11a7b7b9ad..13a0469b2e 100644 --- a/src/storage/common/block_index.cppm +++ b/src/storage/common/block_index.cppm @@ -21,36 +21,61 @@ export module block_index; namespace infinity { -struct BlockEntry; struct SegmentEntry; +struct BlockEntry; +struct TableIndexEntry; +struct SegmentIndexEntry; class Txn; -export struct BlockIndex { -private: - struct BlocksInfo { - HashMap block_map_; - SegmentOffset segment_offset_ = 0; - }; +export struct SegmentSnapshot { + SegmentEntry *segment_entry_{}; + + Vector block_map_; + + SegmentOffset segment_offset_ = 0; +}; +export struct BlockIndex { public: void Insert(SegmentEntry *segment_entry, Txn *txn); - void Reserve(SizeT n); - - inline SizeT BlockCount() const { return global_blocks_.size(); } + inline SizeT BlockCount() const { + SizeT count = 0; + for (const auto &[_, segment_info] : segment_block_index_) { + count += segment_info.block_map_.size(); + } + return count; + } - inline SizeT SegmentCount() const { return segments_.size(); } + inline SizeT SegmentCount() const { return segment_block_index_.size(); } BlockEntry *GetBlockEntry(u32 segment_id, u16 block_id) const; SegmentOffset GetSegmentOffset(SegmentID segment_id) const; - Vector segments_; - HashMap segment_index_; - Vector global_blocks_; +public: + Map segment_block_index_; +}; -private: - HashMap segment_block_index_; +export struct IndexSnapshot { + TableIndexEntry *table_index_entry_; + + Map segment_index_entries_; +}; + +export struct IndexIndex { +public: + void Insert(TableIndexEntry *table_index_entry, Txn *txn); + + void Insert(String index_name, SharedPtr index_snapshot); + + void Insert(TableIndexEntry *table_index_entry, SegmentIndexEntry *segment_index_entry); + + bool IsEmpty() const { return index_snapshots_.empty(); } + +public: + HashMap> index_snapshots_; + Vector index_snapshots_vec_; }; } // namespace infinity diff --git a/src/storage/compaction/DBT_compaction_alg.cpp b/src/storage/compaction/DBT_compaction_alg.cpp index 293b5dfdee..67fa686ef3 100644 --- a/src/storage/compaction/DBT_compaction_alg.cpp +++ b/src/storage/compaction/DBT_compaction_alg.cpp @@ -86,11 +86,11 @@ SegmentEntry *SegmentLayer::FindSegment(SegmentID segment_id) { return nullptr; } -Optional DBTCompactionAlg::CheckCompaction(std::function generate_txn) { +Vector DBTCompactionAlg::CheckCompaction(TransactionID txn_id) { std::unique_lock lock(mtx_); if (status_ == CompactionStatus::kDisable) { - return None; + return {}; } int cur_layer_n = segment_layers_.size(); @@ -100,15 +100,13 @@ Optional DBTCompactionAlg::CheckCompaction(std::functionTxnID(); Vector compact_segments = segment_layer.PickCompacting(txn_id, config_.m_); txn_2_layer_.emplace(txn_id, layer); - return CompactionInfo(std::move(compact_segments), txn); + return compact_segments; } } - return None; + return {}; } void DBTCompactionAlg::AddSegment(SegmentEntry *new_segment) { diff --git a/src/storage/compaction/DBT_compaction_alg.cppm b/src/storage/compaction/DBT_compaction_alg.cppm index 63fcfc3711..6b1353b053 100644 --- a/src/storage/compaction/DBT_compaction_alg.cppm +++ b/src/storage/compaction/DBT_compaction_alg.cppm @@ -78,7 +78,7 @@ public: : CompactionAlg(), config_(m, c, s), max_layer_(config_.CalculateLayer(max_segment_capacity)), table_entry_(table_entry), running_task_n_(0) { } - virtual Optional CheckCompaction(std::function generate_txn) override; + virtual Vector CheckCompaction(TransactionID txn_id) override; virtual void AddSegment(SegmentEntry *new_segment) override; diff --git a/src/storage/compaction/compaction_alg.cppm b/src/storage/compaction/compaction_alg.cppm index c97566fec4..b3657d638e 100644 --- a/src/storage/compaction/compaction_alg.cppm +++ b/src/storage/compaction/compaction_alg.cppm @@ -41,14 +41,6 @@ export enum class CompactionStatus : u8 { So lock is needed */ -export struct CompactionInfo { -public: - CompactionInfo(Vector &&segments, Txn *txn) : segments_(std::move(segments)), txn_(txn) {} - - Vector segments_; - Txn *txn_; -}; - export class CompactionAlg { public: CompactionAlg() : status_(CompactionStatus::kDisable) {} @@ -56,7 +48,7 @@ public: public: virtual ~CompactionAlg() = default; - virtual Optional CheckCompaction(std::function generate_txn) = 0; + virtual Vector CheckCompaction(TransactionID txn_id) = 0; virtual void AddSegment(SegmentEntry *new_segment) = 0; diff --git a/src/storage/compaction_process.cpp b/src/storage/compaction_process.cpp index 7dda5d995c..7b90e77a12 100644 --- a/src/storage/compaction_process.cpp +++ b/src/storage/compaction_process.cpp @@ -20,7 +20,6 @@ module compaction_process; import stl; import bg_task; -import compact_segments_task; import catalog; import txn_manager; import db_entry; @@ -29,10 +28,48 @@ import logger; import infinity_exception; import third_party; import blocking_queue; +import bg_query_state; +import query_context; +import infinity_context; +import compact_statement; +import session; +import compilation_config; +import defer_op; namespace infinity { -CompactionProcessor::CompactionProcessor(Catalog *catalog, TxnManager *txn_mgr) : catalog_(catalog), txn_mgr_(txn_mgr) {} +struct BGQueryContextWrapper { + UniquePtr query_context_; + + SessionManager *session_mgr_; + SharedPtr session_; + + BGQueryContextWrapper(BGQueryContextWrapper &&other) + : query_context_(std::move(other.query_context_)), session_mgr_(other.session_mgr_), session_(std::move(other.session_)) { + other.session_mgr_ = nullptr; + } + + BGQueryContextWrapper(Txn *txn, SessionManager *session_mgr) : session_mgr_(session_mgr) { + session_ = session_mgr_->CreateLocalSession(); + query_context_ = MakeUnique(session_.get()); + query_context_->Init(InfinityContext::instance().config(), + InfinityContext::instance().task_scheduler(), + InfinityContext::instance().storage(), + InfinityContext::instance().resource_manager(), + InfinityContext::instance().session_manager()); + query_context_->SetTxn(txn); + } + + ~BGQueryContextWrapper() { + if (session_mgr_ != nullptr) { + auto *session = query_context_->current_session(); + session_mgr_->RemoveSessionByID(session->session_id()); + } + } +}; + +CompactionProcessor::CompactionProcessor(Catalog *catalog, TxnManager *txn_mgr) + : catalog_(catalog), txn_mgr_(txn_mgr), session_mgr_(InfinityContext::instance().session_manager()) {} void CompactionProcessor::Start() { LOG_INFO("Compaction processor is started."); @@ -50,12 +87,54 @@ void CompactionProcessor::Stop() { void CompactionProcessor::Submit(SharedPtr bg_task) { task_queue_.Enqueue(std::move(bg_task)); } -Vector> CompactionProcessor::ScanForCompact() { - auto generate_txn = [this]() { return txn_mgr_->BeginTxn(MakeUnique("Compact")); }; - +void CompactionProcessor::DoCompact() { Txn *scan_txn = txn_mgr_->BeginTxn(MakeUnique("ScanForCompact")); + bool success = false; + DeferFn defer_fn([&] { + if (!success) { + txn_mgr_->RollBackTxn(scan_txn); + } + }); + + Vector, Txn *>> statements = this->ScanForCompact(scan_txn); + Vector> wrappers; + for (const auto &[statement, txn] : statements) { + BGQueryContextWrapper wrapper(txn, session_mgr_); + BGQueryState state; + bool res = wrapper.query_context_->ExecuteBGStatement(statement.get(), state); + if (res) { + wrappers.emplace_back(std::move(wrapper), std::move(state)); + } + } + for (auto &[wrapper, query_state] : wrappers) { + TxnTimeStamp commit_ts = 0; + wrapper.query_context_->JoinBGStatement(query_state, commit_ts); + } + txn_mgr_->CommitTxn(scan_txn); + success = true; +} - Vector> compaction_tasks; +TxnTimeStamp +CompactionProcessor::ManualDoCompact(const String &schema_name, const String &table_name, bool rollback, Optional> mid_func) { + TxnTimeStamp commit_ts = 0; + + auto statement = MakeUnique(schema_name, table_name); + Txn *txn = txn_mgr_->BeginTxn(MakeUnique("ManualCompact")); + BGQueryContextWrapper wrapper(txn, session_mgr_); + BGQueryState state; + bool res = wrapper.query_context_->ExecuteBGStatement(statement.get(), state); + if (mid_func) { + mid_func.value()(); + } + if (res) { + wrapper.query_context_->JoinBGStatement(state, commit_ts, rollback); + } + return commit_ts; +} + +Vector, Txn *>> CompactionProcessor::ScanForCompact(Txn *scan_txn) { + + Vector, Txn *>> compaction_tasks; TransactionID txn_id = scan_txn->TxnID(); TxnTimeStamp begin_ts = scan_txn->BeginTS(); Vector db_entries = catalog_->Databases(txn_id, begin_ts); @@ -63,19 +142,19 @@ Vector> CompactionProcessor::ScanForCompact() { Vector table_entries = db_entry->TableCollections(txn_id, begin_ts); for (auto *table_entry : table_entries) { while (true) { - auto compaction_info = table_entry->CheckCompaction(generate_txn); - if (!compaction_info) { + Txn *txn = txn_mgr_->BeginTxn(MakeUnique("Compact")); + TransactionID txn_id = txn->TxnID(); + auto compact_segments = table_entry->CheckCompaction(txn_id); + if (compact_segments.empty()) { + txn_mgr_->RollBackTxn(txn); break; } - UniquePtr compact_task = - CompactSegmentsTask::MakeTaskWithPickedSegments(table_entry, std::move(compaction_info->segments_), compaction_info->txn_); - compaction_tasks.emplace_back(std::move(compact_task)); + compaction_tasks.emplace_back(MakeUnique(table_entry, std::move(compact_segments)), txn); } } } - txn_mgr_->CommitTxn(scan_txn); return compaction_tasks; } @@ -110,16 +189,7 @@ void CompactionProcessor::Process() { break; } case BGTaskType::kNotifyCompact: { - Vector> compact_tasks = this->ScanForCompact(); - for (auto &compact_task : compact_tasks) { - LOG_TRACE(fmt::format("Compact {} start.", compact_task->table_name())); - compact_task->Execute(); - if (compact_task->TryCommitTxn()) { - LOG_TRACE(fmt::format("Compact {} done.", compact_task->table_name())); - } else { - LOG_TRACE(fmt::format("Compact {} rollback.", compact_task->table_name())); - } - } + DoCompact(); break; } case BGTaskType::kNotifyOptimize: { diff --git a/src/storage/compaction_process.cppm b/src/storage/compaction_process.cppm index 4e4fd01abd..912e51024c 100644 --- a/src/storage/compaction_process.cppm +++ b/src/storage/compaction_process.cppm @@ -17,15 +17,16 @@ module; export module compaction_process; import stl; -import compact_segments_task; import txn; import bg_task; import blocking_queue; +import base_statement; namespace infinity { class Catalog; class TxnManager; +class SessionManager; export class CompactionProcessor { public: @@ -37,8 +38,15 @@ public: void Submit(SharedPtr bg_task); + void DoCompact(); + + TxnTimeStamp ManualDoCompact(const String &schema_name, + const String &table_name, + bool rollback, + Optional> mid_func = None); // false unit test + private: - Vector> ScanForCompact(); + Vector, Txn *>> ScanForCompact(Txn *scan_txn); void ScanAndOptimize(); @@ -51,9 +59,7 @@ private: Catalog *catalog_{}; TxnManager *txn_mgr_{}; - - // atomic_bool stop_{false}; - // std::chrono::seconds interval_{}; + SessionManager *session_mgr_{}; }; } // namespace infinity \ No newline at end of file diff --git a/src/storage/invertedindex/search/query_builder.cpp b/src/storage/invertedindex/search/query_builder.cpp index 4a55b272ed..d7567f7609 100644 --- a/src/storage/invertedindex/search/query_builder.cpp +++ b/src/storage/invertedindex/search/query_builder.cpp @@ -40,8 +40,8 @@ namespace infinity { QueryBuilder::QueryBuilder(Txn *txn, SharedPtr &base_table_ref) : table_entry_(base_table_ref->table_entry_ptr_), index_reader_(table_entry_->GetFullTextIndexReader(txn)) { u64 total_row_count = 0; - for (SegmentEntry *segment_entry : base_table_ref->block_index_->segments_) { - total_row_count += segment_entry->row_count(); + for (const auto &[segment_id, segment_info] : base_table_ref->block_index_->segment_block_index_) { + total_row_count += segment_info.segment_offset_; } scorer_.Init(total_row_count, &index_reader_); diff --git a/src/storage/meta/catalog.cpp b/src/storage/meta/catalog.cpp index bb96d59ecb..1a036a061a 100644 --- a/src/storage/meta/catalog.cpp +++ b/src/storage/meta/catalog.cpp @@ -391,8 +391,9 @@ Status Catalog::RollbackCompact(TableEntry *table_entry, TransactionID txn_id, T Status Catalog::CommitWrite(TableEntry *table_entry, TransactionID txn_id, TxnTimeStamp commit_ts, - const HashMap &segment_stores) { - return table_entry->CommitWrite(txn_id, commit_ts, segment_stores); + const HashMap &segment_stores, + const DeleteState *delete_state) { + return table_entry->CommitWrite(txn_id, commit_ts, segment_stores, delete_state); } Status Catalog::RollbackWrite(TableEntry *table_entry, TxnTimeStamp commit_ts, const Vector &segment_stores) { diff --git a/src/storage/meta/catalog.cppm b/src/storage/meta/catalog.cppm index 68597b5af8..4c282cf156 100644 --- a/src/storage/meta/catalog.cppm +++ b/src/storage/meta/catalog.cppm @@ -191,8 +191,11 @@ public: static Status RollbackCompact(TableEntry *table_entry, TransactionID txn_id, TxnTimeStamp commit_ts, const TxnCompactStore &compact_store); - static Status - CommitWrite(TableEntry *table_entry, TransactionID txn_id, TxnTimeStamp commit_ts, const HashMap &segment_stores); + static Status CommitWrite(TableEntry *table_entry, + TransactionID txn_id, + TxnTimeStamp commit_ts, + const HashMap &segment_stores, + const DeleteState *delete_state); static Status RollbackWrite(TableEntry *table_entry, TxnTimeStamp commit_ts, const Vector &segment_stores); diff --git a/src/storage/meta/entry/segment_entry.cpp b/src/storage/meta/entry/segment_entry.cpp index 7bc11ed3d2..c4e5f29a92 100644 --- a/src/storage/meta/entry/segment_entry.cpp +++ b/src/storage/meta/entry/segment_entry.cpp @@ -39,7 +39,7 @@ import txn_store; import segment_iter; import catalog_delta_entry; import status; -import compact_segments_task; +import compact_state_data; import cleanup_scanner; import background_process; import wal_entry; @@ -151,27 +151,32 @@ void SegmentEntry::UpdateBlockReplay(SharedPtr new_block, String blo block_entries_[block_id]->UpdateBlockReplay(new_block, std::move(block_filter_binary_data)); } -bool SegmentEntry::TrySetCompacting(CompactSegmentsTask *compact_task) { +bool SegmentEntry::TrySetCompacting(CompactStateData *compact_state_data) { std::unique_lock lock(rw_locker_); - if (status_ == SegmentStatus::kUnsealed) { - UnrecoverableError("Assert: Compactable segment should be sealed."); - } if (status_ != SegmentStatus::kSealed) { return false; } - compact_task_ = compact_task; + compact_state_data_ = compact_state_data; status_ = SegmentStatus::kCompacting; return true; } -void SegmentEntry::SetNoDelete() { +bool SegmentEntry::SetNoDelete() { std::unique_lock lock(rw_locker_); - if (status_ != SegmentStatus::kCompacting) { + if (status_ != SegmentStatus::kCompacting && status_ != SegmentStatus::kNoDelete) { UnrecoverableError("Assert: kNoDelete is only allowed to set on compacting segment."); } status_ = SegmentStatus::kNoDelete; - no_delete_complete_cv_.wait(lock, [this] { return delete_txns_.empty(); }); - compact_task_ = nullptr; + if (!delete_txns_.empty()) { + std::stringstream ss; + for (auto txn_id : delete_txns_) { + ss << txn_id << " "; + } + LOG_WARN(fmt::format("Segment {} cannot set no delete, because has delete txns: {}", segment_id_, ss.str())); + return false; + } + compact_state_data_ = nullptr; + return true; } void SegmentEntry::SetDeprecated(TxnTimeStamp deprecate_ts) { @@ -367,34 +372,6 @@ SizeT SegmentEntry::DeleteData(TransactionID txn_id, } } this->DecreaseRemainRow(delete_row_n); - { - std::unique_lock w_lock(rw_locker_); - if (this->first_delete_ts_ == UNCOMMIT_TS) { - this->first_delete_ts_ = commit_ts; - } - if (status_ == SegmentStatus::kDeprecated) { - UnrecoverableError("Assert: Should not commit delete to deprecated segment."); - } - if (compact_task_ != nullptr) { - if (status_ != SegmentStatus::kCompacting && status_ != SegmentStatus::kNoDelete) { - UnrecoverableError("Assert: compact_task is not nullptr means segment is being compacted"); - } - Vector segment_offsets; - for (const auto &[block_id, delete_rows] : block_row_hashmap) { - for (BlockOffset block_offset : delete_rows) { - SegmentOffset segment_offset = block_id * DEFAULT_BLOCK_CAPACITY + block_offset; - segment_offsets.push_back(segment_offset); - } - } - compact_task_->AddToDelete(segment_id_, std::move(segment_offsets)); - } - { - delete_txns_.erase(txn_id); - if (delete_txns_.empty()) { // == 0 when replay - no_delete_complete_cv_.notify_one(); - } - } - } return delete_row_n; } @@ -404,8 +381,40 @@ void SegmentEntry::CommitFlushed(TxnTimeStamp commit_ts) { } } -void SegmentEntry::CommitSegment(TransactionID txn_id, TxnTimeStamp commit_ts, const TxnSegmentStore &segment_store) { +void SegmentEntry::CommitSegment(TransactionID txn_id, + TxnTimeStamp commit_ts, + const TxnSegmentStore &segment_store, + const DeleteState *delete_state) { std::unique_lock w_lock(rw_locker_); + if (status_ == SegmentStatus::kDeprecated) { + UnrecoverableError("Assert: Should not commit delete to deprecated segment."); + } + + if (delete_state != nullptr) { + auto iter = delete_state->rows_.find(segment_id_); + if (iter != delete_state->rows_.end()) { + const auto &block_row_hashmap = iter->second; + if (this->first_delete_ts_ == UNCOMMIT_TS) { + this->first_delete_ts_ = commit_ts; + } + delete_txns_.erase(txn_id); + + if (compact_state_data_ != nullptr) { + if (status_ != SegmentStatus::kCompacting && status_ != SegmentStatus::kNoDelete) { + UnrecoverableError("Assert: compact_task is not nullptr means segment is being compacted"); + } + Vector segment_offsets; + for (const auto &[block_id, block_offsets] : block_row_hashmap) { + for (auto block_offset : block_offsets) { + segment_offsets.push_back(block_id * DEFAULT_BLOCK_CAPACITY + block_offset); + } + } + compact_state_data_->AddToDelete(segment_id_, segment_offsets); + LOG_INFO(fmt::format("Append {} rows to to_delete_list in compact list", segment_offsets.size())); + } + } + } + min_row_ts_ = std::min(min_row_ts_, commit_ts); if (commit_ts < max_row_ts_) { UnrecoverableError(fmt::format("SegmentEntry commit_ts {} is less than max_row_ts {}", commit_ts, max_row_ts_)); diff --git a/src/storage/meta/entry/segment_entry.cppm b/src/storage/meta/entry/segment_entry.cppm index bb334bada6..ea7f371368 100644 --- a/src/storage/meta/entry/segment_entry.cppm +++ b/src/storage/meta/entry/segment_entry.cppm @@ -36,7 +36,7 @@ namespace infinity { class TxnTableStore; struct TxnSegmentStore; struct TableEntry; -class CompactSegmentsTask; +class CompactStateData; class BlockEntryIter; export struct BlocksGuard { @@ -99,9 +99,9 @@ public: bool SetSealed(); - bool TrySetCompacting(CompactSegmentsTask *compact_task); + bool TrySetCompacting(CompactStateData *compact_state_data); - void SetNoDelete(); + bool SetNoDelete(); void SetDeprecated(TxnTimeStamp deprecate_ts); @@ -189,7 +189,7 @@ public: void CommitFlushed(TxnTimeStamp commit_ts); - void CommitSegment(TransactionID txn_id, TxnTimeStamp commit_ts, const TxnSegmentStore &segment_store); + void CommitSegment(TransactionID txn_id, TxnTimeStamp commit_ts, const TxnSegmentStore &segment_store, const DeleteState *delete_state); void RollbackBlocks(TxnTimeStamp commit_ts, const HashMap &block_entries); @@ -234,10 +234,9 @@ private: // check if a value must not exist in the segment FastRoughFilter fast_rough_filter_; - CompactSegmentsTask *compact_task_{}; + CompactStateData *compact_state_data_{}; SegmentStatus status_; - std::condition_variable_any no_delete_complete_cv_{}; HashSet delete_txns_; // current number of delete txn that write this segment public: diff --git a/src/storage/meta/entry/table_entry.cpp b/src/storage/meta/entry/table_entry.cpp index 4634696f0c..42f2084162 100644 --- a/src/storage/meta/entry/table_entry.cpp +++ b/src/storage/meta/entry/table_entry.cpp @@ -44,7 +44,7 @@ import column_def; import data_type; import default_values; import DBT_compaction_alg; -import compact_segments_task; +import compact_statement; import local_file_system; import build_fast_rough_filter_task; import block_entry; @@ -453,7 +453,7 @@ Status TableEntry::RollbackDelete(TransactionID txn_id, DeleteState &, BufferMan } Status TableEntry::CommitCompact(TransactionID txn_id, TxnTimeStamp commit_ts, TxnCompactStore &compact_store) { - if (compact_store.task_type_ == CompactSegmentsTaskType::kInvalid) { + if (compact_store.type_ == CompactStatementType::kInvalid) { return Status::OK(); } @@ -474,7 +474,7 @@ Status TableEntry::CommitCompact(TransactionID txn_id, TxnTimeStamp commit_ts, T auto *segment_entry = segment_store.segment_entry_; - segment_entry->CommitSegment(txn_id, commit_ts, segment_store); + segment_entry->CommitSegment(txn_id, commit_ts, segment_store, nullptr); for (const auto &old_segment : old_segments) { // old_segment->TrySetDeprecated(commit_ts); @@ -504,13 +504,13 @@ Status TableEntry::CommitCompact(TransactionID txn_id, TxnTimeStamp commit_ts, T return Status::OK(); } - switch (compact_store.task_type_) { - case CompactSegmentsTaskType::kCompactPickedSegments: { + switch (compact_store.type_) { + case CompactStatementType::kAuto: { compaction_alg_->CommitCompact(txn_id); LOG_TRACE(fmt::format("Compact commit picked, tablename: {}", *this->GetTableName())); break; } - case CompactSegmentsTaskType::kCompactTable: { + case CompactStatementType::kManual: { // reinitialize compaction_alg_ with new segments and enable it LOG_TRACE(fmt::format("Compact commit whole, tablename: {}", *this->GetTableName())); compaction_alg_->Enable({}); @@ -553,12 +553,12 @@ Status TableEntry::RollbackCompact(TransactionID txn_id, TxnTimeStamp commit_ts, } } if (compaction_alg_.get() != nullptr) { - switch (compact_store.task_type_) { - case CompactSegmentsTaskType::kCompactPickedSegments: { + switch (compact_store.type_) { + case CompactStatementType::kAuto: { compaction_alg_->RollbackCompact(txn_id); break; } - case CompactSegmentsTaskType::kCompactTable: { + case CompactStatementType::kManual: { Vector old_segments; for (const auto &[_, old_segs] : compact_store.compact_data_) { old_segments.insert(old_segments.end(), old_segs.begin(), old_segs.end()); @@ -575,10 +575,13 @@ Status TableEntry::RollbackCompact(TransactionID txn_id, TxnTimeStamp commit_ts, return Status::OK(); } -Status TableEntry::CommitWrite(TransactionID txn_id, TxnTimeStamp commit_ts, const HashMap &segment_stores) { +Status TableEntry::CommitWrite(TransactionID txn_id, + TxnTimeStamp commit_ts, + const HashMap &segment_stores, + const DeleteState *delete_state) { for (const auto &[segment_id, segment_store] : segment_stores) { auto *segment_entry = segment_store.segment_entry_; - segment_entry->CommitSegment(txn_id, commit_ts, segment_store); + segment_entry->CommitSegment(txn_id, commit_ts, segment_store, delete_state); } return Status::OK(); } @@ -836,7 +839,8 @@ void TableEntry::OptimizeIndex(Txn *txn) { case IndexType::kHnsw: case IndexType::kSecondary: { TxnTimeStamp begin_ts = txn->BeginTS(); - for (auto &[segment_id, segment_index_entry] : table_index_entry->index_by_segment()) { + auto segment_index_guard = table_index_entry->GetSegmentIndexesGuard(); + for (auto &[segment_id, segment_index_entry] : segment_index_guard.index_by_segment_) { SegmentEntry *segment_entry = GetSegmentByID(segment_id, begin_ts).get(); if (segment_entry != nullptr) { auto *merged_chunk_entry = segment_index_entry->RebuildChunkIndexEntries(txn_table_store, segment_entry); @@ -908,7 +912,6 @@ SharedPtr TableEntry::GetBlockIndex(Txn *txn) { // SharedPtr> result = MakeShared>(); SharedPtr result = MakeShared(); std::shared_lock rw_locker(this->rw_locker_); - result->Reserve(this->segment_map_.size()); // Add segment that is not deprecated for (const auto &segment_pair : this->segment_map_) { @@ -918,6 +921,19 @@ SharedPtr TableEntry::GetBlockIndex(Txn *txn) { return result; } +SharedPtr TableEntry::GetIndexIndex(Txn *txn) { + SharedPtr result = MakeShared(); + auto index_meta_map_guard = index_meta_map_.GetMetaMap(); + for (auto &[index_name, table_index_meta] : *index_meta_map_guard) { + auto [table_index_entry, status] = table_index_meta->GetEntryNolock(txn->TxnID(), txn->BeginTS()); + if (!status.ok()) { + continue; + } + result->Insert(table_index_entry, txn); + } + return result; +} + bool TableEntry::CheckDeleteVisible(DeleteState &delete_state, Txn *txn) { for (auto &[segment_id, block_offsets_map] : delete_state.rows_) { auto *segment_entry = GetSegmentByID(segment_id, txn).get(); @@ -984,6 +1000,9 @@ nlohmann::json TableEntry::Serialize(TxnTimeStamp max_commit_ts) { // Serialize segments for (const auto &segment_entry : segment_candidates) { + if (segment_entry->commit_ts_ > max_commit_ts) { + continue; + } json_res["segments"].emplace_back(segment_entry->Serialize(max_commit_ts)); checkpoint_row_count += segment_entry->checkpoint_row_count(); } @@ -1113,28 +1132,20 @@ void TableEntry::AddDeleteToCompactionAlg(SegmentID segment_id) { compaction_alg_->DeleteInSegment(segment_id); } -Optional TableEntry::CheckCompaction(std::function generate_txn) { +Vector TableEntry::CheckCompaction(TransactionID txn_id) { if (compaction_alg_.get() == nullptr) { - return None; + return {}; } - return compaction_alg_->CheckCompaction(generate_txn); + return compaction_alg_->CheckCompaction(txn_id); } -Vector TableEntry::PickCompactSegments() const { - if (compaction_alg_.get() != nullptr) { - compaction_alg_->Disable(); // wait for current compaction to finish - } - Vector result; - std::shared_lock lock(this->rw_locker_); - for (const auto &[segment_id, segment] : this->segment_map_) { - auto status = segment->status(); - if (status == SegmentStatus::kSealed) { - result.emplace_back(segment.get()); - } else if (status == SegmentStatus::kCompacting || status == SegmentStatus::kNoDelete) { - UnrecoverableError("Segment should not be compacting or no delete when picking manually"); - } +bool TableEntry::CompactPrepare() const { + if (compaction_alg_.get() == nullptr) { + LOG_WARN(fmt::format("Table {} compaction algorithm not set", *this->GetTableName())); + return false; } - return result; + compaction_alg_->Disable(); // wait for current compaction to finish + return true; } void TableEntry::PickCleanup(CleanupScanner *scanner) { diff --git a/src/storage/meta/entry/table_entry.cppm b/src/storage/meta/entry/table_entry.cppm index ff8bd5a43f..8564ad4a4e 100644 --- a/src/storage/meta/entry/table_entry.cppm +++ b/src/storage/meta/entry/table_entry.cppm @@ -162,7 +162,10 @@ public: Status RollbackCompact(TransactionID txn_id, TxnTimeStamp commit_ts, const TxnCompactStore &compact_state); - Status CommitWrite(TransactionID txn_id, TxnTimeStamp commit_ts, const HashMap &segment_stores); + Status CommitWrite(TransactionID txn_id, + TxnTimeStamp commit_ts, + const HashMap &segment_stores, + const DeleteState *delete_state); Status RollbackWrite(TxnTimeStamp commit_ts, const Vector &segment_stores); @@ -219,6 +222,8 @@ public: SharedPtr GetBlockIndex(Txn *txn); + SharedPtr GetIndexIndex(Txn *txn); + void GetFulltextAnalyzers(TransactionID txn_id, TxnTimeStamp begin_ts, Map &column2analyzer); public: @@ -278,9 +283,9 @@ public: void AddDeleteToCompactionAlg(SegmentID segment_id); - Optional CheckCompaction(std::function generate_txn); + Vector CheckCompaction(TransactionID txn_id); - Vector PickCompactSegments() const; + bool CompactPrepare() const; private: // the compaction algorithm, mutable because all its interface are protected by lock diff --git a/src/storage/meta/entry/table_index_entry.cpp b/src/storage/meta/entry/table_index_entry.cpp index 4b89778f93..270052c0de 100644 --- a/src/storage/meta/entry/table_index_entry.cpp +++ b/src/storage/meta/entry/table_index_entry.cpp @@ -288,13 +288,22 @@ SharedPtr TableIndexEntry::PopulateEntirely(SegmentEntry *seg } Tuple, Status> -TableIndexEntry::CreateIndexPrepare(TableEntry *table_entry, BlockIndex *block_index, Txn *txn, bool prepare, bool is_replay, bool check_ts) { +TableIndexEntry::CreateIndexPrepare(BaseTableRef *table_ref, Txn *txn, bool prepare, bool is_replay, bool check_ts) { + TableEntry *table_entry = table_ref->table_entry_ptr_; + auto &block_index = table_ref->block_index_; + if (table_ref->index_index_.get() == nullptr) { + table_ref->index_index_ = MakeShared(); + } + auto &index_index = table_ref->index_index_; Vector segment_index_entries; SegmentID unsealed_id = table_entry->unsealed_id(); - for (const auto *segment_entry : block_index->segments_) { - auto create_index_param = SegmentIndexEntry::GetCreateIndexParam(index_base_, segment_entry->row_count(), column_def_); - SegmentID segment_id = segment_entry->segment_id(); + for (const auto &[segment_id, segment_info] : block_index->segment_block_index_) { + SegmentOffset segment_offset = segment_info.segment_offset_; + + auto create_index_param = SegmentIndexEntry::GetCreateIndexParam(index_base_, segment_offset, column_def_); + auto *segment_entry = segment_info.segment_entry_; SharedPtr segment_index_entry = SegmentIndexEntry::NewIndexEntry(this, segment_id, txn, create_index_param.get()); + index_index->Insert(this, segment_index_entry.get()); if (!is_replay) { segment_index_entry->CreateIndexPrepare(segment_entry, txn, prepare, check_ts); } @@ -308,13 +317,18 @@ TableIndexEntry::CreateIndexPrepare(TableEntry *table_entry, BlockIndex *block_i return {segment_index_entries, Status::OK()}; } -Status TableIndexEntry::CreateIndexDo(const TableEntry *table_entry, HashMap &create_index_idxes, Txn *txn) { +Status TableIndexEntry::CreateIndexDo(BaseTableRef *table_ref, HashMap &create_index_idxes, Txn *txn) { if (this->index_base_->column_names_.size() != 1) { // TODO RecoverableError(Status::NotSupport("Not implemented")); } - Map> index_by_segment = GetIndexBySegmentSnapshot(table_entry, txn); - for (auto &[segment_id, segment_index_entry] : index_by_segment) { + auto &index_index = table_ref->index_index_; + auto iter = index_index->index_snapshots_.find(*index_base_->index_name_); + if (iter == index_index->index_snapshots_.end()) { + return Status::OK(); + } + auto &segment_index_snapshots = iter->second; + for (auto &[segment_id, segment_index_entry] : segment_index_snapshots->segment_index_entries_) { atomic_u64 &create_index_idx = create_index_idxes.at(segment_id); auto status = segment_index_entry->CreateIndexDo(create_index_idx); if (!status.ok()) { diff --git a/src/storage/meta/entry/table_index_entry.cppm b/src/storage/meta/entry/table_index_entry.cppm index ce6a1cf44e..0ef08a0351 100644 --- a/src/storage/meta/entry/table_index_entry.cppm +++ b/src/storage/meta/entry/table_index_entry.cppm @@ -46,6 +46,11 @@ struct SegmentEntry; class BaseTableRef; class AddTableIndexEntryOp; +export struct SegmentIndexesGuard { + const Map> &index_by_segment_; + std::shared_lock lock_; +}; + export struct TableIndexEntry : public BaseEntry, public EntryInterface { friend struct TableEntry; @@ -95,6 +100,9 @@ public: inline const SharedPtr &column_def() const { return column_def_; } Map> &index_by_segment() { return index_by_segment_; } + + SegmentIndexesGuard GetSegmentIndexesGuard() { return {index_by_segment_, std::shared_lock(rw_locker_)}; } + Map> GetIndexBySegmentSnapshot(const TableEntry *table_entry, Txn *txn); const SharedPtr &index_dir() const { return index_dir_; } String GetPathNameTail() const; @@ -113,9 +121,9 @@ public: SharedPtr PopulateEntirely(SegmentEntry *segment_entry, Txn *txn, const PopulateEntireConfig &config); Tuple, Status> - CreateIndexPrepare(TableEntry *table_entry, BlockIndex *block_index, Txn *txn, bool prepare, bool is_replay, bool check_ts = true); + CreateIndexPrepare(BaseTableRef *table_ref, Txn *txn, bool prepare, bool is_replay, bool check_ts = true); - Status CreateIndexDo(const TableEntry *table_entry, HashMap &create_index_idxes, Txn *txn); + Status CreateIndexDo(BaseTableRef *table_ref, HashMap &create_index_idxes, Txn *txn); MemoryPool &GetFulltextByteSlicePool() { return byte_slice_pool_; } RecyclePool &GetFulltextBufferPool() { return buffer_pool_; } diff --git a/src/storage/secondary_index/common_query_filter.cpp b/src/storage/secondary_index/common_query_filter.cpp index 2e238e7d26..17852e307c 100644 --- a/src/storage/secondary_index/common_query_filter.cpp +++ b/src/storage/secondary_index/common_query_filter.cpp @@ -21,6 +21,7 @@ import stl; import bitmask; import base_expression; import base_table_ref; +import block_index; import segment_entry; import fast_rough_filter; import table_index_entry; @@ -118,7 +119,7 @@ void MergeIntoBitmask(const VectorBuffer *input_bool_column_buffer, CommonQueryFilter::CommonQueryFilter(SharedPtr original_filter, SharedPtr base_table_ref, TxnTimeStamp begin_ts) : begin_ts_(begin_ts), original_filter_(std::move(original_filter)), base_table_ref_(std::move(base_table_ref)) { - const HashMap &segment_index = base_table_ref_->block_index_->segment_index_; + const auto &segment_index = base_table_ref_->block_index_->segment_block_index_; if (segment_index.empty()) { finish_build_.test_and_set(std::memory_order_release); } else { @@ -133,9 +134,9 @@ CommonQueryFilter::CommonQueryFilter(SharedPtr original_filter, void CommonQueryFilter::BuildFilter(u32 task_id, Txn *txn) { auto *buffer_mgr = txn->buffer_mgr(); TxnTimeStamp begin_ts = txn->BeginTS(); - const HashMap &segment_index = base_table_ref_->block_index_->segment_index_; + const auto &segment_index = base_table_ref_->block_index_->segment_block_index_; const SegmentID segment_id = tasks_[task_id]; - const SegmentEntry *segment_entry = segment_index.at(segment_id); + const SegmentEntry *segment_entry = segment_index.at(segment_id).segment_entry_; if (!fast_rough_filter_evaluator_->Evaluate(begin_ts, *segment_entry->GetFastRoughFilter())) { // skip this segment return; diff --git a/src/storage/storage.cpp b/src/storage/storage.cpp index ff8bee68ba..853dfa92b8 100644 --- a/src/storage/storage.cpp +++ b/src/storage/storage.cpp @@ -44,6 +44,9 @@ import periodic_trigger_thread; import periodic_trigger; import log_file; +import query_context; +import infinity_context; + namespace infinity { Storage::Storage(Config *config_ptr) : config_ptr_(config_ptr) {} diff --git a/src/storage/storage.cppm b/src/storage/storage.cppm index 649b49068d..e5c2054b43 100644 --- a/src/storage/storage.cppm +++ b/src/storage/storage.cppm @@ -43,6 +43,8 @@ public: [[nodiscard]] inline BGTaskProcessor *bg_processor() const noexcept { return bg_processor_.get(); } + [[nodiscard]] inline CompactionProcessor *compaction_processor() const noexcept { return compact_processor_.get(); } + void Init(); void UnInit(); diff --git a/src/storage/txn/txn.cpp b/src/storage/txn/txn.cpp index b49c52fc52..0738410496 100644 --- a/src/storage/txn/txn.cpp +++ b/src/storage/txn/txn.cpp @@ -46,7 +46,7 @@ import catalog_delta_entry; import bg_task; import background_process; import base_table_ref; -import compact_segments_task; +import compact_statement; import default_values; import chunk_index_entry; @@ -120,7 +120,7 @@ Status Txn::Delete(TableEntry *table_entry, const Vector &row_ids, bool c } Status -Txn::Compact(TableEntry *table_entry, Vector, Vector>> &&segment_data, CompactSegmentsTaskType type) { +Txn::Compact(TableEntry *table_entry, Vector, Vector>> &&segment_data, CompactStatementType type) { TxnTableStore *table_store = this->GetTxnTableStore(table_entry); auto [err_mgs, compact_status] = table_store->Compact(std::move(segment_data), type); @@ -270,8 +270,7 @@ Tuple, Status> Txn::GetTableIndexInfo(const String &db Status Txn::CreateIndexPrepare(TableIndexEntry *table_index_entry, BaseTableRef *table_ref, bool prepare, bool check_ts) { auto *table_entry = table_ref->table_entry_ptr_; - auto [segment_index_entries, status] = - table_index_entry->CreateIndexPrepare(table_entry, table_ref->block_index_.get(), this, prepare, false, check_ts); + auto [segment_index_entries, status] = table_index_entry->CreateIndexPrepare(table_ref, this, prepare, false, check_ts); if (!status.ok()) { return Status::OK(); } @@ -296,11 +295,8 @@ Status Txn::CreateIndexDo(BaseTableRef *table_ref, const String &index_name, Has if (!status.ok()) { return status; } - if (table_index_entry->txn_id_ != txn_id_) { - UnrecoverableError("Index is not created by this txn. Something error happened."); - } - return table_index_entry->CreateIndexDo(table_entry, create_index_idxes, this); + return table_index_entry->CreateIndexDo(table_ref, create_index_idxes, this); } Status Txn::CreateIndexFinish(const TableEntry *table_entry, const TableIndexEntry *table_index_entry) { diff --git a/src/storage/txn/txn.cppm b/src/storage/txn/txn.cppm index dee4201c82..ece48f7679 100644 --- a/src/storage/txn/txn.cppm +++ b/src/storage/txn/txn.cppm @@ -59,7 +59,7 @@ struct WalCmd; class CatalogDeltaEntry; class CatalogDeltaOperation; class BaseTableRef; -enum class CompactSegmentsTaskType; +enum class CompactStatementType; export class Txn { public: @@ -163,8 +163,7 @@ public: Status Delete(TableEntry *table_entry, const Vector &row_ids, bool check_conflict = true); - Status - Compact(TableEntry *table_entry, Vector, Vector>> &&segment_data, CompactSegmentsTaskType type); + Status Compact(TableEntry *table_entry, Vector, Vector>> &&segment_data, CompactStatementType type); // Getter BufferManager *buffer_mgr() const { return buffer_mgr_; } diff --git a/src/storage/txn/txn_store.cpp b/src/storage/txn/txn_store.cpp index e6b48bb966..9da1339a6f 100644 --- a/src/storage/txn/txn_store.cpp +++ b/src/storage/txn/txn_store.cpp @@ -36,7 +36,7 @@ import internal_types; import data_type; import background_process; import bg_task; -import compact_segments_task; +import compact_statement; import build_fast_rough_filter_task; namespace infinity { @@ -111,9 +111,9 @@ void TxnIndexStore::Commit(TransactionID txn_id, TxnTimeStamp commit_ts) const { ///----------------------------------------------------------------------------- -TxnCompactStore::TxnCompactStore() : task_type_(CompactSegmentsTaskType::kInvalid) {} +TxnCompactStore::TxnCompactStore() : type_(CompactStatementType::kInvalid) {} -TxnCompactStore::TxnCompactStore(CompactSegmentsTaskType type) : task_type_(type) {} +TxnCompactStore::TxnCompactStore(CompactStatementType type) : type_(type) {} ///----------------------------------------------------------------------------- @@ -226,8 +226,8 @@ Tuple, Status> TxnTableStore::Delete(const Vector &row_ } Tuple, Status> TxnTableStore::Compact(Vector, Vector>> &&segment_data, - CompactSegmentsTaskType type) { - if (compact_state_.task_type_ != CompactSegmentsTaskType::kInvalid) { + CompactStatementType type) { + if (compact_state_.type_ != CompactStatementType::kInvalid) { UnrecoverableError("Attempt to compact table store twice"); } compact_state_ = TxnCompactStore(type); @@ -325,7 +325,7 @@ void TxnTableStore::PrepareCommit(TransactionID txn_id, TxnTimeStamp commit_ts, Catalog::Append(table_entry_, txn_id, this, commit_ts, buffer_mgr); // Attention: "compact" needs to be ahead of "delete" - if (compact_state_.task_type_ != CompactSegmentsTaskType::kInvalid) { + if (compact_state_.type_ != CompactStatementType::kInvalid) { LOG_TRACE(fmt::format("Commit compact, table dir: {}, commit ts: {}", *table_entry_->TableEntryDir(), commit_ts)); Catalog::CommitCompact(table_entry_, txn_id, commit_ts, compact_state_); } @@ -349,7 +349,7 @@ void TxnTableStore::PrepareCommit(TransactionID txn_id, TxnTimeStamp commit_ts, * @brief Call for really commit the data to disk. */ void TxnTableStore::Commit(TransactionID txn_id, TxnTimeStamp commit_ts) const { - Catalog::CommitWrite(table_entry_, txn_id, commit_ts, txn_segments_store_); + Catalog::CommitWrite(table_entry_, txn_id, commit_ts, txn_segments_store_, &delete_state_); for (const auto &[index_name, txn_index_store] : txn_indexes_store_) { Catalog::CommitCreateIndex(txn_index_store.get(), commit_ts); txn_index_store->Commit(txn_id, commit_ts); diff --git a/src/storage/txn/txn_store.cppm b/src/storage/txn/txn_store.cppm index 2cbe6b089e..4831ab7070 100644 --- a/src/storage/txn/txn_store.cppm +++ b/src/storage/txn/txn_store.cppm @@ -38,7 +38,7 @@ class SegmentIndexEntry; class ChunkIndexEntry; class BGTaskProcessor; class TxnManager; -enum class CompactSegmentsTaskType; +enum class CompactStatementType; class CatalogDeltaEntry; class BufferManager; @@ -80,10 +80,10 @@ public: export struct TxnCompactStore { Vector>> compact_data_; - CompactSegmentsTaskType task_type_; + CompactStatementType type_; TxnCompactStore(); - TxnCompactStore(CompactSegmentsTaskType type); + TxnCompactStore(CompactStatementType type); }; export class TxnTableStore { @@ -106,8 +106,7 @@ public: Tuple, Status> Delete(const Vector &row_ids); - Tuple, Status> Compact(Vector, Vector>> &&segment_data, - CompactSegmentsTaskType type); + Tuple, Status> Compact(Vector, Vector>> &&segment_data, CompactStatementType type); void Rollback(TransactionID txn_id, TxnTimeStamp abort_ts); diff --git a/src/storage/wal/wal_manager.cpp b/src/storage/wal/wal_manager.cpp index 0b907deaef..46f0c0091e 100644 --- a/src/storage/wal/wal_manager.cpp +++ b/src/storage/wal/wal_manager.cpp @@ -36,7 +36,7 @@ import extra_ddl_info; import infinity_exception; import block_column_entry; -import compact_segments_task; +import compact_state_data; import build_fast_rough_filter_task; import catalog_delta_entry; import column_def; @@ -53,6 +53,7 @@ import log_file; import default_values; import defer_op; import index_base; +import base_table_ref; module wal_manager; @@ -656,8 +657,8 @@ void WalManager::WalCmdCreateIndexReplay(const WalCmdCreateIndex &cmd, Transacti auto fake_txn = Txn::NewReplayTxn(storage_->buffer_manager(), storage_->txn_manager(), storage_->catalog(), txn_id); auto txn = MakeUnique(nullptr /*buffer_mgr*/, nullptr /*txn_mgr*/, nullptr /*catalog*/, txn_id, begin_ts); - auto block_index = table_entry->GetBlockIndex(txn.get()); - table_index_entry->CreateIndexPrepare(table_entry, block_index.get(), fake_txn.get(), false, true); + auto base_table_ref = MakeShared(table_entry, table_entry->GetBlockIndex(txn.get())); + table_index_entry->CreateIndexPrepare(base_table_ref.get(), fake_txn.get(), false, true); auto *txn_store = fake_txn->GetTxnTableStore(table_entry); for (const auto &[index_name, txn_index_store] : txn_store->txn_indexes_store()) { @@ -745,7 +746,7 @@ void WalManager::WalCmdDeleteReplay(const WalCmdDelete &cmd, TransactionID txn_i table_store->Delete(cmd.row_ids_); fake_txn->FakeCommit(commit_ts); Catalog::Delete(table_store->table_entry_, fake_txn->TxnID(), (void *)table_store, fake_txn->CommitTS(), table_store->delete_state_); - Catalog::CommitWrite(table_store->table_entry_, fake_txn->TxnID(), commit_ts, table_store->txn_segments()); + Catalog::CommitWrite(table_store->table_entry_, fake_txn->TxnID(), commit_ts, table_store->txn_segments(), &table_store->delete_state_); } void WalManager::WalCmdCompactReplay(const WalCmdCompact &cmd, TransactionID txn_id, TxnTimeStamp commit_ts) { @@ -764,7 +765,9 @@ void WalManager::WalCmdCompactReplay(const WalCmdCompact &cmd, TransactionID txn if (!segment_entry->TrySetCompacting(nullptr)) { // fake set because check UnrecoverableError("Assert: Replay segment should be compactable."); } - segment_entry->SetNoDelete(); + if (!segment_entry->SetNoDelete()) { + UnrecoverableError("Assert: Replay segment should be compactable."); + } segment_entry->SetDeprecated(commit_ts); } } @@ -784,7 +787,7 @@ void WalManager::WalCmdAppendReplay(const WalCmdAppend &cmd, TransactionID txn_i fake_txn->FakeCommit(commit_ts); Catalog::Append(table_store->table_entry_, fake_txn->TxnID(), table_store, commit_ts, storage_->buffer_manager(), true); - Catalog::CommitWrite(table_store->table_entry_, fake_txn->TxnID(), commit_ts, table_store->txn_segments()); + Catalog::CommitWrite(table_store->table_entry_, fake_txn->TxnID(), commit_ts, table_store->txn_segments(), nullptr); } // // TMP deprecated diff --git a/src/unit_test/storage/bg_task/cleanup_task.cpp b/src/unit_test/storage/bg_task/cleanup_task.cpp index 655ccb76f6..1edc10ffaa 100644 --- a/src/unit_test/storage/bg_task/cleanup_task.cpp +++ b/src/unit_test/storage/bg_task/cleanup_task.cpp @@ -34,13 +34,13 @@ import physical_import; import status; import compilation_config; import bg_task; -import compact_segments_task; import index_base; import third_party; import base_table_ref; import index_secondary; import infinity_exception; import wal_manager; +import compaction_process; using namespace infinity; @@ -298,6 +298,7 @@ TEST_F(CleanupTaskTest, test_compact_and_cleanup) { TxnManager *txn_mgr = storage->txn_manager(); BufferManager *buffer_mgr = storage->buffer_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); TxnTimeStamp last_commit_ts = 0; Vector> column_defs; @@ -320,7 +321,7 @@ TEST_F(CleanupTaskTest, test_compact_and_cleanup) { auto *txn = txn_mgr->BeginTxn(MakeUnique("get table1")); auto [table_entry, status] = txn->GetTableByName(*db_name, *table_name); EXPECT_TRUE(table_entry != nullptr); - table_entry->SetCompactionAlg(nullptr); + // table_entry->SetCompactionAlg(nullptr); for (int i = 0; i < kImportN; ++i) { Vector> column_vectors; @@ -357,18 +358,9 @@ TEST_F(CleanupTaskTest, test_compact_and_cleanup) { txn_mgr->CommitTxn(txn); } { - auto txn = txn_mgr->BeginTxn(MakeUnique("get table1")); - - auto [table_entry, status] = txn->GetTableByName(*db_name, *table_name); - EXPECT_TRUE(table_entry != nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn); - compact_task->Execute(); - } - last_commit_ts = txn_mgr->CommitTxn(txn); + auto last_commit_ts = compaction_processor->ManualDoCompact(*db_name, *table_name, false); + EXPECT_NE(last_commit_ts, 0u); } - WaitCleanup(storage, last_commit_ts); InfinityContext::instance().UnInit(); @@ -388,6 +380,7 @@ TEST_F(CleanupTaskTest, test_with_index_compact_and_cleanup) { TxnManager *txn_mgr = storage->txn_manager(); BufferManager *buffer_mgr = storage->buffer_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); TxnTimeStamp last_commit_ts = 0; auto db_name = MakeShared("default_db"); @@ -412,7 +405,7 @@ TEST_F(CleanupTaskTest, test_with_index_compact_and_cleanup) { auto *txn = txn_mgr->BeginTxn(MakeUnique("get table1")); auto [table_entry, status] = txn->GetTableByName(*db_name, *table_name); EXPECT_TRUE(table_entry != nullptr); - table_entry->SetCompactionAlg(nullptr); + // table_entry->SetCompactionAlg(nullptr); for (int i = 0; i < kImportN; ++i) { Vector> column_vectors; @@ -467,16 +460,8 @@ TEST_F(CleanupTaskTest, test_with_index_compact_and_cleanup) { txn_mgr->CommitTxn(txn); } { - auto txn = txn_mgr->BeginTxn(MakeUnique("compact")); - - auto [table_entry, status] = txn->GetTableByName(*db_name, *table_name); - EXPECT_TRUE(table_entry != nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn); - compact_task->Execute(); - } - last_commit_ts = txn_mgr->CommitTxn(txn); + last_commit_ts = compaction_processor->ManualDoCompact(*db_name, *table_name, false); + EXPECT_NE(last_commit_ts, 0u); } WaitCleanup(storage, last_commit_ts); diff --git a/src/unit_test/storage/bg_task/compact_segments_task.cpp b/src/unit_test/storage/bg_task/compact_segments_task.cpp index e6c248d7d9..87db50d85d 100644 --- a/src/unit_test/storage/bg_task/compact_segments_task.cpp +++ b/src/unit_test/storage/bg_task/compact_segments_task.cpp @@ -18,7 +18,6 @@ import stl; import storage; -import compact_segments_task; import global_resource_usage; import infinity_context; import status; @@ -41,6 +40,10 @@ import column_def; import data_type; import segment_entry; import block_entry; +import compaction_process; +import compilation_config; +import logger; +import third_party; using namespace infinity; @@ -51,7 +54,6 @@ class CompactTaskTest : public BaseTest { auto *txn = txn_mgr->BeginTxn(MakeUnique("import table")); auto [table_entry, status] = txn->GetTableByName("default_db", table_name); - table_entry->SetCompactionAlg(nullptr); // close auto compaction to test manual compaction auto column_count = table_entry->ColumnCount(); SegmentID segment_id = Catalog::GetNextSegmentID(table_entry); @@ -79,21 +81,33 @@ class CompactTaskTest : public BaseTest { txn_mgr->CommitTxn(txn); } } -}; -TEST_F(CompactTaskTest, compact_to_single_segment) { - { - String table_name = "tbl1"; + void SetUp() override { + auto config_path = std::make_shared(std::string(test_data_path()) + "/config/test_cleanup_task.toml"); + #ifdef INFINITY_DEBUG infinity::GlobalResourceUsage::Init(); #endif - std::shared_ptr config_path = nullptr; RemoveDbDirs(); infinity::InfinityContext::instance().Init(config_path); + } + + void TearDown() override { + infinity::InfinityContext::instance().UnInit(); +#ifdef INFINITY_DEBUG + infinity::GlobalResourceUsage::UnInit(); +#endif + } +}; + +TEST_F(CompactTaskTest, compact_to_single_segment) { + { + String table_name = "tbl1"; Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); + CompactionProcessor *compaction_process = storage->compaction_processor(); Vector> columns; { @@ -118,16 +132,8 @@ TEST_F(CompactTaskTest, compact_to_single_segment) { this->AddSegments(txn_mgr, table_name, segment_sizes, buffer_manager); { // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName("default_db", table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - compact_task->Execute(); - } - txn_mgr->CommitTxn(txn4); + auto commit_ts = compaction_process->ManualDoCompact("default_db", table_name, false); + EXPECT_NE(commit_ts, 0u); } { @@ -151,26 +157,17 @@ TEST_F(CompactTaskTest, compact_to_single_segment) { txn_mgr->CommitTxn(txn5); } - infinity::InfinityContext::instance().UnInit(); -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::UnInit(); -#endif } } TEST_F(CompactTaskTest, compact_to_two_segment) { { String table_name = "tbl1"; -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::Init(); -#endif - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); + CompactionProcessor *compaction_process = storage->compaction_processor(); Vector> columns; { @@ -196,17 +193,9 @@ TEST_F(CompactTaskTest, compact_to_two_segment) { this->AddSegments(txn_mgr, table_name, segment_sizes, buffer_manager); - { // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName("default_db", table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - compact_task->Execute(); - } - txn_mgr->CommitTxn(txn4); + { + auto commit_ts = compaction_process->ManualDoCompact("default_db", table_name, false); + EXPECT_NE(commit_ts, 0u); } { auto txn5 = txn_mgr->BeginTxn(MakeUnique("check table")); @@ -232,26 +221,17 @@ TEST_F(CompactTaskTest, compact_to_two_segment) { txn_mgr->CommitTxn(txn5); } - infinity::InfinityContext::instance().UnInit(); -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::UnInit(); -#endif } } TEST_F(CompactTaskTest, compact_with_delete) { { String table_name = "tbl1"; -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::Init(); -#endif - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); + CompactionProcessor *compaction_process = storage->compaction_processor(); Vector> columns; { @@ -303,17 +283,9 @@ TEST_F(CompactTaskTest, compact_with_delete) { txn_mgr->CommitTxn(txn3); } - { // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName("default_db", table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - compact_task->Execute(); - } - txn_mgr->CommitTxn(txn4); + { + auto commit_ts = compaction_process->ManualDoCompact("default_db", table_name, false); + EXPECT_NE(commit_ts, 0u); } { auto txn5 = txn_mgr->BeginTxn(MakeUnique("check table")); @@ -335,26 +307,17 @@ TEST_F(CompactTaskTest, compact_with_delete) { txn_mgr->CommitTxn(txn5); } - infinity::InfinityContext::instance().UnInit(); -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::UnInit(); -#endif } } TEST_F(CompactTaskTest, delete_in_compact_process) { { String table_name = "tbl1"; -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::Init(); -#endif - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); Vector> columns; { @@ -407,45 +370,35 @@ TEST_F(CompactTaskTest, delete_in_compact_process) { } { // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName("default_db", table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - CompactSegmentsTaskState state; - compact_task->CompactSegments(state); + auto txn5 = txn_mgr->BeginTxn(MakeUnique("delete table")); - { - auto txn5 = txn_mgr->BeginTxn(MakeUnique("delete table")); - - Vector delete_row_ids; - for (int i = 0; i < (int)segment_sizes.size(); ++i) { - int delete_n2 = segment_sizes[i] / 4; - Vector offsets; - for (int j = 0; j < delete_n2; ++j) { - offsets.push_back(rand() % (segment_sizes[i] - segment_sizes[i] / 2) + segment_sizes[i] / 2); - } - std::sort(offsets.begin(), offsets.end()); - offsets.erase(std::unique(offsets.begin(), offsets.end()), offsets.end()); - for (SegmentOffset offset : offsets) { - delete_row_ids.emplace_back(i, offset); - } - delete_n += offsets.size(); - } - - auto [table_entry, status] = txn5->GetTableByName("default_db", table_name); - EXPECT_TRUE(status.ok()); - txn5->Delete(table_entry, delete_row_ids); - txn_mgr->CommitTxn(txn5); + Vector delete_row_ids; + for (int i = 0; i < (int)segment_sizes.size(); ++i) { + int delete_n2 = segment_sizes[i] / 4; + Vector offsets; + for (int j = 0; j < delete_n2; ++j) { + offsets.push_back(rand() % (segment_sizes[i] - segment_sizes[i] / 2) + segment_sizes[i] / 2); } - - compact_task->SaveSegmentsData(state); - compact_task->ApplyDeletes(state); + std::sort(offsets.begin(), offsets.end()); + offsets.erase(std::unique(offsets.begin(), offsets.end()), offsets.end()); + for (SegmentOffset offset : offsets) { + delete_row_ids.emplace_back(i, offset); + } + delete_n += offsets.size(); } - txn_mgr->CommitTxn(txn4); + auto [table_entry, status] = txn5->GetTableByName("default_db", table_name); + EXPECT_TRUE(status.ok()); + txn5->Delete(table_entry, delete_row_ids); + + Thread t([&]() { + auto commit_ts = compaction_processor->ManualDoCompact("default_db", table_name, false); + EXPECT_NE(commit_ts, 0u); + }); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + txn_mgr->CommitTxn(txn5); + row_count -= delete_n; + t.join(); } { auto txn5 = txn_mgr->BeginTxn(MakeUnique("check table")); @@ -463,32 +416,23 @@ TEST_F(CompactTaskTest, delete_in_compact_process) { EXPECT_NE(compact_segment, nullptr); EXPECT_NE(compact_segment->status(), SegmentStatus::kDeprecated); - EXPECT_EQ(compact_segment->actual_row_count(), row_count - delete_n); + EXPECT_EQ(compact_segment->actual_row_count(), row_count); txn_mgr->CommitTxn(txn5); } - infinity::InfinityContext::instance().UnInit(); -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::UnInit(); -#endif } } // Cannot compile the test. So annotate it. TEST_F(CompactTaskTest, uncommit_delete_in_compact_process) { - { - String table_name = "tbl1"; -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::Init(); -#endif - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); + for (int task_i = 0; task_i < 2; ++task_i) { + String table_name = fmt::format("tbl{}", task_i); Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); Vector> columns; { @@ -539,74 +483,65 @@ TEST_F(CompactTaskTest, uncommit_delete_in_compact_process) { txn_mgr->CommitTxn(txn3); } - { // add compact - auto compact_txn = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = compact_txn->GetTableByName("default_db", table_name); - EXPECT_NE(table_entry, nullptr); - + // add compact + { + Vector delete_row_ids; Vector delete_row_ids2; - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, compact_txn); - CompactSegmentsTaskState state; - compact_task->CompactSegments(state); + int delete_row_n1 = 0; + int delete_row_n2 = 0; + + for (int i = 0; i < (int)segment_sizes.size(); ++i) { + Vector offsets; + Vector offsets2; + for (int j = 0; j < (int)(segment_sizes[i] / 6); ++j) { + offsets.push_back(rand() % (segment_sizes[i] / 3) + segment_sizes[i] / 3); + offsets2.push_back(rand() % (segment_sizes[i] - segment_sizes[i] / 3 * 2) + segment_sizes[i] / 3 * 2); + } + std::sort(offsets.begin(), offsets.end()); + std::sort(offsets2.begin(), offsets2.end()); + offsets.erase(std::unique(offsets.begin(), offsets.end()), offsets.end()); + offsets2.erase(std::unique(offsets2.begin(), offsets2.end()), offsets2.end()); + for (SegmentOffset offset : offsets) { + delete_row_ids.emplace_back(i, offset); + } + for (SegmentOffset offset2 : offsets2) { + delete_row_ids2.emplace_back(i, offset2); + } - Vector delete_row_ids; + delete_row_n1 += offsets.size(); + delete_row_n2 += offsets2.size(); + } - int delete_row_n1 = 0; - int delete_row_n2 = 0; + auto delete_txn1 = txn_mgr->BeginTxn(MakeUnique("delete table")); + auto [table_entry, status] = delete_txn1->GetTableByName("default_db", table_name); + EXPECT_TRUE(status.ok()); + delete_txn1->Delete(table_entry, delete_row_ids); - for (int i = 0; i < (int)segment_sizes.size(); ++i) { - Vector offsets; - Vector offsets2; - for (int j = 0; j < (int)(segment_sizes[i] / 6); ++j) { - offsets.push_back(rand() % (segment_sizes[i] / 3) + segment_sizes[i] / 3); - offsets2.push_back(rand() % (segment_sizes[i] - segment_sizes[i] / 3 * 2) + segment_sizes[i] / 3 * 2); - } - std::sort(offsets.begin(), offsets.end()); - std::sort(offsets2.begin(), offsets2.end()); - offsets.erase(std::unique(offsets.begin(), offsets.end()), offsets.end()); - offsets2.erase(std::unique(offsets2.begin(), offsets2.end()), offsets2.end()); - for (SegmentOffset offset : offsets) { - delete_row_ids.emplace_back(i, offset); - } - for (SegmentOffset offset2 : offsets2) { - delete_row_ids2.emplace_back(i, offset2); - } + bool slow_delete2 = task_i & 1; - delete_row_n1 += offsets.size(); - delete_row_n2 += offsets2.size(); - } + auto commit_ts = compaction_processor->ManualDoCompact("default_db", table_name, false, [&]() { + txn_mgr->CommitTxn(delete_txn1); + delete_n += delete_row_n1; auto delete_txn2 = txn_mgr->BeginTxn(MakeUnique("delete table")); auto [table_entry, status] = delete_txn2->GetTableByName("default_db", table_name); EXPECT_TRUE(status.ok()); - delete_txn2->Delete(table_entry, delete_row_ids2); - { - auto delete_txn1 = txn_mgr->BeginTxn(MakeUnique("delete table")); - - auto [table_entry, status] = delete_txn1->GetTableByName("default_db", table_name); - EXPECT_TRUE(status.ok()); - delete_txn1->Delete(table_entry, delete_row_ids); - txn_mgr->CommitTxn(delete_txn1); - - delete_n += delete_row_n1; + if (slow_delete2) { + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); } - - compact_task->SaveSegmentsData(state); - { - Thread t([&]() { - // std::this_thread::sleep_for(std::chrono::seconds(1)); - txn_mgr->CommitTxn(delete_txn2); - delete_n += delete_row_n2; - }); - t.join(); + try { + delete_txn2->Delete(table_entry, delete_row_ids2); + txn_mgr->CommitTxn(delete_txn2); + LOG_INFO("Delete 2 is committed"); + delete_n += delete_row_n2; + EXPECT_FALSE(slow_delete2); + } catch (const RecoverableException &e) { + LOG_INFO("Delete 2 is row backed"); } - compact_task->ApplyDeletes(state); - } - txn_mgr->CommitTxn(compact_txn); + }); + EXPECT_NE(commit_ts, 0u); { auto txn5 = txn_mgr->BeginTxn(MakeUnique("delete table")); try { @@ -641,35 +576,14 @@ TEST_F(CompactTaskTest, uncommit_delete_in_compact_process) { txn_mgr->CommitTxn(txn5); } } - infinity::InfinityContext::instance().UnInit(); -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::UnInit(); -#endif } } TEST_F(CompactTaskTest, compact_not_exist_table) { -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::Init(); -#endif - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); - Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_mgr = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); - - auto ExpectRollback = [&](Txn *txn) { - try { - txn_mgr->CommitTxn(txn); - FAIL(); - } catch (const RecoverableException &e) { - EXPECT_EQ(e.ErrorCode(), ErrorCode::kTxnConflict); - } catch (...) { - FAIL(); - } - }; + CompactionProcessor *compaction_process = storage->compaction_processor(); String table_name = "tb1"; SharedPtr tbl1_def = nullptr; @@ -695,10 +609,6 @@ TEST_F(CompactTaskTest, compact_not_exist_table) { this->AddSegments(txn_mgr, table_name, segment_sizes, buffer_mgr); } { - auto *compact_txn = txn_mgr->BeginTxn(MakeUnique("get table")); - auto [table_entry, status] = compact_txn->GetTableByName("default_db", table_name); - ASSERT_TRUE(status.ok()); - { // drop tb1 auto drop_txn = txn_mgr->BeginTxn(MakeUnique("drop table")); auto status = drop_txn->DropTableCollectionByName("default_db", table_name, ConflictType::kError); @@ -706,11 +616,8 @@ TEST_F(CompactTaskTest, compact_not_exist_table) { txn_mgr->CommitTxn(drop_txn); } - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, compact_txn); - - compact_task->Execute(); - - ExpectRollback(compact_txn); + auto commit_ts = compaction_process->ManualDoCompact("default_db", table_name, false); + EXPECT_EQ(commit_ts, 0u); } //------------------------------------------ @@ -729,32 +636,22 @@ TEST_F(CompactTaskTest, compact_not_exist_table) { this->AddSegments(txn_mgr, table_name, segment_sizes, buffer_mgr); } { - auto *compact_txn = txn_mgr->BeginTxn(MakeUnique("get table")); - auto [table_entry, status] = compact_txn->GetTableByName("default_db", table_name); - ASSERT_TRUE(status.ok()); - - { // drop tb1 - auto drop_txn = txn_mgr->BeginTxn(MakeUnique("drop table")); - auto status = drop_txn->DropTableCollectionByName("default_db", table_name, ConflictType::kError); - ASSERT_TRUE(status.ok()); - txn_mgr->CommitTxn(drop_txn); - } - { // create table with same name - auto *txn = txn_mgr->BeginTxn(MakeUnique("create table")); - - Status status = txn->CreateTable("default_db", tbl1_def, ConflictType::kIgnore); - EXPECT_TRUE(status.ok()); + auto commit_ts = compaction_process->ManualDoCompact("default_db", table_name, false, [&]() { + { // drop tb1 + auto drop_txn = txn_mgr->BeginTxn(MakeUnique("drop table")); + auto status = drop_txn->DropTableCollectionByName("default_db", table_name, ConflictType::kError); + ASSERT_TRUE(status.ok()); + txn_mgr->CommitTxn(drop_txn); + } + { // create table with same name + auto *txn = txn_mgr->BeginTxn(MakeUnique("create table")); - txn_mgr->CommitTxn(txn); - } - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, compact_txn); - compact_task->Execute(); + Status status = txn->CreateTable("default_db", tbl1_def, ConflictType::kIgnore); + EXPECT_TRUE(status.ok()); - ExpectRollback(compact_txn); + txn_mgr->CommitTxn(txn); + } + }); + EXPECT_EQ(commit_ts, 0u); } - - infinity::InfinityContext::instance().UnInit(); -#ifdef INFINITY_DEBUG - infinity::GlobalResourceUsage::UnInit(); -#endif } \ No newline at end of file diff --git a/src/unit_test/storage/compaction/DBT_compaction_alg.cpp b/src/unit_test/storage/compaction/DBT_compaction_alg.cpp index c5d26d7764..3991afa5d7 100644 --- a/src/unit_test/storage/compaction/DBT_compaction_alg.cpp +++ b/src/unit_test/storage/compaction/DBT_compaction_alg.cpp @@ -15,12 +15,8 @@ #include "unit_test/base_test.h" import stl; -import storage; -import txn_manager; -import infinity_context; import segment_entry; import DBT_compaction_alg; -import txn; using namespace infinity; @@ -61,17 +57,11 @@ SegmentID MockSegmentEntry::cur_segment_id_ = 0; class DBTCompactionTest : public BaseTest {}; TEST_F(DBTCompactionTest, AddSegments) { - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); - - Storage *storage = infinity::InfinityContext::instance().storage(); - TxnManager *txn_mgr = storage->txn_manager(); + TransactionID txn_id = 0; int m = 3; int c = 3; int s = 1; - std::function GetTxn = [&]() { return txn_mgr->BeginTxn(MakeUnique("compact")); }; DBTCompactionAlg DBTCompact(m, c, s, MockSegmentEntry::segment_capacity); DBTCompact.Enable(Vector{}); @@ -81,26 +71,22 @@ TEST_F(DBTCompactionTest, AddSegments) { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(2); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(2); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; + auto segments = DBTCompact.CheckCompaction(++txn_id); - TransactionID txn_id = txn->TxnID(); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -110,28 +96,22 @@ TEST_F(DBTCompactionTest, AddSegments) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(3); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(5); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -141,42 +121,36 @@ TEST_F(DBTCompactionTest, AddSegments) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(3); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(6); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(9); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(11); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -186,34 +160,25 @@ TEST_F(DBTCompactionTest, AddSegments) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(100); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - - infinity::InfinityContext::instance().UnInit(); } TEST_F(DBTCompactionTest, AddAndDeleteInSegments) { - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); - - Storage *storage = infinity::InfinityContext::instance().storage(); - TxnManager *txn_mgr = storage->txn_manager(); + TransactionID txn_id = 0; int m = 3; int c = 3; int s = 1; - std::function GetTxn = [&]() { return txn_mgr->BeginTxn(MakeUnique("compact")); }; DBTCompactionAlg DBTCompact(m, c, s, MockSegmentEntry::segment_capacity); DBTCompact.Enable(Vector{}); @@ -223,26 +188,21 @@ TEST_F(DBTCompactionTest, AddAndDeleteInSegments) { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -253,33 +213,28 @@ TEST_F(DBTCompactionTest, AddAndDeleteInSegments) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { new_segment->ShrinkSegment(2); DBTCompact.DeleteInSegment(new_segment->segment_id()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto [segments, txn] = ret.value(); - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -290,34 +245,28 @@ TEST_F(DBTCompactionTest, AddAndDeleteInSegments) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(1); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { new_segment->ShrinkSegment(1); DBTCompact.DeleteInSegment(new_segment->segment_id()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -327,28 +276,19 @@ TEST_F(DBTCompactionTest, AddAndDeleteInSegments) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } - - infinity::InfinityContext::instance().UnInit(); } TEST_F(DBTCompactionTest, FillSegmentCapacity) { - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); - - Storage *storage = infinity::InfinityContext::instance().storage(); - TxnManager *txn_mgr = storage->txn_manager(); + TransactionID txn_id = 0; int m = 4; int c = 10; int s = 10; // layer0: 0~9, layer1: 10~99, layer2: 100~999 - std::function GetTxn = [&]() { return txn_mgr->BeginTxn(MakeUnique("compact")); }; DBTCompactionAlg DBTCompact(m, c, s, MockSegmentEntry::segment_capacity); DBTCompact.Enable(Vector{}); @@ -357,32 +297,28 @@ TEST_F(DBTCompactionTest, FillSegmentCapacity) { auto segment_entry = MockSegmentEntry::Make(200); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(200); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(200); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(200); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto [segments, txn] = ret.value(); - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 4u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -392,42 +328,33 @@ TEST_F(DBTCompactionTest, FillSegmentCapacity) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(200); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(200); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - - infinity::InfinityContext::instance().UnInit(); } // add rollback test, rollback every compact operation once before success commit it TEST_F(DBTCompactionTest, RollbackTest) { - std::shared_ptr config_path = nullptr; - RemoveDbDirs(); - infinity::InfinityContext::instance().Init(config_path); - - Storage *storage = infinity::InfinityContext::instance().storage(); - TxnManager *txn_mgr = storage->txn_manager(); + TransactionID txn_id = 0; int m = 3; int c = 3; int s = 1; - std::function GetTxn = [&]() { return txn_mgr->BeginTxn(MakeUnique("compact")); }; DBTCompactionAlg DBTCompact(m, c, s, MockSegmentEntry::segment_capacity); DBTCompact.Enable(Vector{}); @@ -437,35 +364,27 @@ TEST_F(DBTCompactionTest, RollbackTest) { auto segment_entry = MockSegmentEntry::Make(2); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(2); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(2); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_FALSE(segments.empty()); DBTCompact.RollbackCompact(txn_id); - txn_mgr->CommitTxn(txn); } { - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_FALSE(segments.empty()); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -475,38 +394,27 @@ TEST_F(DBTCompactionTest, RollbackTest) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(8); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(6); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_FALSE(segments.empty()); DBTCompact.RollbackCompact(txn_id); - txn_mgr->CommitTxn(txn); } { - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - // auto [segments, txn] = ret.value(); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -516,50 +424,41 @@ TEST_F(DBTCompactionTest, RollbackTest) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(3); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(6); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(14); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { auto segment_entry = MockSegmentEntry::Make(17); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_FALSE(segments.empty()); DBTCompact.RollbackCompact(txn_id); - txn_mgr->CommitTxn(txn); } { - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto [segments, txn] = ret.value(); - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); EXPECT_EQ(compacted_segments.size(), 1u); @@ -570,36 +469,27 @@ TEST_F(DBTCompactionTest, RollbackTest) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } { auto segment_entry = MockSegmentEntry::Make(100); segment_entries.emplace_back(segment_entry); DBTCompact.AddSegment(segment_entry.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_FALSE(ret.has_value()); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } { shrink_segment->ShrinkSegment(48); DBTCompact.DeleteInSegment(shrink_segment->segment_id()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_FALSE(segments.empty()); DBTCompact.RollbackCompact(txn_id); - txn_mgr->CommitTxn(txn); } { - auto ret = DBTCompact.CheckCompaction(GetTxn); - EXPECT_TRUE(ret.has_value()); - const auto &segments = ret->segments_; - auto *txn = ret->txn_; - - TransactionID txn_id = txn->TxnID(); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_FALSE(segments.empty()); EXPECT_EQ(segments.size(), 3u); auto compacted_segments = MockSegmentEntry::MockCompact(segments); segment_entries.insert(segment_entries.end(), compacted_segments.begin(), compacted_segments.end()); @@ -609,11 +499,8 @@ TEST_F(DBTCompactionTest, RollbackTest) { DBTCompact.CommitCompact(txn_id); for (auto &segment : compacted_segments) { DBTCompact.AddSegment(segment.get()); - auto ret = DBTCompact.CheckCompaction(GetTxn); - ASSERT_EQ(ret.has_value(), false); + auto segments = DBTCompact.CheckCompaction(++txn_id); + EXPECT_TRUE(segments.empty()); } - txn_mgr->CommitTxn(txn); } - - infinity::InfinityContext::instance().UnInit(); } diff --git a/src/unit_test/storage/wal/catalog_delta_replay.cpp b/src/unit_test/storage/wal/catalog_delta_replay.cpp index fa0945cced..3cf33c3fa0 100644 --- a/src/unit_test/storage/wal/catalog_delta_replay.cpp +++ b/src/unit_test/storage/wal/catalog_delta_replay.cpp @@ -34,7 +34,6 @@ import buffer_manager; import physical_import; import status; import compilation_config; -import compact_segments_task; import index_base; import index_base; import third_party; @@ -51,6 +50,7 @@ import infinity_exception; import default_values; import block_index; import wal_manager; +import compaction_process; using namespace infinity; @@ -79,7 +79,7 @@ class CatalogDeltaReplayTest : public BaseTest { auto *txn = txn_mgr->BeginTxn(MakeUnique("import data")); auto [table_entry, status] = txn->GetTableByName("default_db", table_name); - table_entry->SetCompactionAlg(nullptr); // close auto compaction to test manual compaction + // table_entry->SetCompactionAlg(nullptr); // close auto compaction to test manual compaction auto column_count = table_entry->ColumnCount(); SegmentID segment_id = Catalog::GetNextSegmentID(table_entry); @@ -704,6 +704,7 @@ TEST_F(CatalogDeltaReplayTest, replay_compact_to_single_rollback) { Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); TxnManager *txn_mgr = storage->txn_manager(); Vector> columns; @@ -730,18 +731,8 @@ TEST_F(CatalogDeltaReplayTest, replay_compact_to_single_rollback) { this->AddSegments(txn_mgr, table_name, segment_sizes, buffer_manager); { - // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName("default_db", table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - compact_task->Execute(); - } - // rollback - txn_mgr->RollBackTxn(txn4); + auto commit_ts = compaction_processor->ManualDoCompact("default_db", table_name, true); + EXPECT_EQ(commit_ts, 0u); } { @@ -1104,6 +1095,7 @@ TEST_F(CatalogDeltaReplayTest, replay_table_single_index_and_compact) { InfinityContext::instance().Init(config_path); Storage *storage = InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); TxnManager *txn_mgr = storage->txn_manager(); TxnTimeStamp last_commit_ts = 0; @@ -1225,18 +1217,8 @@ TEST_F(CatalogDeltaReplayTest, replay_table_single_index_and_compact) { this->AddSegments(txn_mgr, *table_name, segment_sizes, buffer_manager); { - - auto txn_cpt = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn_cpt->GetTableByName(*db_name, *table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn_cpt); - compact_task->Execute(); - } - - last_commit_ts = txn_mgr->CommitTxn(txn_cpt); + last_commit_ts = compaction_processor->ManualDoCompact(*db_name, *table_name, false); + EXPECT_NE(last_commit_ts, 0u); } { diff --git a/src/unit_test/storage/wal/checkpoint.cpp b/src/unit_test/storage/wal/checkpoint.cpp index 7159c58956..439046f2f5 100644 --- a/src/unit_test/storage/wal/checkpoint.cpp +++ b/src/unit_test/storage/wal/checkpoint.cpp @@ -34,7 +34,6 @@ import buffer_manager; import physical_import; import status; import compilation_config; -import compact_segments_task; import index_base; import index_base; import third_party; @@ -52,6 +51,7 @@ import default_values; import global_resource_usage; import infinity; import background_process; +import compaction_process; import wal_manager; using namespace infinity; @@ -117,7 +117,7 @@ class CheckpointTest : public BaseTest { auto *txn = txn_mgr->BeginTxn(MakeUnique("import table")); auto [table_entry, status] = txn->GetTableByName("default_db", table_name); - table_entry->SetCompactionAlg(nullptr); // close auto compaction to test manual compaction + // table_entry->SetCompactionAlg(nullptr); // close auto compaction to test manual compaction auto column_count = table_entry->ColumnCount(); SegmentID segment_id = Catalog::GetNextSegmentID(table_entry); @@ -162,6 +162,7 @@ TEST_F(CheckpointTest, test_cleanup_and_checkpoint) { BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); TxnTimeStamp last_commit_ts = 0; + CompactionProcessor *compaction_processor = storage->compaction_processor(); Vector> columns; { @@ -186,16 +187,8 @@ TEST_F(CheckpointTest, test_cleanup_and_checkpoint) { this->AddSegments(txn_mgr, *table_name, segment_sizes, buffer_manager); { // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName(*db_name, *table_name); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - compact_task->Execute(); - } - txn_mgr->CommitTxn(txn4); + auto commit_ts = compaction_processor->ManualDoCompact(*db_name, *table_name, false); + EXPECT_NE(commit_ts, 0u); } { diff --git a/src/unit_test/storage/wal/wal_replay.cpp b/src/unit_test/storage/wal/wal_replay.cpp index 8da6f5a3af..0bc48b8e8f 100644 --- a/src/unit_test/storage/wal/wal_replay.cpp +++ b/src/unit_test/storage/wal/wal_replay.cpp @@ -41,7 +41,6 @@ import index_hnsw; import index_full_text; import bg_task; import background_process; -import compact_segments_task; import default_values; import base_table_ref; import internal_types; @@ -59,6 +58,7 @@ import block_column_entry; import table_index_entry; import base_entry; import compilation_config; +import compaction_process; using namespace infinity; @@ -679,6 +679,7 @@ TEST_F(WalReplayTest, wal_replay_compact) { Storage *storage = infinity::InfinityContext::instance().storage(); BufferManager *buffer_manager = storage->buffer_manager(); TxnManager *txn_mgr = storage->txn_manager(); + CompactionProcessor *compaction_processor = storage->compaction_processor(); Vector> columns; { @@ -738,16 +739,8 @@ TEST_F(WalReplayTest, wal_replay_compact) { } { // add compact - auto txn4 = txn_mgr->BeginTxn(MakeUnique("compact table")); - - auto [table_entry, status] = txn4->GetTableByName("default_db", "tbl1"); - EXPECT_NE(table_entry, nullptr); - - { - auto compact_task = CompactSegmentsTask::MakeTaskWithWholeTable(table_entry, txn4); - compact_task->Execute(); - } - txn_mgr->CommitTxn(txn4); + auto commit_ts = compaction_processor->ManualDoCompact("default_db", "tbl1", false); + EXPECT_NE(commit_ts, 0u); } infinity::InfinityContext::instance().UnInit(); #ifdef INFINITY_DEBUG diff --git a/src/unit_test/test_helper/sql_runner.cpp b/src/unit_test/test_helper/sql_runner.cpp index 5006fb4328..0377b0afa0 100644 --- a/src/unit_test/test_helper/sql_runner.cpp +++ b/src/unit_test/test_helper/sql_runner.cpp @@ -90,7 +90,7 @@ SharedPtr SQLRunner::Run(const String &sql_text, bool print) { query_context_ptr->logical_planner()->Build(statement, bind_context); query_context_ptr->set_max_node_id(bind_context->GetNewLogicalNodeId()); - SharedPtr logical_plan = query_context_ptr->logical_planner()->LogicalPlan(); + SharedPtr logical_plan = query_context_ptr->logical_planner()->LogicalPlans()[0]; // Apply optimized rule to the logical plan query_context_ptr->optimizer()->optimize(logical_plan, statement->Type()); @@ -100,7 +100,7 @@ SharedPtr SQLRunner::Run(const String &sql_text, bool print) { // Create execution pipeline // Fragment Builder, only for test now. plan fragment is same as pipeline. - auto plan_fragment = query_context_ptr->fragment_builder()->BuildFragment(physical_plan.get()); + auto plan_fragment = query_context_ptr->fragment_builder()->BuildFragment({physical_plan.get()}); auto notifier = MakeUnique(); diff --git a/test/data/csv/embedding_2.csv b/test/data/csv/embedding_2.csv new file mode 100644 index 0000000000..8fccd07756 --- /dev/null +++ b/test/data/csv/embedding_2.csv @@ -0,0 +1,4 @@ +"[0.1, 0.1]","[0.1, 0.2, 0.3, -0.2]","[0.1, 0.2, 0.3, -0.2]" +"[0.2, 0.2]","[0.2, 0.1, 0.3, 0.4]","[0.2, 0.1, 0.3, 0.4]" +"[0.3, 0.3]","[0.3, 0.2, 0.1, 0.4]","[0.3, 0.2, 0.1, 0.4]" +"[0.4, 0.4]","[0.4, 0.3, 0.2, 0.1]","[0.4, 0.3, 0.2, 0.1]" \ No newline at end of file diff --git a/test/data/csv/embedding_3.csv b/test/data/csv/embedding_3.csv new file mode 100644 index 0000000000..3f269dafa1 --- /dev/null +++ b/test/data/csv/embedding_3.csv @@ -0,0 +1,4 @@ +1,"[0.1, 0.2, 0.3, -0.2]","[0.1, 0.2, 0.3, -0.2]" +2,"[0.2, 0.1, 0.3, 0.4]","[0.2, 0.1, 0.3, 0.4]" +3,"[0.3, 0.2, 0.1, 0.4]","[0.3, 0.2, 0.1, 0.4]" +4,"[0.4, 0.3, 0.2, 0.1]","[0.4, 0.3, 0.2, 0.1]" \ No newline at end of file diff --git a/test/sql/dml/compact/test_compact_many_index.slt b/test/sql/dml/compact/test_compact_many_index.slt new file mode 100644 index 0000000000..ffa492396a --- /dev/null +++ b/test/sql/dml/compact/test_compact_many_index.slt @@ -0,0 +1,75 @@ +statement ok +DROP TABLE IF EXISTS tbl1; + +statement ok +CREATE TABLE tbl1(c1 EMBEDDING(FLOAT,2), c2 EMBEDDING(FLOAT, 4), c3 EMBEDDING(FLOAT, 4)); + +statement ok +COPY tbl1 FROM '/var/infinity/test_data/embedding_2.csv' WITH (DELIMITER ','); + +statement ok +COPY tbl1 FROM '/var/infinity/test_data/embedding_2.csv' WITH (DELIMITER ','); + +statement ok +CREATE INDEX idx1 ON tbl1 (c1) USING Hnsw WITH (M = 16, ef_construction = 200, metric = l2); + +statement ok +CREATE INDEX idx2 ON tbl1 (c2) USING Hnsw WITH (M = 16, ef_construction = 200, metric = l2); + +statement ok +CREATE INDEX idx3 ON tbl1 (c3) USING Hnsw WITH (M = 16, ef_construction = 200, metric = l2); + +statement ok +COMPACT TABLE tbl1; + +query I +SELECT c1 FROM tbl1 SEARCH KNN(c2, [0.3, 0.3, 0.2, 0.2], 'float', 'l2', 3); +---- +0.4,0.4 +0.4,0.4 +0.3,0.3 + +query I +SELECT c1 FROM tbl1 SEARCH KNN(c3, [0.3, 0.3, 0.2, 0.2], 'float', 'l2', 3); +---- +0.4,0.4 +0.4,0.4 +0.3,0.3 + +statement ok +DROP TABLE IF EXISTS tbl2; + +statement ok +CREATE TABLE tbl2(c1 INT, c2 EMBEDDING(FLOAT, 4), c3 EMBEDDING(FLOAT, 4)); + +statement ok +COPY tbl2 FROM '/var/infinity/test_data/embedding_3.csv' WITH (DELIMITER ','); + +statement ok +COPY tbl2 FROM '/var/infinity/test_data/embedding_3.csv' WITH (DELIMITER ','); + +statement ok +CREATE INDEX idx1 ON tbl2 (c1); + +statement ok +CREATE INDEX idx2 ON tbl2 (c2) USING Hnsw WITH (M = 16, ef_construction = 200, metric = l2); + +statement ok +CREATE INDEX idx3 ON tbl2 (c3) USING Hnsw WITH (M = 16, ef_construction = 200, metric = l2); + +statement ok +COMPACT TABLE tbl2; + +query I +SELECT c1 FROM tbl2 SEARCH KNN(c2, [0.3, 0.3, 0.2, 0.2], 'float', 'l2', 3); +---- +4 +4 +3 + +query I +SELECT c1 FROM tbl2 SEARCH KNN(c3, [0.3, 0.3, 0.2, 0.2], 'float', 'l2', 3); +---- +4 +4 +3 diff --git a/tools/generate_compact.py b/tools/generate_compact.py index d48815b752..4521158533 100644 --- a/tools/generate_compact.py +++ b/tools/generate_compact.py @@ -31,6 +31,7 @@ def generate(generate_if_exists: bool, copy_dir: str): random.shuffle(x) delete_x = set(random.sample(range(row_n), row_n // 100)) y = [False if i in delete_x else True for i in x] + true_num = sum(y) with open(csv_path, "w") as csv_file: for x1, y1 in zip(x, y): @@ -64,11 +65,14 @@ def generate(generate_if_exists: bool, copy_dir: str): slt_file.write("\n") slt_file.write("query I\n") - slt_file.write("SELECT c1 FROM {};\n".format(table_name)) + slt_file.write("SELECT COUNT(*) FROM {};\n".format(table_name)) + slt_file.write("----\n") + slt_file.write("{}\n".format(true_num)) + slt_file.write("\n") + + slt_file.write("query I\n") + slt_file.write("SELECT c1 FROM {} WHERE c2 = False;\n".format(table_name)) slt_file.write("----\n") - for x1, y1 in zip(x, y): - if y1: - slt_file.write("{}\n".format(x1)) slt_file.write("\n") slt_file.write("statement ok\n") diff --git a/tools/generate_many_import.py b/tools/generate_many_import.py index ae6eacdb06..14eeda0700 100644 --- a/tools/generate_many_import.py +++ b/tools/generate_many_import.py @@ -57,23 +57,26 @@ def generate(generate_if_exists: bool, copy_dir: str): slt_file.write("\n") slt_file.write("statement ok\n") - slt_file.write("DROP TABLE {};\n".format(table_name)) + slt_file.write("COMPACT TABLE {};\n".format(table_name)) slt_file.write("\n") - # # The delete will throw exception when compacting, so add this to wait for sometime - # slt_file.write("statement ok\n") - # slt_file.write("SELECT * FROM {};\n".format(table_name)) - # slt_file.write("\n") + for v in range(max_v): + slt_file.write("statement ok\n") + slt_file.write("DELETE FROM {} WHERE c1 = {};\n".format(table_name, v)) + slt_file.write("\n") + + slt_file.write("statement ok\n") + slt_file.write("COMPACT TABLE {};\n".format(table_name)) + slt_file.write("\n") - # for v in range(max_v): - # slt_file.write("statement ok\n") - # slt_file.write("DELETE FROM {} WHERE c1 = {};\n".format(table_name, v)) - # slt_file.write("\n") + slt_file.write("query I\n") + slt_file.write("SELECT * FROM {};\n".format(table_name)) + slt_file.write("----\n") + slt_file.write("\n") - # slt_file.write("query I\n") - # slt_file.write("SELECT * FROM {};\n".format(table_name)) - # slt_file.write("----\n") - # slt_file.write("\n") + slt_file.write("statement ok\n") + slt_file.write("DROP TABLE {};\n".format(table_name)) + slt_file.write("\n") if __name__ == "__main__":