From b82b0ba7d3bd21ebc4a9329a0602fcf9195f4fad Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Thu, 16 Mar 2023 15:00:39 +0800 Subject: [PATCH] Support late materialization (#6966) ref pingcap/tiflash#5829 --- contrib/tipb | 2 +- dbms/src/Core/Block.cpp | 10 +- dbms/src/Core/Block.h | 15 +- dbms/src/DataStreams/FilterBlockInputStream.h | 2 + .../src/DataStreams/FilterTransformAction.cpp | 12 +- dbms/src/DataStreams/FilterTransformAction.h | 3 +- dbms/src/Debug/MockStorage.cpp | 5 +- .../Coprocessor/DAGExpressionAnalyzer.cpp | 40 +- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 10 +- dbms/src/Flash/Coprocessor/DAGQueryInfo.h | 11 +- .../Coprocessor/DAGStorageInterpreter.cpp | 60 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 18 + dbms/src/Flash/Coprocessor/DAGUtils.h | 1 + .../Flash/Coprocessor/InterpreterUtils.cpp | 9 +- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 3 +- dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 1 + dbms/src/Flash/Coprocessor/TiDBTableScan.h | 11 + .../DeltaMerge/BitmapFilter/BitmapFilter.cpp | 10 + .../DeltaMerge/BitmapFilter/BitmapFilter.h | 2 + .../BitmapFilterBlockInputStream.cpp | 36 +- .../BitmapFilterBlockInputStream.h | 11 +- .../DeltaMerge/ColumnFile/ColumnFileBig.cpp | 2 +- .../DeltaMerge/DMSegmentThreadInputStream.h | 4 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 9 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 5 +- .../Storages/DeltaMerge/File/DMFileReader.cpp | 51 +- .../DeltaMerge/Filter/PushDownFilter.h | 61 ++ .../Storages/DeltaMerge/Filter/RSOperator.h | 2 +- .../DeltaMerge/FilterParser/FilterParser.cpp | 20 +- .../DeltaMerge/FilterParser/FilterParser.h | 4 +- .../LateMaterializationBlockInputStream.cpp | 36 +- .../LateMaterializationBlockInputStream.h | 6 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 155 +++- dbms/src/Storages/DeltaMerge/Segment.h | 17 +- .../Storages/DeltaMerge/SegmentReadTaskPool.h | 6 +- .../Storages/DeltaMerge/StableValueSpace.cpp | 4 +- .../tests/gtest_dm_delta_merge_store.cpp | 2 +- .../tests/gtest_dm_minmax_index.cpp | 2 +- .../DeltaMerge/tests/gtest_segment_bitmap.cpp | 148 +-- .../tests/gtest_segment_test_basic.cpp | 1 + .../DeltaMerge/tests/gtest_segment_util.cpp | 154 ++++ .../DeltaMerge/tests/gtest_segment_util.h | 56 ++ .../gtest_skippable_block_input_stream.cpp | 182 ++-- dbms/src/Storages/StorageDeltaMerge.cpp | 119 ++- dbms/src/Storages/StorageDeltaMerge.h | 18 +- .../Storages/StorageDisaggregatedRemote.cpp | 7 +- .../Storages/tests/gtest_filter_parser.cpp | 6 +- .../tests/gtests_parse_push_down_filter.cpp | 856 ++++++++++++++++++ 48 files changed, 1751 insertions(+), 454 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h create mode 100644 dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp diff --git a/contrib/tipb b/contrib/tipb index df70cb49883..5362260ee6f 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit df70cb4988354363fefcd20272054bfbf45cfc3f +Subproject commit 5362260ee6f7720ffeca48e8797ef76c31ac0f98 diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 62174b84eb7..4aa9a4d4849 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -530,14 +530,16 @@ Block hstackBlocks(Blocks && blocks, const Block & header) return {}; Block res = header.cloneEmpty(); - size_t num_rows = blocks.front().rows(); for (const auto & block : blocks) { RUNTIME_CHECK_MSG(block.rows() == num_rows, "Cannot hstack blocks with different number of rows"); for (const auto & elem : block) { - res.getByName(elem.name).column = std::move(elem.column); + if (likely(res.has(elem.name))) + { + res.getByName(elem.name).column = std::move(elem.column); + } } } @@ -662,12 +664,12 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out WriteBufferFromString lhs_diff_writer(out_lhs_diff); WriteBufferFromString rhs_diff_writer(out_rhs_diff); - for (auto it = left_columns.rbegin(); it != left_columns.rend(); ++it) + for (auto it = left_columns.rbegin(); it != left_columns.rend(); ++it) // NOLINT { lhs_diff_writer << it->dumpStructure(); lhs_diff_writer << ", position: " << lhs.getPositionByName(it->name) << '\n'; } - for (auto it = right_columns.rbegin(); it != right_columns.rend(); ++it) + for (auto it = right_columns.rbegin(); it != right_columns.rend(); ++it) // NOLINT { rhs_diff_writer << it->dumpStructure(); rhs_diff_writer << ", position: " << rhs.getPositionByName(it->name) << '\n'; diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 73184ab48e5..ff7bea3cb91 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -172,10 +172,21 @@ using Blocks = std::vector; using BlocksList = std::list; using BucketBlocksListMap = std::map; -/// join blocks by columns +/// Join blocks by columns +/// The schema of the output block is the same as the header block. +/// The columns not in the header block will be ignored. +/// For example: +/// header: (a UInt32, b UInt32, c UInt32, d UInt32) +/// block1: (a UInt32, b UInt32, c UInt32, e UInt32), rows: 3 +/// block2: (d UInt32), rows: 3 +/// result: (a UInt32, b UInt32, c UInt32, d UInt32), rows: 3 Block hstackBlocks(Blocks && blocks, const Block & header); -/// join blocks by rows +/// Join blocks by rows +/// For example: +/// block1: (a UInt32, b UInt32, c UInt32), rows: 2 +/// block2: (a UInt32, b UInt32, c UInt32), rows: 3 +/// result: (a UInt32, b UInt32, c UInt32), rows: 5 Block vstackBlocks(Blocks && blocks); Block popBlocksListFront(BlocksList & blocks); diff --git a/dbms/src/DataStreams/FilterBlockInputStream.h b/dbms/src/DataStreams/FilterBlockInputStream.h index 9fea8fe1440..741349ca357 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.h +++ b/dbms/src/DataStreams/FilterBlockInputStream.h @@ -49,6 +49,8 @@ class FilterBlockInputStream : public IProfilingBlockInputStream return readImpl(filter_ignored, false); } + // Note: When return_filter is true, res_filter will be point to the filter column of the returned block. + // If res_filter is nullptr, it means the filter conditions are always true. Block readImpl(FilterPtr & res_filter, bool return_filter) override; private: diff --git a/dbms/src/DataStreams/FilterTransformAction.cpp b/dbms/src/DataStreams/FilterTransformAction.cpp index 474b7b46b5e..b628dd43123 100644 --- a/dbms/src/DataStreams/FilterTransformAction.cpp +++ b/dbms/src/DataStreams/FilterTransformAction.cpp @@ -114,18 +114,18 @@ bool FilterTransformAction::transform(Block & block, FilterPtr & res_filter, boo filter_holder = filter_and_holder.data_holder; } - size_t filtered_rows = countBytesInFilter(*filter); - - /// If the current block is completely filtered out, let's move on to the next one. - if (filtered_rows == 0) - return false; - if (return_filter) { res_filter = filter; return true; } + size_t filtered_rows = countBytesInFilter(*filter); + + /// If the current block is completely filtered out, let's move on to the next one. + if (filtered_rows == 0) + return false; + /// If all the rows pass through the filter. if (filtered_rows == rows) { diff --git a/dbms/src/DataStreams/FilterTransformAction.h b/dbms/src/DataStreams/FilterTransformAction.h index a7b664cf881..092b98b1898 100644 --- a/dbms/src/DataStreams/FilterTransformAction.h +++ b/dbms/src/DataStreams/FilterTransformAction.h @@ -32,7 +32,8 @@ struct FilterTransformAction bool alwaysFalse() const; // return false if all filter out. - // if return_filter is true and all rows are passed, set res_filter = nullptr. + // When return_filter is true, res_filter will be set to the filter column. + // Always return true, and when filter conditions are always true, set res_filter = nullptr. bool transform(Block & block, FilterPtr & res_filter, bool return_filter); Block getHeader() const; ExpressionActionsPtr getExperssion() const; diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 67451bb0162..27b62b5b250 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -153,7 +153,6 @@ std::tuple MockStorage::prepareFor column_names.push_back(column_info.name); auto scan_context = std::make_shared(); - SelectQueryInfo query_info; query_info.query = std::make_shared(); query_info.keep_order = false; @@ -170,11 +169,11 @@ BlockInputStreamPtr MockStorage::getStreamFromDeltaMerge(Context & context, Int6 auto analyzer = std::make_unique(names_and_types_map_for_delta_merge[table_id], context); query_info.dag_query = std::make_unique( filter_conditions->conditions, + google::protobuf::RepeatedPtrField{}, // Not care now analyzer->getPreparedSets(), analyzer->getCurrentInputColumns(), context.getTimezoneInfo()); - auto [before_where, filter_column_name, project_after_where] = ::DB::buildPushDownFilter(*filter_conditions, *analyzer); - + auto [before_where, filter_column_name, project_after_where] = ::DB::buildPushDownFilter(filter_conditions->conditions, *analyzer); BlockInputStreams ins = storage->read(column_names, query_info, context, stage, 8192, 1); // TODO: Support config max_block_size and num_streams // TODO: set num_streams, then ins.size() != 1 BlockInputStreamPtr in = ins[0]; diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index d3d8b8f3726..7cb65e121f7 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -898,13 +898,14 @@ String DAGExpressionAnalyzer::appendTimeZoneCast( return cast_expr_name; } -bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( +std::pair> DAGExpressionAnalyzer::buildExtraCastsAfterTS( const ExpressionActionsPtr & actions, const std::vector & need_cast_column, const ColumnInfos & table_scan_columns) { bool has_cast = false; - + std::vector casted_columns; + casted_columns.reserve(need_cast_column.size()); // For TimeZone tipb::Expr tz_expr = constructTZExpr(context.getTimezoneInfo()); String tz_col = getActions(tz_expr, actions); @@ -917,10 +918,10 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( static const String dur_func_name = "FunctionConvertDurationFromNanos"; for (size_t i = 0; i < need_cast_column.size(); ++i) { + String casted_name = source_columns[i].name; if (!context.getTimezoneInfo().is_utc_timezone && need_cast_column[i] == ExtraCastAfterTSMode::AppendTimeZoneCast) { - String casted_name = appendTimeZoneCast(tz_col, source_columns[i].name, timezone_func_name, actions); - source_columns[i].name = casted_name; + casted_name = appendTimeZoneCast(tz_col, source_columns[i].name, timezone_func_name, actions); has_cast = true; } @@ -931,18 +932,17 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( const auto fsp = table_scan_columns[i].decimal < 0 ? 0 : table_scan_columns[i].decimal; tipb::Expr fsp_expr = constructInt64LiteralTiExpr(fsp); fsp_col = getActions(fsp_expr, actions); - String casted_name = appendDurationCast(fsp_col, source_columns[i].name, dur_func_name, actions); - source_columns[i].name = casted_name; + casted_name = appendDurationCast(fsp_col, source_columns[i].name, dur_func_name, actions); + // We will replace the source_columns[i] with the casted column later + // so we need to update the type of the source_column[i] source_columns[i].type = actions->getSampleBlock().getByName(casted_name).type; has_cast = true; } + + casted_columns.emplace_back(std::move(casted_name)); } - NamesWithAliases project_cols; - for (auto & col : source_columns) - project_cols.emplace_back(col.name, col.name); - actions->add(ExpressionAction::project(project_cols)); - return has_cast; + return {has_cast, casted_columns}; } bool DAGExpressionAnalyzer::appendExtraCastsAfterTS( @@ -951,13 +951,27 @@ bool DAGExpressionAnalyzer::appendExtraCastsAfterTS( const TiDBTableScan & table_scan) { auto & step = initAndGetLastStep(chain); + auto & actions = step.actions; - bool has_cast = buildExtraCastsAfterTS(step.actions, need_cast_column, table_scan.getColumns()); + auto [has_cast, casted_columns] = buildExtraCastsAfterTS(actions, need_cast_column, table_scan.getColumns()); + + if (!has_cast) + return false; + + // Add a projection to replace the original columns with the casted columns. + // For example: + // we have a block with columns (a int64, b float, c int64) + // after the cast, the block will be (a int64, b float, c int64, casted_c MyDuration) + // After this projection, the block will be (a int64, b float, c MyDuration) + NamesWithAliases project_cols; + for (size_t i = 0; i < need_cast_column.size(); ++i) + project_cols.emplace_back(casted_columns[i], source_columns[i].name); + actions->add(ExpressionAction::project(project_cols)); for (auto & col : source_columns) step.required_output.push_back(col.name); - return has_cast; + return true; } String DAGExpressionAnalyzer::appendDurationCast( diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 60a8ffe40fb..047cc528437 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -204,6 +204,11 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const tipb::Aggregation & agg); + std::pair> buildExtraCastsAfterTS( + const ExpressionActionsPtr & actions, + const std::vector & need_cast_column, + const ColumnInfos & table_scan_columns); + #ifndef DBMS_PUBLIC_GTEST private: #endif @@ -285,11 +290,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const String & expr_name, bool force_uint8); - bool buildExtraCastsAfterTS( - const ExpressionActionsPtr & actions, - const std::vector & need_cast_column, - const ColumnInfos & table_scan_columns); - /// @ret: if some new expression actions are added. /// @key_names: column names of keys. /// @original_key_names: original column names of keys.(only used for null-aware semi join) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h index 5d95ce7dcd6..07fbe9f97f5 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -28,19 +28,24 @@ struct DAGQueryInfo { DAGQueryInfo( const google::protobuf::RepeatedPtrField & filters_, + const google::protobuf::RepeatedPtrField & pushed_down_filters_, DAGPreparedSets dag_sets_, const NamesAndTypes & source_columns_, const TimezoneInfo & timezone_info_) - : filters(filters_) + : source_columns(source_columns_) + , filters(filters_) + , pushed_down_filters(pushed_down_filters_) , dag_sets(std::move(dag_sets_)) - , source_columns(source_columns_) , timezone_info(timezone_info_){}; + + const NamesAndTypes & source_columns; // filters in dag request const google::protobuf::RepeatedPtrField & filters; + // filters have been push down to storage engine in dag request + const google::protobuf::RepeatedPtrField & pushed_down_filters; // Prepared sets extracted from dag request, which are used for indices // by storage engine. DAGPreparedSets dag_sets; - const NamesAndTypes & source_columns; const TimezoneInfo & timezone_info; }; diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 67f78e1ee00..07e62424847 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -175,7 +175,7 @@ bool hasRegionToRead(const DAGContext & dag_context, const TiDBTableScan & table // add timezone cast for timestamp type, this is used to support session level timezone // -std::tuple addExtraCastsAfterTs( +std::pair addExtraCastsAfterTs( DAGExpressionAnalyzer & analyzer, const std::vector & need_cast_column, const TiDBTableScan & table_scan) @@ -184,11 +184,9 @@ std::tuple addExtraCastsAfterT for (auto b : need_cast_column) has_need_cast_column |= (b != ExtraCastAfterTSMode::None); if (!has_need_cast_column) - return {false, nullptr, nullptr}; + return {false, nullptr}; ExpressionActionsChain chain; - analyzer.initChain(chain); - auto original_source_columns = analyzer.getCurrentInputColumns(); // execute timezone cast or duration cast if needed for local table scan if (analyzer.appendExtraCastsAfterTS(chain, need_cast_column, table_scan)) { @@ -196,23 +194,11 @@ std::tuple addExtraCastsAfterT assert(extra_cast); chain.finalize(); chain.clear(); - - // After `analyzer.appendExtraCastsAfterTS`, analyzer.getCurrentInputColumns() has been modified. - // For remote read, `timezone cast and duration cast` had been pushed down, don't need to execute cast expressions. - // To keep the schema of local read streams and remote read streams the same, do project action for remote read streams. - NamesWithAliases project_for_remote_read; - const auto & after_cast_source_columns = analyzer.getCurrentInputColumns(); - for (size_t i = 0; i < after_cast_source_columns.size(); ++i) - project_for_remote_read.emplace_back(original_source_columns[i].name, after_cast_source_columns[i].name); - assert(!project_for_remote_read.empty()); - ExpressionActionsPtr project_for_cop_read = std::make_shared(original_source_columns); - project_for_cop_read->add(ExpressionAction::project(project_for_remote_read)); - - return {true, extra_cast, project_for_cop_read}; + return {true, extra_cast}; } else { - return {false, nullptr, nullptr}; + return {false, nullptr}; } } @@ -412,7 +398,7 @@ void DAGStorageInterpreter::executeCastAfterTableScan( DAGPipeline & pipeline) { // execute timezone cast or duration cast if needed for local table scan - auto [has_cast, extra_cast, project_for_cop_read] = addExtraCastsAfterTs(*analyzer, is_need_add_cast_column, table_scan); + auto [has_cast, extra_cast] = addExtraCastsAfterTs(*analyzer, is_need_add_cast_column, table_scan); if (has_cast) { assert(remote_read_streams_start_index <= pipeline.streams.size()); @@ -424,13 +410,6 @@ void DAGStorageInterpreter::executeCastAfterTableScan( stream = std::make_shared(stream, extra_cast, log->identifier()); stream->setExtraInfo("cast after local tableScan"); } - // remote streams - while (i < pipeline.streams.size()) - { - auto & stream = pipeline.streams[i++]; - stream = std::make_shared(stream, project_for_cop_read, log->identifier()); - stream->setExtraInfo("cast after remote tableScan"); - } } } @@ -609,6 +588,7 @@ std::unordered_map DAGStorageInterpreter::generateSele query_info.query = dagContext().dummy_ast; query_info.dag_query = std::make_unique( filter_conditions.conditions, + table_scan.getPushedDownFilters(), analyzer->getPreparedSets(), analyzer->getCurrentInputColumns(), context.getTimezoneInfo()); @@ -1003,7 +983,9 @@ std::unordered_map DAG std::tuple> DAGStorageInterpreter::getColumnsForTableScan() { Names required_columns_tmp; + required_columns_tmp.reserve(table_scan.getColumnSize()); NamesAndTypes source_columns_tmp; + source_columns_tmp.reserve(table_scan.getColumnSize()); std::vector need_cast_column; need_cast_column.reserve(table_scan.getColumnSize()); String handle_column_name = MutableSupport::tidb_pk_column_name; @@ -1043,12 +1025,28 @@ std::tuple> DAGStorageIn source_columns_tmp.emplace_back(std::move(pair)); } required_columns_tmp.emplace_back(std::move(name)); - if (cid != -1 && ci.tp == TiDB::TypeTimestamp) - need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); - else if (cid != -1 && ci.tp == TiDB::TypeTime) - need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); - else + } + + std::unordered_set col_name_set; + for (const auto & expr : table_scan.getPushedDownFilters()) + { + getColumnNamesFromExpr(expr, source_columns_tmp, col_name_set); + } + for (const auto & col : table_scan.getColumns()) + { + if (col_name_set.contains(col.name)) + { need_cast_column.push_back(ExtraCastAfterTSMode::None); + } + else + { + if (col.id != -1 && col.tp == TiDB::TypeTimestamp) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); + else if (col.id != -1 && col.tp == TiDB::TypeTime) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); + else + need_cast_column.push_back(ExtraCastAfterTSMode::None); + } } return {required_columns_tmp, source_columns_tmp, need_cast_column}; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 83563c47338..65daaa6a825 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1141,6 +1141,24 @@ String getColumnNameForColumnExpr(const tipb::Expr & expr, const std::vector & input_col, std::unordered_set & col_name_set) +{ + if (expr.children_size() == 0) + { + if (isColumnExpr(expr)) + { + col_name_set.insert(getColumnNameForColumnExpr(expr, input_col)); + } + } + else + { + for (const auto & child : expr.children()) + { + getColumnNamesFromExpr(child, input_col, col_name_set); + } + } +} + NameAndTypePair getColumnNameAndTypeForColumnExpr(const tipb::Expr & expr, const std::vector & input_col) { auto column_index = decodeDAGInt64(expr.val()); diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 1fe665e9c89..4241637c279 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -46,6 +46,7 @@ String getFieldTypeName(Int32 tp); String getJoinExecTypeName(const tipb::JoinExecType & tp); bool isColumnExpr(const tipb::Expr & expr); String getColumnNameForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); +void getColumnNamesFromExpr(const tipb::Expr & expr, const std::vector & input_col, std::unordered_set & col_name_set); NameAndTypePair getColumnNameAndTypeForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); const String & getTypeName(const tipb::Expr & expr); String exprToString(const tipb::Expr & expr, const std::vector & input_col); diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index a6d85e36ed4..1b87225cc76 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -176,14 +176,14 @@ void executeCreatingSets( } std::tuple buildPushDownFilter( - const FilterConditions & filter_conditions, + const google::protobuf::RepeatedPtrField & conditions, DAGExpressionAnalyzer & analyzer) { - assert(filter_conditions.hasValue()); + assert(!conditions.empty()); ExpressionActionsChain chain; analyzer.initChain(chain); - String filter_column_name = analyzer.appendWhere(chain, filter_conditions.conditions); + String filter_column_name = analyzer.appendWhere(chain, conditions); ExpressionActionsPtr before_where = chain.getLastActions(); chain.addStep(); @@ -207,7 +207,7 @@ void executePushedDownFilter( LoggerPtr log, DAGPipeline & pipeline) { - auto [before_where, filter_column_name, project_after_where] = ::DB::buildPushDownFilter(filter_conditions, analyzer); + auto [before_where, filter_column_name, project_after_where] = ::DB::buildPushDownFilter(filter_conditions.conditions, analyzer); assert(remote_read_streams_start_index <= pipeline.streams.size()); // for remote read, filter had been pushed down, don't need to execute again. @@ -238,4 +238,5 @@ void executeGeneratedColumnPlaceholder( stream->setExtraInfo("generated column placeholder above table scan"); } } + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 3d2ebd33afc..dafc4ad02c2 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -69,7 +69,7 @@ void executeCreatingSets( const LoggerPtr & log); std::tuple buildPushDownFilter( - const FilterConditions & filter_conditions, + const google::protobuf::RepeatedPtrField & conditions, DAGExpressionAnalyzer & analyzer); void executePushedDownFilter( @@ -84,4 +84,5 @@ void executeGeneratedColumnPlaceholder( const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline); + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index 5b78ec72dfa..68ef961c471 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -25,6 +25,7 @@ TiDBTableScan::TiDBTableScan( , executor_id(executor_id_) , is_partition_table_scan(table_scan->tp() == tipb::TypePartitionTableScan) , columns(is_partition_table_scan ? std::move(TiDB::toTiDBColumnInfos(table_scan->partition_table_scan().columns())) : std::move(TiDB::toTiDBColumnInfos(table_scan->tbl_scan().columns()))) + , pushed_down_filters(is_partition_table_scan ? std::move(table_scan->partition_table_scan().pushed_down_filter_conditions()) : std::move(table_scan->tbl_scan().pushed_down_filter_conditions())) // Only No-partition table need keep order when tablescan executor required keep order. // If keep_order is not set, keep order for safety. , keep_order(!is_partition_table_scan && (table_scan->tbl_scan().keep_order() || !table_scan->tbl_scan().has_keep_order())) diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.h b/dbms/src/Flash/Coprocessor/TiDBTableScan.h index 6903c4ca5bb..9ea5e16ec26 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.h +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -71,6 +71,11 @@ class TiDBTableScan return table_scan; } + const google::protobuf::RepeatedPtrField & getPushedDownFilters() const + { + return pushed_down_filters; + } + private: const tipb::Executor * table_scan; String executor_id; @@ -85,6 +90,12 @@ class TiDBTableScan /// physical_table_ids contains the table ids of its partitions std::vector physical_table_ids; Int64 logical_table_id; + + /// pushed_down_filter_conditions is the filter conditions that are + /// pushed down to table scan by late materialization. + /// They will be executed on Storage layer. + const google::protobuf::RepeatedPtrField pushed_down_filters; + bool keep_order; bool is_fast_scan; }; diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp index eb76380fd6c..e578310ec72 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp @@ -95,6 +95,16 @@ bool BitmapFilter::get(IColumn::Filter & f, UInt32 start, UInt32 limit) const } } +void BitmapFilter::rangeAnd(IColumn::Filter & f, UInt32 start, UInt32 limit) const +{ + RUNTIME_CHECK(start + limit <= filter.size() && f.size() == limit); + auto begin = filter.cbegin() + start; + if (!all_match) + { + std::transform(f.begin(), f.end(), begin, f.begin(), [](const UInt8 a, const bool b) { return a != 0 && b; }); + } +} + void BitmapFilter::runOptimize() { all_match = std::find(filter.begin(), filter.end(), false) == filter.end(); diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h index 540759ade4a..56b12fb2f18 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h @@ -31,6 +31,8 @@ class BitmapFilter void set(UInt32 start, UInt32 limit); // If return true, all data is match and do not fill the filter. bool get(IColumn::Filter & f, UInt32 start, UInt32 limit) const; + // filter[start, limit] & f -> f + void rangeAnd(IColumn::Filter & f, UInt32 start, UInt32 limit) const; void runOptimize(); diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.cpp index 74b8803f012..8beaa2452fe 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.cpp @@ -16,13 +16,14 @@ #include #include #include +#include namespace DB::DM { BitmapFilterBlockInputStream::BitmapFilterBlockInputStream( const ColumnDefines & columns_to_read, - BlockInputStreamPtr stable_, - BlockInputStreamPtr delta_, + SkippableBlockInputStreamPtr stable_, + SkippableBlockInputStreamPtr delta_, size_t stable_rows_, const BitmapFilterPtr & bitmap_filter_, const String & req_id_) @@ -36,7 +37,7 @@ BitmapFilterBlockInputStream::BitmapFilterBlockInputStream( Block BitmapFilterBlockInputStream::readImpl(FilterPtr & res_filter, bool return_filter) { - auto [block, from_delta] = readBlock(); + auto [block, from_delta] = readBlock(stable, delta); if (block) { if (from_delta) @@ -69,33 +70,4 @@ Block BitmapFilterBlockInputStream::readImpl(FilterPtr & res_filter, bool return return block; } -// -std::pair BitmapFilterBlockInputStream::readBlock() -{ - if (stable == nullptr && delta == nullptr) - { - return {{}, false}; - } - - if (stable == nullptr) - { - return {delta->read(), true}; - } - - auto block = stable->read(); - if (block) - { - return {block, false}; - } - else - { - stable = nullptr; - if (delta != nullptr) - { - block = delta->read(); - } - return {block, true}; - } -} - } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.h b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.h index fc8125e70b1..913f9072c48 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.h +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilterBlockInputStream.h @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB::DM { @@ -28,8 +29,8 @@ class BitmapFilterBlockInputStream : public IProfilingBlockInputStream public: BitmapFilterBlockInputStream( const ColumnDefines & columns_to_read, - BlockInputStreamPtr stable_, - BlockInputStreamPtr delta_, + SkippableBlockInputStreamPtr stable_, + SkippableBlockInputStreamPtr delta_, size_t stable_rows_, const BitmapFilterPtr & bitmap_filter_, const String & req_id_); @@ -50,11 +51,9 @@ class BitmapFilterBlockInputStream : public IProfilingBlockInputStream Block readImpl(FilterPtr & res_filter, bool return_filter) override; private: - std::pair readBlock(); - Block header; - BlockInputStreamPtr stable; - BlockInputStreamPtr delta; + SkippableBlockInputStreamPtr stable; + SkippableBlockInputStreamPtr delta; size_t stable_rows; BitmapFilterPtr bitmap_filter; const LoggerPtr log; diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp index 83a48b20a14..0380528f239 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp @@ -45,7 +45,7 @@ void ColumnFileBig::calculateStat(const DMContext & context) index_cache, /*set_cache_if_miss*/ false, {segment_range}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, {}, context.db_context.getFileProvider(), context.getReadLimiter(), diff --git a/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h b/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h index d8ccaa52930..1a9cdc27790 100644 --- a/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h +++ b/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h @@ -45,7 +45,7 @@ class DMSegmentThreadInputStream : public IProfilingBlockInputStream const SegmentReadTaskPoolPtr & task_pool_, AfterSegmentRead after_segment_read_, const ColumnDefines & columns_to_read_, - const RSOperatorPtr & filter_, + const PushDownFilterPtr & filter_, UInt64 max_version_, size_t expected_block_size_, ReadMode read_mode_, @@ -139,7 +139,7 @@ class DMSegmentThreadInputStream : public IProfilingBlockInputStream SegmentReadTaskPoolPtr task_pool; AfterSegmentRead after_segment_read; ColumnDefines columns_to_read; - RSOperatorPtr filter; + PushDownFilterPtr filter; Block header; const UInt64 max_version; const size_t expected_block_size; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index c2c47217907..a4c6b7f48f8 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -990,7 +991,7 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, const RowKeyRanges & sorted_ranges, size_t num_streams, UInt64 max_version, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, const String & tracing_id, bool keep_order, bool is_fast_scan, @@ -1023,6 +1024,8 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, GET_METRIC(tiflash_storage_read_tasks_count).Increment(tasks.size()); size_t final_num_stream = std::max(1, std::min(num_streams, tasks.size())); + auto read_mode = getReadMode(db_context, is_fast_scan, keep_order); + RUNTIME_CHECK_MSG(!filter || !filter->before_where || read_mode == ReadMode::Bitmap, "Push down filters needs bitmap"); auto read_task_pool = std::make_shared( physical_table_id, dm_context, @@ -1030,7 +1033,7 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, filter, max_version, expected_block_size, - getReadMode(db_context, is_fast_scan, keep_order), + read_mode, std::move(tasks), after_segment_read, log_tracing_id, @@ -1080,7 +1083,7 @@ SourceOps DeltaMergeStore::readSourceOps( const RowKeyRanges & sorted_ranges, size_t num_streams, UInt64 max_version, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, const String & tracing_id, bool keep_order, bool is_fast_scan, diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index b70930c5779..28b71c2024c 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -344,7 +345,7 @@ class DeltaMergeStore : private boost::noncopyable const RowKeyRanges & sorted_ranges, size_t num_streams, UInt64 max_version, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, const String & tracing_id, bool keep_order, bool is_fast_scan = false, @@ -365,7 +366,7 @@ class DeltaMergeStore : private boost::noncopyable const RowKeyRanges & sorted_ranges, size_t num_streams, UInt64 max_version, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, const String & tracing_id, bool keep_order, bool is_fast_scan = false, diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index 7865aac4bfc..64bc0d16155 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp @@ -31,6 +31,7 @@ #include #include + namespace CurrentMetrics { extern const Metric OpenFileForRead; @@ -305,20 +306,42 @@ Block DMFileReader::readWithFilter(const IColumn::Filter & filter) auto & use_packs = pack_filter.getUsePacks(); size_t start_row_offset = next_row_offset; - - size_t read_rows = 0; size_t next_pack_id_cp = next_pack_id; - while (next_pack_id_cp < use_packs.size() && read_rows + pack_stats[next_pack_id_cp].rows <= filter.size()) + { - const auto begin = filter.cbegin() + read_rows; - const auto end = filter.cbegin() + read_rows + pack_stats[next_pack_id_cp].rows; - use_packs[next_pack_id_cp] = use_packs[next_pack_id_cp] && std::find(begin, end, 1) != end; - read_rows += pack_stats[next_pack_id_cp].rows; - ++next_pack_id_cp; + // Use std::find to find the first 1 in the filter, these rows before the first 1 should be skipped. + // For example, filter is [0, 0, 0, 1, 1, 1, 1, 0, 0, 0, 0, 0], each pack has 3 rows + // 1. begin points to index 0, it points to index 3, skip = 3 >= 3, so use_packs[0] = false, read_rows = 3, next_pack_id_cp = 1 + // 2. it != filter.cend(), so use_packs[1] = true, read_rows = 6, next_pack_id_cp = 2 + // 3. begin points to index 6, and it points to index 6, skip = 0 < 3 + // 4. it != filter.cend(), so use_packs[2] = true, read_rows = 9, next_pack_id_cp = 3 + // 5. begin points to index 9, and it points to index 12, skip = 3 >= 3, so use_packs[3] = false, read_rows = 12, next_pack_id_cp = 4 + // 6. it == filter.cend(), break + // read_rows = filter.size() = 12, next_pack_id_cp = 4 + // This algorithm should be more efficient than check each pack one by one. + size_t read_rows = 0; + while (read_rows < filter.size()) + { + const auto begin = filter.cbegin() + read_rows; + const auto it = std::find(begin, filter.cend(), 1); + auto skip = std::distance(begin, it); + while (next_pack_id_cp < use_packs.size() && skip >= pack_stats[next_pack_id_cp].rows) + { + use_packs[next_pack_id_cp] = false; + skip -= pack_stats[next_pack_id_cp].rows; + read_rows += pack_stats[next_pack_id_cp].rows; + ++next_pack_id_cp; + } + if (it == filter.cend()) + break; + use_packs[next_pack_id_cp] = true; + read_rows += pack_stats[next_pack_id_cp].rows; + ++next_pack_id_cp; + } + // filter.size() equals to the number of rows in the next block + // so read_rows should be equal to filter.size() here. + RUNTIME_CHECK(read_rows == filter.size()); } - // filter.size() equals to the number of rows in the next block - // so read_rows should be equal to filter.size() here. - RUNTIME_CHECK(read_rows == filter.size()); // mark the next pack after next read as not used temporarily // to avoid reading it and its following packs in this round @@ -332,7 +355,7 @@ Block DMFileReader::readWithFilter(const IColumn::Filter & filter) Blocks blocks; blocks.reserve(next_pack_id_cp - next_pack_id); - read_rows = 0; + size_t read_rows = 0; for (size_t i = next_pack_id; i < next_pack_id_cp; ++i) { // When the next pack is not used or the pack is the last pack, call read() to read theses packs and filter them @@ -345,9 +368,7 @@ Block DMFileReader::readWithFilter(const IColumn::Filter & filter) { Block block = read(); - IColumn::Filter block_filter; - block_filter.resize(block.rows()); - std::copy(filter.cbegin() + read_rows, filter.cbegin() + read_rows + block.rows(), block_filter.begin()); + IColumn::Filter block_filter(filter.cbegin() + read_rows, filter.cbegin() + read_rows + block.rows()); read_rows += block.rows(); if (size_t passed_count = countBytesInFilter(block_filter); passed_count != block.rows()) diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h new file mode 100644 index 00000000000..0d7e21e10cd --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -0,0 +1,61 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +namespace DB::DM +{ + +class PushDownFilter; +using PushDownFilterPtr = std::shared_ptr; +inline static const PushDownFilterPtr EMPTY_FILTER{}; + +class PushDownFilter : public std::enable_shared_from_this +{ +public: + PushDownFilter(const RSOperatorPtr & rs_operator_, + const ExpressionActionsPtr & beofre_where_, + const ColumnDefines & filter_columns_, + const String filter_column_name_, + const ExpressionActionsPtr & extra_cast_) + : rs_operator(rs_operator_) + , before_where(beofre_where_) + , filter_column_name(std::move(filter_column_name_)) + , filter_columns(std::move(filter_columns_)) + , extra_cast(extra_cast_) + {} + + explicit PushDownFilter(const RSOperatorPtr & rs_operator_) + : rs_operator(rs_operator_) + , before_where(nullptr) + , filter_columns({}) + , extra_cast(nullptr) + {} + + // Rough set operator + RSOperatorPtr rs_operator; + // Filter expression actions and the name of the tmp filter column + // Used construct the FilterBlockInputStream + ExpressionActionsPtr before_where; + String filter_column_name; + // The columns needed by the filter expression + ColumnDefines filter_columns; + // The expression actions used to cast the timestamp/datetime column + ExpressionActionsPtr extra_cast; +}; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h index a2789b88c30..1cc79f51cc0 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h +++ b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h @@ -28,7 +28,7 @@ using RSOperatorPtr = std::shared_ptr; using RSOperators = std::vector; using Fields = std::vector; -inline static const RSOperatorPtr EMPTY_FILTER{}; +inline static const RSOperatorPtr EMPTY_RS_OPERATOR{}; struct RSCheckParam { diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index 88a7f1a9a80..47a8e4ef28e 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -251,7 +251,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, { assert(isFunctionExpr(expr)); - RSOperatorPtr op = EMPTY_FILTER; + RSOperatorPtr op = EMPTY_RS_OPERATOR; if (unlikely(isAggFunctionExpr(expr))) { op = createUnsupported(expr.ShortDebugString(), "agg function: " + tipb::ExprType_Name(expr.tp()), false); @@ -392,21 +392,29 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, FilterParser::AttrCreatorByColumnID && creator, const LoggerPtr & log) { - RSOperatorPtr op = EMPTY_FILTER; - if (dag_info.filters.empty()) + RSOperatorPtr op = EMPTY_RS_OPERATOR; + if (dag_info.filters.empty() && dag_info.pushed_down_filters.empty()) return op; - if (dag_info.filters.size() == 1) + if (dag_info.filters.size() == 1 && dag_info.pushed_down_filters.empty()) { op = cop::tryParse(dag_info.filters[0], columns_to_read, creator, dag_info.timezone_info, log); } + else if (dag_info.pushed_down_filters.size() == 1 && dag_info.filters.empty()) + { + op = cop::tryParse(dag_info.pushed_down_filters[0], columns_to_read, creator, dag_info.timezone_info, log); + } else { /// By default, multiple conditions with operator "and" RSOperators children; - for (int i = 0; i < dag_info.filters.size(); ++i) + children.reserve(dag_info.filters.size() + dag_info.pushed_down_filters.size()); + for (const auto & filter : dag_info.filters) + { + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); + } + for (const auto & filter : dag_info.pushed_down_filters) { - const auto & filter = dag_info.filters[i]; children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); } op = createAnd(children); diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h index 36c53ca0a85..c6a12eaaa29 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include @@ -35,6 +36,7 @@ struct DAGQueryInfo; namespace DM { + class RSOperator; using RSOperatorPtr = std::shared_ptr; @@ -42,7 +44,7 @@ class FilterParser { public: /// From dag. - using AttrCreatorByColumnID = std::function; + using AttrCreatorByColumnID = std::function; static RSOperatorPtr parseDAGQuery( const DAGQueryInfo & dag_info, const ColumnDefines & columns_to_read, diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp index 7deea0f02c6..270b92ca15f 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp @@ -24,11 +24,13 @@ namespace DB::DM LateMaterializationBlockInputStream::LateMaterializationBlockInputStream( const ColumnDefines & columns_to_read, + const String & filter_column_name_, BlockInputStreamPtr filter_column_stream_, SkippableBlockInputStreamPtr rest_column_stream_, const BitmapFilterPtr & bitmap_filter_, const String & req_id_) : header(toEmptyBlock(columns_to_read)) + , filter_column_name(filter_column_name_) , filter_column_stream(filter_column_stream_) , rest_column_stream(rest_column_stream_) , bitmap_filter(bitmap_filter_) @@ -50,19 +52,17 @@ Block LateMaterializationBlockInputStream::readImpl() if (!filter_column_block) return filter_column_block; - RUNTIME_CHECK_MSG(filter, "Late materialization meets unexpected null filter"); - - // Get mvcc-filter - size_t rows = filter_column_block.rows(); - mvcc_filter.resize(rows); - bool all_match = bitmap_filter->get(mvcc_filter, filter_column_block.startOffset(), rows); - if (!all_match) + // If filter is nullptr, it means that these push down filters are always true. + if (!filter) { - // if mvcc-filter is all match, use filter directly - // else use `mvcc-filter & filter` to get the final filter - std::transform(mvcc_filter.cbegin(), mvcc_filter.cend(), filter->cbegin(), filter->begin(), [](const UInt8 a, const UInt8 b) { return a != 0 && b != 0; }); + Block rest_column_block = rest_column_stream->read(); + return hstackBlocks({std::move(filter_column_block), std::move(rest_column_block)}, header); } + size_t rows = filter_column_block.rows(); + // bitmap_filter[start_offset, start_offset + rows] & filter -> filter + bitmap_filter->rangeAnd(*filter, filter_column_block.startOffset(), rows); + if (size_t passed_count = countBytesInFilter(*filter); passed_count == 0) { // if all rows are filtered, skip the next block of rest_column_stream @@ -94,6 +94,8 @@ Block LateMaterializationBlockInputStream::readImpl() rest_column_block = rest_column_stream->readWithFilter(*filter); for (auto & col : filter_column_block) { + if (col.name == filter_column_name) + continue; col.column = col.column->filter(*filter, passed_count); } } @@ -102,12 +104,14 @@ Block LateMaterializationBlockInputStream::readImpl() // if the number of rows left after filtering out is small, we can't skip any packs of the next block // so we call read() to get the next block, and then filter it. rest_column_block = rest_column_stream->read(); - for (auto & col : filter_column_block) + for (auto & col : rest_column_block) { col.column = col.column->filter(*filter, passed_count); } - for (auto & col : rest_column_block) + for (auto & col : filter_column_block) { + if (col.name == filter_column_name) + continue; col.column = col.column->filter(*filter, passed_count); } } @@ -118,15 +122,15 @@ Block LateMaterializationBlockInputStream::readImpl() } // make sure the position and size of filter_column_block and rest_column_block are the same - RUNTIME_CHECK_MSG(rest_column_block.rows() == filter_column_block.rows() && rest_column_block.startOffset() == filter_column_block.startOffset(), - "Late materialization meets unexpected size of block unmatched, filter_column_block: [start_offset={}, rows={}], rest_column_block: [start_offset={}, rows={}], pass_count={}", + RUNTIME_CHECK_MSG(rest_column_block.startOffset() == filter_column_block.startOffset(), + "Late materialization meets unexpected block unmatched, filter_column_block: [start_offset={}, rows={}], rest_column_block: [start_offset={}, rows={}], pass_count={}", filter_column_block.startOffset(), filter_column_block.rows(), rest_column_block.startOffset(), rest_column_block.rows(), passed_count); - - // TODO: remove tmp filter column in filter_column_block + // join filter_column_block and rest_column_block by columns, + // the tmp column added by FilterBlockInputStream will be removed. return hstackBlocks({std::move(filter_column_block), std::move(rest_column_block)}, header); } } diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h index 3897e0447c2..8a884ca63a1 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h @@ -35,6 +35,7 @@ class LateMaterializationBlockInputStream : public IProfilingBlockInputStream public: LateMaterializationBlockInputStream( const ColumnDefines & columns_to_read, + const String & filter_column_name_, BlockInputStreamPtr filter_column_stream_, SkippableBlockInputStreamPtr rest_column_stream_, const BitmapFilterPtr & bitmap_filter_, @@ -49,7 +50,9 @@ class LateMaterializationBlockInputStream : public IProfilingBlockInputStream private: Block header; - + // The name of the tmp filter column in filter_column_block which is added by the FilterBlockInputStream. + // The column is used to filter the block, but it is not included in the returned block. + const String & filter_column_name; // The stream used to read the filter column, and filter the block. BlockInputStreamPtr filter_column_stream; // The stream used to read the rest columns. @@ -58,7 +61,6 @@ class LateMaterializationBlockInputStream : public IProfilingBlockInputStream BitmapFilterPtr bitmap_filter; const LoggerPtr log; - IColumn::Filter mvcc_filter{}; }; } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 96c72404531..cf0aee022d5 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -17,6 +17,8 @@ #include #include #include +#include +#include #include #include #include @@ -34,9 +36,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -684,16 +688,16 @@ BlockInputStreamPtr Segment::getInputStream(const ReadMode & read_mode, const ColumnDefines & columns_to_read, const SegmentSnapshotPtr & segment_snap, const RowKeyRanges & read_ranges, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, UInt64 max_version, size_t expected_block_size) { switch (read_mode) { case ReadMode::Normal: - return getInputStreamModeNormal(dm_context, columns_to_read, segment_snap, read_ranges, filter, max_version, expected_block_size); + return getInputStreamModeNormal(dm_context, columns_to_read, segment_snap, read_ranges, filter ? filter->rs_operator : EMPTY_RS_OPERATOR, max_version, expected_block_size); case ReadMode::Fast: - return getInputStreamModeFast(dm_context, columns_to_read, segment_snap, read_ranges, filter, expected_block_size); + return getInputStreamModeFast(dm_context, columns_to_read, segment_snap, read_ranges, filter ? filter->rs_operator : EMPTY_RS_OPERATOR, expected_block_size); case ReadMode::Raw: return getInputStreamModeRaw(dm_context, columns_to_read, segment_snap, read_ranges, expected_block_size); case ReadMode::Bitmap: @@ -822,7 +826,7 @@ BlockInputStreamPtr Segment::getInputStreamForDataExport(const DMContext & dm_co BlockInputStreamPtr data_stream = getPlacedStream(dm_context, *read_info.read_columns, data_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, segment_snap->stable, read_info.getDeltaReader(), read_info.index_begin, @@ -957,7 +961,7 @@ BlockInputStreamPtr Segment::getInputStreamModeRaw(const DMContext & dm_context, dm_context, *new_columns_to_read, data_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, std::numeric_limits::max(), expected_block_size, /* enable_handle_clean_read */ false); @@ -1374,7 +1378,7 @@ std::optional Segment::getSplitPointSlow( BlockInputStreamPtr stream = getPlacedStream(dm_context, *pk_col_defs, rowkey_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, segment_snap->stable, delta_reader, read_info.index_begin, @@ -1399,7 +1403,7 @@ std::optional Segment::getSplitPointSlow( BlockInputStreamPtr stream = getPlacedStream(dm_context, *pk_col_defs, rowkey_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, segment_snap->stable, delta_reader, read_info.index_begin, @@ -1663,7 +1667,7 @@ std::optional Segment::prepareSplitPhysical( // BlockInputStreamPtr my_data = getPlacedStream(dm_context, *read_info.read_columns, my_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, segment_snap->stable, my_delta_reader, read_info.index_begin, @@ -1692,7 +1696,7 @@ std::optional Segment::prepareSplitPhysical( // BlockInputStreamPtr other_data = getPlacedStream(dm_context, *read_info.read_columns, other_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, segment_snap->stable, other_delta_reader, read_info.index_begin, @@ -1872,7 +1876,7 @@ StableValueSpacePtr Segment::prepareMerge(DMContext & dm_context, // BlockInputStreamPtr stream = getPlacedStream(dm_context, *read_info.read_columns, rowkey_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, segment_snap->stable, read_info.getDeltaReader(), read_info.index_begin, @@ -2023,7 +2027,7 @@ bool Segment::compactDelta(DMContext & dm_context) return delta->compact(dm_context); } -void Segment::placeDeltaIndex(DMContext & dm_context) +void Segment::placeDeltaIndex(DMContext & dm_context) const { // Update delta-index with persisted packs. TODO: can use a read snapshot here? auto segment_snap = createSnapshot(dm_context, /*for_update=*/true, CurrentMetrics::DT_SnapshotOfPlaceIndex); @@ -2032,7 +2036,7 @@ void Segment::placeDeltaIndex(DMContext & dm_context) placeDeltaIndex(dm_context, segment_snap); } -void Segment::placeDeltaIndex(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap) +void Segment::placeDeltaIndex(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap) const { getReadInfo(dm_context, /*read_columns=*/{getExtraHandleColumnDefine(is_common_handle)}, @@ -2362,7 +2366,7 @@ bool Segment::placeUpsert(const DMContext & dm_context, dm_context, {handle, getVersionColumnDefine()}, {place_handle_range}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, stable_snap, delta_reader, compacted_index->begin(), @@ -2413,7 +2417,7 @@ bool Segment::placeDelete(const DMContext & dm_context, dm_context, {handle, getVersionColumnDefine()}, delete_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, stable_snap, delta_reader, compacted_index->begin(), @@ -2448,7 +2452,7 @@ bool Segment::placeDelete(const DMContext & dm_context, dm_context, {handle, getVersionColumnDefine()}, {place_handle_range}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, stable_snap, delta_reader, compacted_index->begin(), @@ -2682,10 +2686,12 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(BitmapFilterPtr && bitma UInt64 max_version, size_t expected_block_size) { + // set `is_fast_scan` to true to try to enable clean read auto enable_handle_clean_read = !hasColumn(columns_to_read, EXTRA_HANDLE_COLUMN_ID); constexpr auto is_fast_scan = true; auto enable_del_clean_read = !hasColumn(columns_to_read, TAG_COLUMN_ID); - BlockInputStreamPtr stable_stream = segment_snap->stable->getInputStream( + + SkippableBlockInputStreamPtr stable_stream = segment_snap->stable->getInputStream( dm_context, columns_to_read, read_ranges, @@ -2697,7 +2703,7 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(BitmapFilterPtr && bitma enable_del_clean_read); auto columns_to_read_ptr = std::make_shared(columns_to_read); - BlockInputStreamPtr delta_stream = std::make_shared( + SkippableBlockInputStreamPtr delta_stream = std::make_shared( dm_context, segment_snap->delta, columns_to_read_ptr, @@ -2712,6 +2718,101 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(BitmapFilterPtr && bitma dm_context.tracing_id); } +BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bitmap_filter, + const DMContext & dm_context, + const ColumnDefines & columns_to_read, + const SegmentSnapshotPtr & segment_snap, + const RowKeyRanges & data_ranges, + const PushDownFilterPtr & filter, + UInt64 max_version, + size_t expected_block_size) +{ + // set `is_fast_scan` to true to try to enable clean read + auto enable_handle_clean_read = !hasColumn(columns_to_read, EXTRA_HANDLE_COLUMN_ID); + constexpr auto is_fast_scan = true; + auto enable_del_clean_read = !hasColumn(columns_to_read, TAG_COLUMN_ID); + + // construct filter column stream + const auto & filter_columns = filter->filter_columns; + SkippableBlockInputStreamPtr filter_column_stable_stream = segment_snap->stable->getInputStream( + dm_context, + filter_columns, + data_ranges, + filter->rs_operator, + max_version, + expected_block_size, + enable_handle_clean_read, + is_fast_scan, + enable_del_clean_read); + + auto filter_columns_to_read_ptr = std::make_shared(filter_columns); + SkippableBlockInputStreamPtr filter_column_delta_stream = std::make_shared( + dm_context, + segment_snap->delta, + filter_columns_to_read_ptr, + this->rowkey_range); + + BlockInputStreamPtr filter_column_stream = std::make_shared( + filter_columns, + filter_column_stable_stream, + filter_column_delta_stream, + segment_snap->stable->getDMFilesRows(), + dm_context.tracing_id); + + // construct extra cast stream if needed + if (filter->extra_cast) + { + filter_column_stream = std::make_shared(filter_column_stream, filter->extra_cast, dm_context.tracing_id); + filter_column_stream->setExtraInfo("cast after tableScan"); + } + + // construct filter stream + filter_column_stream = std::make_shared(filter_column_stream, filter->before_where, filter->filter_column_name, dm_context.tracing_id); + filter_column_stream->setExtraInfo("push down filter"); + if (filter_columns.size() == columns_to_read.size()) + { + LOG_ERROR(log, "Late materialization filter columns size equal to read columns size, which is not expected."); + // no need to read columns again + return filter_column_stream; + } + + ColumnDefines rest_columns_to_read{columns_to_read}; + // remove columns of pushed down filter + for (const auto & col : filter_columns) + { + rest_columns_to_read.erase(std::remove_if(rest_columns_to_read.begin(), rest_columns_to_read.end(), [&](const ColumnDefine & c) { return c.id == col.id; }), rest_columns_to_read.end()); + } + + // construct rest column stream + SkippableBlockInputStreamPtr rest_column_stable_stream = segment_snap->stable->getInputStream( + dm_context, + rest_columns_to_read, + data_ranges, + filter->rs_operator, + max_version, + expected_block_size, + enable_handle_clean_read, + is_fast_scan, + enable_del_clean_read); + + auto rest_columns_to_read_ptr = std::make_shared(rest_columns_to_read); + SkippableBlockInputStreamPtr rest_column_delta_stream = std::make_shared( + dm_context, + segment_snap->delta, + rest_columns_to_read_ptr, + this->rowkey_range); + + SkippableBlockInputStreamPtr rest_column_stream = std::make_shared( + rest_columns_to_read, + rest_column_stable_stream, + rest_column_delta_stream, + segment_snap->stable->getDMFilesRows(), + dm_context.tracing_id); + + // construct late materialization stream + return std::make_shared(columns_to_read, filter->filter_column_name, filter_column_stream, rest_column_stream, bitmap_filter, dm_context.tracing_id); +} + RowKeyRanges Segment::shrinkRowKeyRanges(const RowKeyRanges & read_ranges) { RowKeyRanges real_ranges; @@ -2728,7 +2829,7 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(const DMContext & dm_con const ColumnDefines & columns_to_read, const SegmentSnapshotPtr & segment_snap, const RowKeyRanges & read_ranges, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, UInt64 max_version, size_t expected_block_size) { @@ -2741,17 +2842,31 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(const DMContext & dm_con dm_context, segment_snap, real_ranges, - filter, + filter ? filter->rs_operator : EMPTY_RS_OPERATOR, max_version, expected_block_size); + if (filter && filter->before_where) + { + // if has filter conditions pushed down, use late materialization + return getLateMaterializationStream( + std::move(bitmap_filter), + dm_context, + columns_to_read, + segment_snap, + real_ranges, + filter, + max_version, + expected_block_size); + } + return getBitmapFilterInputStream( std::move(bitmap_filter), segment_snap, dm_context, columns_to_read, real_ranges, - filter, + filter ? filter->rs_operator : EMPTY_RS_OPERATOR, max_version, expected_block_size); } diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index c420368d1bb..b44facd69c8 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -196,7 +196,7 @@ class Segment const ColumnDefines & columns_to_read, const SegmentSnapshotPtr & segment_snap, const RowKeyRanges & read_ranges, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, UInt64 max_version, size_t expected_block_size); @@ -491,8 +491,8 @@ class Segment /// Flush delta's cache packs. bool flushCache(DMContext & dm_context); - void placeDeltaIndex(DMContext & dm_context); - void placeDeltaIndex(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap); + void placeDeltaIndex(DMContext & dm_context) const; + void placeDeltaIndex(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap) const; /// Compact the delta layer, merging fragment column files into bigger column files. /// It does not merge the delta into stable layer. @@ -665,10 +665,19 @@ class Segment const ColumnDefines & columns_to_read, const SegmentSnapshotPtr & segment_snap, const RowKeyRanges & read_ranges, - const RSOperatorPtr & filter, + const PushDownFilterPtr & filter, UInt64 max_version, size_t expected_block_size); + BlockInputStreamPtr getLateMaterializationStream(BitmapFilterPtr && bitmap_filter, + const DMContext & dm_context, + const ColumnDefines & columns_to_read, + const SegmentSnapshotPtr & segment_snap, + const RowKeyRanges & data_ranges, + const PushDownFilterPtr & filter, + UInt64 max_version, + size_t expected_block_size); + private: /// The version of this segment. After split / merge / mergeDelta / replaceData, epoch got increased by 1. diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h index 956927e34c2..2bdc816cad0 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -15,7 +15,7 @@ #pragma once #include #include -#include +#include #include #include @@ -163,7 +163,7 @@ class SegmentReadTaskPool : private boost::noncopyable int64_t table_id_, const DMContextPtr & dm_context_, const ColumnDefines & columns_to_read_, - const RSOperatorPtr & filter_, + const PushDownFilterPtr & filter_, uint64_t max_version_, size_t expected_block_size_, ReadMode read_mode_, @@ -261,7 +261,7 @@ class SegmentReadTaskPool : private boost::noncopyable const int64_t table_id; DMContextPtr dm_context; ColumnDefines columns_to_read; - RSOperatorPtr filter; + PushDownFilterPtr filter; const uint64_t max_version; const size_t expected_block_size; const ReadMode read_mode; diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index b3464e2621b..2159251802d 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -60,7 +60,7 @@ void StableValueSpace::setFiles(const DMFiles & files_, const RowKeyRange & rang index_cache, /*set_cache_if_miss*/ true, {range}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, {}, dm_context->db_context.getFileProvider(), dm_context->getReadLimiter(), @@ -340,7 +340,7 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const context.db_context.getGlobalContext().getMinMaxIndexCache(), /*set_cache_if_miss*/ false, {rowkey_range}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, {}, context.db_context.getFileProvider(), context.getReadLimiter(), diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index d6fe72f832a..b42757a2781 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -554,7 +554,7 @@ try {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, /* num_streams= */ 1, /* max_version= */ std::numeric_limits::max(), - filter, + std::make_shared(filter), TRACING_NAME, /* keep_order= */ false, /* is_fast_scan= */ false, diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp index 322a69451b0..eb8d7af92cd 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -134,7 +134,7 @@ bool checkMatch( store->mergeDeltaAll(context); const ColumnDefine & col_to_read = check_pk ? getExtraHandleColumnDefine(is_common_handle) : cd; - auto streams = store->read(context, context.getSettingsRef(), {col_to_read}, {all_range}, 1, std::numeric_limits::max(), filter, name, false); + auto streams = store->read(context, context.getSettingsRef(), {col_to_read}, {all_range}, 1, std::numeric_limits::max(), std::make_shared(filter), name, false); auto rows = getInputStreamNRows(streams[0]); store->drop(); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp index 6df17338e1b..7677e9511a2 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp @@ -12,157 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include #include -#include -#include #include #include #include +#include #include #include #include -#include - -#include using namespace std::chrono_literals; using namespace DB::tests; namespace DB::DM::tests { -template -::testing::AssertionResult sequenceEqual(const E * expected, const A * actual, size_t size) -{ - for (size_t i = 0; i < size; i++) - { - if (expected[i] != actual[i]) - { - return ::testing::AssertionFailure() - << fmt::format("Value at index {} mismatch: expected {} vs actual {}. expected => {} actual => {}", - i, - expected[i], - actual[i], - std::vector(expected, expected + size), - std::vector(actual, actual + size)); - } - } - return ::testing::AssertionSuccess(); -} - -template -std::vector genSequence(T begin, T end) -{ - auto size = end - begin; - std::vector v(size); - std::iota(v.begin(), v.end(), begin); - return v; -} - -template -std::vector genSequence(const std::vector> & ranges) -{ - std::vector res; - for (auto [begin, end] : ranges) - { - auto v = genSequence(begin, end); - res.insert(res.end(), v.begin(), v.end()); - } - return res; -} - -// "[a, b)" => std::pair{a, b} -template -std::pair parseRange(String & str_range) -{ - boost::algorithm::trim(str_range); - RUNTIME_CHECK(str_range.front() == '[' && str_range.back() == ')', str_range); - std::vector values; - str_range = str_range.substr(1, str_range.size() - 2); - boost::split(values, str_range, boost::is_any_of(",")); - RUNTIME_CHECK(values.size() == 2, str_range); - return {static_cast(std::stol(values[0])), static_cast(std::stol(values[1]))}; -} - -// "[a, b)|[c, d)" => [std::pair{a, b}, std::pair{c, d}] -template -std::vector> parseRanges(std::string_view str_ranges) -{ - std::vector ranges; - boost::split(ranges, str_ranges, boost::is_any_of("|")); - RUNTIME_CHECK(!ranges.empty(), str_ranges); - std::vector> vector_ranges; - vector_ranges.reserve(ranges.size()); - for (auto & r : ranges) - { - vector_ranges.emplace_back(parseRange(r)); - } - return vector_ranges; -} - -template -std::vector genSequence(std::string_view str_ranges) -{ - auto vector_ranges = parseRanges(str_ranges); - return genSequence(vector_ranges); -} - -struct SegDataUnit -{ - String type; - std::pair range; -}; - -// "type:[a, b)" => SegDataUnit -SegDataUnit parseSegDataUnit(String & s) -{ - boost::algorithm::trim(s); - std::vector values; - boost::split(values, s, boost::is_any_of(":")); - RUNTIME_CHECK(values.size() == 2, s); - return SegDataUnit{boost::algorithm::trim_copy(values[0]), parseRange(values[1])}; -} - -void check(const std::vector & seg_data_units) -{ - RUNTIME_CHECK(!seg_data_units.empty()); - std::vector stable_units; - std::vector mem_units; - for (size_t i = 0; i < seg_data_units.size(); i++) - { - const auto & type = seg_data_units[i].type; - if (type == "s") - { - stable_units.emplace_back(i); - } - else if (type == "d_mem" || type == "d_mem_del") - { - mem_units.emplace_back(i); - } - auto [begin, end] = seg_data_units[i].range; - RUNTIME_CHECK(begin < end, begin, end); - } - RUNTIME_CHECK(stable_units.empty() || (stable_units.size() == 1 && stable_units[0] == 0)); - std::vector expected_mem_units(mem_units.size()); - std::iota(expected_mem_units.begin(), expected_mem_units.end(), seg_data_units.size() - mem_units.size()); - RUNTIME_CHECK(mem_units == expected_mem_units, expected_mem_units, mem_units); -} - -std::vector parseSegData(std::string_view seg_data) -{ - std::vector str_seg_data_units; - boost::split(str_seg_data_units, seg_data, boost::is_any_of("|")); - RUNTIME_CHECK(!str_seg_data_units.empty(), seg_data); - std::vector seg_data_units; - seg_data_units.reserve(str_seg_data_units.size()); - for (auto & s : str_seg_data_units) - { - seg_data_units.emplace_back(parseSegDataUnit(s)); - } - check(seg_data_units); - return seg_data_units; -} class SegmentBitmapFilterTest : public SegmentTestBasic { @@ -521,7 +384,7 @@ TEST_F(SegmentBitmapFilterTest, CleanStable) *dm_context, snap, {seg->getRowKeyRange()}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, std::numeric_limits::max(), DEFAULT_BLOCK_SIZE); ASSERT_NE(bitmap_filter, nullptr); @@ -543,7 +406,7 @@ TEST_F(SegmentBitmapFilterTest, NotCleanStable) *dm_context, snap, {seg->getRowKeyRange()}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, std::numeric_limits::max(), DEFAULT_BLOCK_SIZE); ASSERT_NE(bitmap_filter, nullptr); @@ -563,7 +426,7 @@ TEST_F(SegmentBitmapFilterTest, NotCleanStable) *dm_context, snap, {seg->getRowKeyRange()}, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, 1, DEFAULT_BLOCK_SIZE); ASSERT_NE(bitmap_filter, nullptr); @@ -591,7 +454,7 @@ TEST_F(SegmentBitmapFilterTest, StableRange) *dm_context, snap, {buildRowKeyRange(10000, 50000)}, // [10000, 50000) - EMPTY_FILTER, + EMPTY_RS_OPERATOR, std::numeric_limits::max(), DEFAULT_BLOCK_SIZE); ASSERT_NE(bitmap_filter, nullptr); @@ -642,4 +505,5 @@ try ASSERT_TRUE(sequenceEqual(expected_right_row_id.data(), right_r->data(), right_r->size())); } CATCH + } // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp index c7a5436f307..ae487ff43ed 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -765,6 +765,7 @@ std::pair SegmentTestBasic::getSegmentForRead(Pa RUNTIME_CHECK(snapshot != nullptr); return {segment, snapshot}; } + std::vector SegmentTestBasic::readSegment(PageIdU64 segment_id, bool need_row_id, const RowKeyRanges & ranges) { auto [segment, snapshot] = getSegmentForRead(segment_id); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp new file mode 100644 index 00000000000..6709ea1e48f --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp @@ -0,0 +1,154 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include + +namespace DB::DM::tests +{ + +// "[a, b)" => std::pair{a, b} +template +std::pair parseRange(String & str_range) +{ + boost::algorithm::trim(str_range); + RUNTIME_CHECK(str_range.front() == '[' && str_range.back() == ')', str_range); + std::vector values; + str_range = str_range.substr(1, str_range.size() - 2); + boost::split(values, str_range, boost::is_any_of(",")); + RUNTIME_CHECK(values.size() == 2, str_range); + return {static_cast(std::stol(values[0])), static_cast(std::stol(values[1]))}; +} + +// "[a, b)|[c, d)" => [std::pair{a, b}, std::pair{c, d}] +template +std::vector> parseRanges(std::string_view str_ranges) +{ + std::vector ranges; + boost::split(ranges, str_ranges, boost::is_any_of("|")); + RUNTIME_CHECK(!ranges.empty(), str_ranges); + std::vector> vector_ranges; + vector_ranges.reserve(ranges.size()); + for (auto & r : ranges) + { + vector_ranges.emplace_back(parseRange(r)); + } + return vector_ranges; +} + +// "type:[a, b)" => SegDataUnit +SegDataUnit parseSegDataUnit(String & s) +{ + boost::algorithm::trim(s); + std::vector values; + boost::split(values, s, boost::is_any_of(":")); + RUNTIME_CHECK(values.size() == 2, s); + return SegDataUnit{boost::algorithm::trim_copy(values[0]), parseRange(values[1])}; +} + +void check(const std::vector & seg_data_units) +{ + RUNTIME_CHECK(!seg_data_units.empty()); + std::vector stable_units; + std::vector mem_units; + for (size_t i = 0; i < seg_data_units.size(); i++) + { + const auto & type = seg_data_units[i].type; + if (type == "s") + { + stable_units.emplace_back(i); + } + else if (type == "d_mem" || type == "d_mem_del") + { + mem_units.emplace_back(i); + } + auto [begin, end] = seg_data_units[i].range; + RUNTIME_CHECK(begin < end, begin, end); + } + RUNTIME_CHECK(stable_units.empty() || (stable_units.size() == 1 && stable_units[0] == 0)); + std::vector expected_mem_units(mem_units.size()); + std::iota(expected_mem_units.begin(), expected_mem_units.end(), seg_data_units.size() - mem_units.size()); + RUNTIME_CHECK(mem_units == expected_mem_units, expected_mem_units, mem_units); +} + +std::vector parseSegData(std::string_view seg_data) +{ + std::vector str_seg_data_units; + boost::split(str_seg_data_units, seg_data, boost::is_any_of("|")); + RUNTIME_CHECK(!str_seg_data_units.empty(), seg_data); + std::vector seg_data_units; + seg_data_units.reserve(str_seg_data_units.size()); + for (auto & s : str_seg_data_units) + { + seg_data_units.emplace_back(parseSegDataUnit(s)); + } + check(seg_data_units); + return seg_data_units; +} + +template +std::vector genSequence(T begin, T end) +{ + auto size = end - begin; + std::vector v(size); + std::iota(v.begin(), v.end(), begin); + return v; +} + +template +std::vector genSequence(const std::vector> & ranges) +{ + std::vector res; + for (auto [begin, end] : ranges) + { + auto v = genSequence(begin, end); + res.insert(res.end(), v.begin(), v.end()); + } + return res; +} + +template +std::vector genSequence(std::string_view str_ranges) +{ + auto vector_ranges = parseRanges(str_ranges); + return genSequence(vector_ranges); +} + +template +::testing::AssertionResult sequenceEqual(const E * expected, const A * actual, size_t size) +{ + for (size_t i = 0; i < size; i++) + { + if (expected[i] != actual[i]) + { + return ::testing::AssertionFailure() + << fmt::format("Value at index {} mismatch: expected {} vs actual {}. expected => {} actual => {}", + i, + expected[i], + actual[i], + std::vector(expected, expected + size), + std::vector(actual, actual + size)); + } + } + return ::testing::AssertionSuccess(); +} + +template std::vector genSequence(std::string_view str_ranges); +template std::vector genSequence(std::string_view str_ranges); +template ::testing::AssertionResult sequenceEqual(const UInt32 * expected, const UInt32 * actual, size_t size); +template ::testing::AssertionResult sequenceEqual(const Int64 * expected, const Int64 * actual, size_t size); + +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h new file mode 100644 index 00000000000..ae013618dda --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h @@ -0,0 +1,56 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + + +namespace DB::DM::tests +{ + +// "[a, b)" => std::pair{a, b} +template +std::pair parseRange(String & str_range); + +// "[a, b)|[c, d)" => [std::pair{a, b}, std::pair{c, d}] +template +std::vector> parseRanges(std::string_view str_ranges); + +struct SegDataUnit +{ + String type; + std::pair range; +}; + +// "type:[a, b)" => SegDataUnit +SegDataUnit parseSegDataUnit(String & s); + +void check(const std::vector & seg_data_units); + +std::vector parseSegData(std::string_view seg_data); + +template +std::vector genSequence(T begin, T end); + +template +std::vector genSequence(const std::vector> & ranges); + +template +std::vector genSequence(std::string_view str_ranges); + +template +::testing::AssertionResult sequenceEqual(const E * expected, const A * actual, size_t size); + +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_skippable_block_input_stream.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_skippable_block_input_stream.cpp index 4b5ffc8ff26..1fccc519642 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_skippable_block_input_stream.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_skippable_block_input_stream.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -24,100 +25,9 @@ #include - namespace DB::DM::tests { -namespace -{ - -// "[a, b)" => std::pair{a, b} -template -std::pair parseRange(String & str_range) -{ - boost::algorithm::trim(str_range); - RUNTIME_CHECK(str_range.front() == '[' && str_range.back() == ')', str_range); - std::vector values; - str_range = str_range.substr(1, str_range.size() - 2); - boost::split(values, str_range, boost::is_any_of(",")); - RUNTIME_CHECK(values.size() == 2, str_range); - return {static_cast(std::stol(values[0])), static_cast(std::stol(values[1]))}; -} - -// "[a, b)|[c, d)" => [std::pair{a, b}, std::pair{c, d}] -template -std::vector> parseRanges(std::string_view str_ranges) -{ - std::vector ranges; - boost::split(ranges, str_ranges, boost::is_any_of("|")); - RUNTIME_CHECK(!ranges.empty(), str_ranges); - std::vector> vector_ranges; - vector_ranges.reserve(ranges.size()); - for (auto & r : ranges) - { - vector_ranges.emplace_back(parseRange(r)); - } - return vector_ranges; -} - -struct SegDataUnit -{ - String type; - std::pair range; -}; - -// "type:[a, b)" => SegDataUnit -SegDataUnit parseSegDataUnit(String & s) -{ - boost::algorithm::trim(s); - std::vector values; - boost::split(values, s, boost::is_any_of(":")); - RUNTIME_CHECK(values.size() == 2, s); - return SegDataUnit{boost::algorithm::trim_copy(values[0]), parseRange(values[1])}; -} - -void check(const std::vector & seg_data_units) -{ - RUNTIME_CHECK(!seg_data_units.empty()); - std::vector stable_units; - std::vector mem_units; - for (size_t i = 0; i < seg_data_units.size(); i++) - { - const auto & type = seg_data_units[i].type; - if (type == "s") - { - stable_units.emplace_back(i); - } - else if (type == "d_mem" || type == "d_mem_del") - { - mem_units.emplace_back(i); - } - auto [begin, end] = seg_data_units[i].range; - RUNTIME_CHECK(begin < end, begin, end); - } - RUNTIME_CHECK(stable_units.empty() || (stable_units.size() == 1 && stable_units[0] == 0)); - std::vector expected_mem_units(mem_units.size()); - std::iota(expected_mem_units.begin(), expected_mem_units.end(), seg_data_units.size() - mem_units.size()); - RUNTIME_CHECK(mem_units == expected_mem_units, expected_mem_units, mem_units); -} - -std::vector parseSegData(std::string_view seg_data) -{ - std::vector str_seg_data_units; - boost::split(str_seg_data_units, seg_data, boost::is_any_of("|")); - RUNTIME_CHECK(!str_seg_data_units.empty(), seg_data); - std::vector seg_data_units; - seg_data_units.reserve(str_seg_data_units.size()); - for (auto & s : str_seg_data_units) - { - seg_data_units.emplace_back(parseSegDataUnit(s)); - } - check(seg_data_units); - return seg_data_units; -} - -} // namespace - class SkippableBlockInputStreamTest : public SegmentTestBasic { protected: @@ -139,7 +49,7 @@ class SkippableBlockInputStreamTest : public SegmentTestBasic *dm_context, columns_to_read, read_ranges, - EMPTY_FILTER, + EMPTY_RS_OPERATOR, std::numeric_limits::max(), DEFAULT_BLOCK_SIZE, enable_handle_clean_read, @@ -180,16 +90,20 @@ class SkippableBlockInputStreamTest : public SegmentTestBasic stream = getInputStream(segment, snapshot, columns_to_read, read_ranges); + size_t offset = 0; stream->readPrefix(); for (size_t i = 0; i < expected_blks.size(); ++i) { if (std::find(skip_block_idxs.begin(), skip_block_idxs.end(), i) != skip_block_idxs.end()) { + offset += expected_blks[i].rows(); size_t skipped_rows = stream->skipNextBlock(); ASSERT_EQ(skipped_rows, expected_blks[i].rows()); continue; } auto blk = stream->read(); + ASSERT_EQ(offset, blk.startOffset()); + offset += blk.rows(); ASSERT_BLOCK_EQ(expected_blks[i], blk); } ASSERT_BLOCK_EQ(stream->read(), Block{}); @@ -214,17 +128,19 @@ class SkippableBlockInputStreamTest : public SegmentTestBasic stream1->readPrefix(); stream2->readPrefix(); - std::default_random_engine e; + std::default_random_engine e(time(nullptr)); for (auto blk = stream1->read(); blk; blk = stream1->read()) { IColumn::Filter filter(blk.rows(), 1); - std::transform(filter.begin(), filter.end(), filter.begin(), [&e](auto) { return e() % 2 == 0 ? 0 : 1; }); + std::transform(filter.begin(), filter.end(), filter.begin(), [&e](auto) { return e() % 8192 == 0 ? 1 : 0; }); + filter[e() % blk.rows()] = 1; // should not be all 0. size_t passed_count = countBytesInFilter(filter); for (auto & col : blk) { col.column = col.column->filter(filter, passed_count); } auto blk2 = stream2->readWithFilter(filter); + ASSERT_EQ(blk.startOffset(), blk2.startOffset()); ASSERT_BLOCK_EQ(blk, blk2); } ASSERT_BLOCK_EQ(stream2->read(), Block{}); @@ -232,6 +148,69 @@ class SkippableBlockInputStreamTest : public SegmentTestBasic stream2->readSuffix(); } + void testLateMaterializationCase(std::string_view seg_data) + { + auto seg_data_units = parseSegData(seg_data); + for (const auto & unit : seg_data_units) + { + writeSegment(unit); + } + + auto [segment, snapshot] = getSegmentForRead(SEG_ID); + ColumnDefines columns_to_read = {getExtraHandleColumnDefine(options.is_common_handle), + getVersionColumnDefine()}; + + auto stream = getInputStream(segment, snapshot, columns_to_read, read_ranges); + + stream->readPrefix(); + std::vector expected_blks; + for (auto blk = stream->read(); blk; blk = stream->read()) + { + expected_blks.push_back(std::move(blk)); + } + stream->readSuffix(); + + stream = getInputStream(segment, snapshot, columns_to_read, read_ranges); + + std::default_random_engine e(time(nullptr)); + + size_t offset = 0; + stream->readPrefix(); + for (auto & eblk : expected_blks) + { + if (e() % 3 == 0) + { + offset += eblk.rows(); + size_t skipped_rows = stream->skipNextBlock(); + ASSERT_EQ(skipped_rows, eblk.rows()); + } + else if (e() % 3 == 1) + { + IColumn::Filter filter(eblk.rows(), 1); + std::transform(filter.begin(), filter.end(), filter.begin(), [&e](auto) { return e() % 8192 == 0 ? 1 : 0; }); + filter[e() % eblk.rows()] = 1; // should not be all 0. + auto blk = stream->readWithFilter(filter); + ASSERT_EQ(offset, blk.startOffset()); + ASSERT_EQ(blk.rows(), countBytesInFilter(filter)); + offset += eblk.rows(); + for (auto & col : eblk) + { + col.column = col.column->filter(filter, -1); + } + ASSERT_BLOCK_EQ(eblk, blk); + } + else + { + auto blk = stream->read(); + ASSERT_EQ(offset, blk.startOffset()); + offset += blk.rows(); + ASSERT_BLOCK_EQ(eblk, blk); + } + } + ASSERT_BLOCK_EQ(stream->read(), Block{}); + stream->readSuffix(); + } + void writeSegment(const SegDataUnit & unit) { const auto & type = unit.type; @@ -280,6 +259,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)"); testReadWithFilterCase("d_mem:[0, 1000)"); + testLateMaterializationCase("d_mem:[0, 1000)"); } CATCH @@ -288,6 +268,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)|d_mem:[0, 1000)", {0}); testReadWithFilterCase("d_mem:[0, 1000)|d_mem:[0, 1000)"); + testLateMaterializationCase("d_mem:[0, 1000)|d_mem:[0, 1000)"); } CATCH @@ -296,6 +277,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)|d_mem:[100, 200)", {3, 6, 9}); testReadWithFilterCase("d_mem:[0, 1000)|d_mem:[100, 200)"); + testLateMaterializationCase("d_mem:[0, 1000)|d_mem:[100, 200)"); } CATCH @@ -304,6 +286,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)|d_mem:[-100, 100)", {0, 1, 3, 4, 5, 6, 7, 8}); testReadWithFilterCase("d_mem:[0, 1000)|d_mem:[-100, 100)"); + testLateMaterializationCase("d_mem:[0, 1000)|d_mem:[-100, 100)"); } CATCH @@ -312,6 +295,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)|d_mem_del:[0, 1000)", {4, 5, 6}); testReadWithFilterCase("d_mem:[0, 1000)|d_mem_del:[0, 1000)"); + testLateMaterializationCase("d_mem:[0, 1000)|d_mem_del:[0, 1000)"); } CATCH @@ -320,6 +304,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)|d_mem_del:[100, 200)", {}); testReadWithFilterCase("d_mem:[0, 1000)|d_mem_del:[100, 200)"); + testLateMaterializationCase("d_mem:[0, 1000)|d_mem_del:[100, 200)"); } CATCH @@ -328,6 +313,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)|d_mem_del:[-100, 100)", {0, 1, 2, 3, 4, 5, 6, 7, 8}); testReadWithFilterCase("d_mem:[0, 1000)|d_mem_del:[-100, 100)"); + testLateMaterializationCase("d_mem:[0, 1000)|d_mem_del:[-100, 100)"); } CATCH @@ -336,6 +322,7 @@ try { testSkipBlockCase("d_tiny:[100, 500)|d_mem:[200, 1000)", {1, 2, 3, 4, 5, 6}); testReadWithFilterCase("d_tiny:[100, 500)|d_mem:[200, 1000)"); + testLateMaterializationCase("d_tiny:[100, 500)|d_mem:[200, 1000)"); } CATCH @@ -344,6 +331,7 @@ try { testSkipBlockCase("d_tiny:[100, 500)|d_tiny_del:[200, 300)|d_mem:[0, 100)", {7, 8, 9}); testReadWithFilterCase("d_tiny:[100, 500)|d_tiny_del:[200, 300)|d_mem:[0, 100)"); + testLateMaterializationCase("d_tiny:[100, 500)|d_tiny_del:[200, 300)|d_mem:[0, 100)"); } CATCH @@ -352,6 +340,7 @@ try { testSkipBlockCase("d_tiny:[100, 500)|d_dr:[250, 300)|d_mem:[240, 290)", {1, 2, 3, 4, 5, 9}); testReadWithFilterCase("d_tiny:[100, 500)|d_dr:[250, 300)|d_mem:[240, 290)"); + testLateMaterializationCase("d_tiny:[100, 500)|d_dr:[250, 300)|d_mem:[240, 290)"); } CATCH @@ -360,6 +349,7 @@ try { testSkipBlockCase("d_tiny:[100, 500)|d_big:[250, 1000)|d_mem:[240, 290)", {1, 3, 4, 9}); testReadWithFilterCase("d_tiny:[100, 500)|d_big:[250, 1000)|d_mem:[240, 290)"); + testLateMaterializationCase("d_tiny:[100, 500)|d_big:[250, 1000)|d_mem:[240, 290)"); } CATCH @@ -368,6 +358,7 @@ try { testSkipBlockCase("s:[0, 1024)|d_dr:[0, 1023)", {0}); testReadWithFilterCase("s:[0, 1024)|d_dr:[0, 1023)"); + testLateMaterializationCase("s:[0, 1024)|d_dr:[0, 1023)"); } CATCH @@ -376,6 +367,7 @@ try { testSkipBlockCase("s:[0, 102294)|d_dr:[0, 1023)", {2}); testReadWithFilterCase("s:[0, 102294)|d_dr:[0, 1023)"); + testLateMaterializationCase("s:[0, 102294)|d_dr:[0, 1023)"); } CATCH @@ -385,6 +377,7 @@ try { testSkipBlockCase("s:[0, 1024)|d_dr:[128, 256)|d_tiny_del:[300, 310)", {0}); testReadWithFilterCase("s:[0, 1024)|d_dr:[128, 256)|d_tiny_del:[300, 310)"); + testLateMaterializationCase("s:[0, 1024)|d_dr:[128, 256)|d_tiny_del:[300, 310)"); } CATCH @@ -394,6 +387,7 @@ try { testSkipBlockCase("s:[0, 1024)|d_dr:[128, 256)|d_tiny_del:[300, 310)|d_tiny:[200, 255)|d_mem:[298, 305)", {1, 2}); testReadWithFilterCase("s:[0, 1024)|d_dr:[128, 256)|d_tiny_del:[300, 310)|d_tiny:[200, 255)|d_mem:[298, 305)"); + testLateMaterializationCase("s:[0, 1024)|d_dr:[128, 256)|d_tiny_del:[300, 310)|d_tiny:[200, 255)|d_mem:[298, 305)"); } CATCH diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 3191e63aaf9..295e57e6528 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -25,6 +25,8 @@ #include #include #include +#include +#include #include #include #include @@ -39,6 +41,7 @@ #include #include #include +#include #include #include #include @@ -701,12 +704,13 @@ DM::RowKeyRanges StorageDeltaMerge::parseMvccQueryInfo( return ranges; } -DM::RSOperatorPtr StorageDeltaMerge::parseRoughSetFilter(const SelectQueryInfo & query_info, - const ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger) +DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryInfo & query_info, + const ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger) { - DM::RSOperatorPtr rs_operator = DM::EMPTY_FILTER; + // build rough set operator + DM::RSOperatorPtr rs_operator = DM::EMPTY_RS_OPERATOR; const bool enable_rs_filter = context.getSettingsRef().dt_enable_rough_set_filter; if (enable_rs_filter) { @@ -726,13 +730,104 @@ DM::RSOperatorPtr StorageDeltaMerge::parseRoughSetFilter(const SelectQueryInfo & }; rs_operator = FilterParser::parseDAGQuery(*query_info.dag_query, columns_to_read, std::move(create_attr_by_column_id), log); } - if (likely(rs_operator != DM::EMPTY_FILTER)) + if (likely(rs_operator != DM::EMPTY_RS_OPERATOR)) LOG_DEBUG(tracing_logger, "Rough set filter: {}", rs_operator->toDebugString()); } else LOG_DEBUG(tracing_logger, "Rough set filter is disabled."); - return rs_operator; + // build push down filter + if (likely(query_info.dag_query) && !query_info.dag_query->pushed_down_filters.empty()) + { + NamesAndTypes columns_to_read_name_and_type; + for (const auto & col : columns_to_read) + { + columns_to_read_name_and_type.emplace_back(col.name, col.type); + } + + std::unordered_set filter_column_names; + for (const auto & filter : query_info.dag_query->pushed_down_filters) + { + DB::getColumnNamesFromExpr(filter, columns_to_read_name_and_type, filter_column_names); + } + ColumnDefines filter_columns; + filter_columns.reserve(filter_column_names.size()); + for (const auto & name : filter_column_names) + { + auto iter = std::find_if( + columns_to_read.begin(), + columns_to_read.end(), + [&name](const ColumnDefine & d) -> bool { return d.name == name; }); + RUNTIME_CHECK(iter != columns_to_read.end()); + filter_columns.push_back(*iter); + } + + ColumnInfos table_scan_column_info; + table_scan_column_info.reserve(columns_to_read.size()); + const auto & table_infos = tidb_table_info.columns; + for (const auto & col : columns_to_read) + { + auto iter = std::find_if( + table_infos.begin(), + table_infos.end(), + [col](const ColumnInfo & c) -> bool { return c.id == col.id; }); + RUNTIME_CHECK(iter != table_infos.end()); + table_scan_column_info.push_back(*iter); + } + + std::vector need_cast_column; + need_cast_column.reserve(columns_to_read.size()); + for (const auto & col : table_scan_column_info) + { + if (!filter_column_names.contains(col.name)) + { + need_cast_column.push_back(ExtraCastAfterTSMode::None); + } + else + { + if (col.id != -1 && col.tp == TiDB::TypeTimestamp) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); + else if (col.id != -1 && col.tp == TiDB::TypeTime) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); + else + need_cast_column.push_back(ExtraCastAfterTSMode::None); + } + } + + std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, context); + ExpressionActionsChain chain; + auto & step = analyzer->initAndGetLastStep(chain); + auto & actions = step.actions; + ExpressionActionsPtr extra_cast = nullptr; + if (auto [has_cast, casted_columns] = analyzer->buildExtraCastsAfterTS(actions, need_cast_column, table_scan_column_info); has_cast) + { + NamesWithAliases project_cols; + for (size_t i = 0; i < columns_to_read.size(); ++i) + { + if (filter_column_names.contains(columns_to_read[i].name)) + { + project_cols.emplace_back(casted_columns[i], columns_to_read[i].name); + } + } + actions->add(ExpressionAction::project(project_cols)); + + for (auto & col : filter_columns) + step.required_output.push_back(col.name); + + extra_cast = chain.getLastActions(); + assert(extra_cast); + chain.finalize(); + chain.clear(); + LOG_DEBUG(tracing_logger, "Extra cast: {}", extra_cast->dumpActions()); + } + + // build filter expression actions + auto [before_where, filter_column_name, _] = ::DB::buildPushDownFilter(query_info.dag_query->pushed_down_filters, *analyzer); + LOG_DEBUG(tracing_logger, "Push down filter: {}", before_where->dumpActions()); + + return std::make_shared(rs_operator, before_where, filter_columns, filter_column_name, extra_cast); + } + return std::make_shared(rs_operator); } BlockInputStreams StorageDeltaMerge::read( @@ -773,7 +868,7 @@ BlockInputStreams StorageDeltaMerge::read( auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, query_info.req_id, tracing_logger); - auto rs_operator = parseRoughSetFilter(query_info, columns_to_read, context, tracing_logger); + auto filter = parsePushDownFilter(query_info, columns_to_read, context, tracing_logger); const auto & scan_context = mvcc_query_info.scan_context; @@ -784,7 +879,7 @@ BlockInputStreams StorageDeltaMerge::read( ranges, num_streams, /*max_version=*/mvcc_query_info.read_tso, - rs_operator, + filter, query_info.req_id, query_info.keep_order, /* is_fast_scan */ query_info.is_fast_scan, @@ -829,7 +924,7 @@ SourceOps StorageDeltaMerge::readSourceOps( auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, query_info.req_id, tracing_logger); - auto rs_operator = parseRoughSetFilter(query_info, columns_to_read, context, tracing_logger); + auto filter = parsePushDownFilter(query_info, columns_to_read, context, tracing_logger); const auto & scan_context = mvcc_query_info.scan_context; @@ -841,7 +936,7 @@ SourceOps StorageDeltaMerge::readSourceOps( ranges, num_streams, /*max_version=*/mvcc_query_info.read_tso, - rs_operator, + filter, query_info.req_id, query_info.keep_order, /* is_fast_scan */ query_info.is_fast_scan, @@ -1041,7 +1136,7 @@ void StorageDeltaMerge::deleteRows(const Context & context, size_t delete_rows) LOG_ERROR(log, "Rows after delete range not match, expected: {}, got: {}", (total_rows - delete_rows), after_delete_rows); } -DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() +DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() const { if (storeInited()) { diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index a27435e8c26..75d32add76d 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include @@ -171,7 +171,7 @@ class StorageDeltaMerge return getAndMaybeInitStore(); } - DM::DeltaMergeStorePtr getStoreIfInited(); + DM::DeltaMergeStorePtr getStoreIfInited() const; bool isCommonHandle() const override { return is_common_handle; } @@ -196,7 +196,7 @@ class StorageDeltaMerge const String & db_engine, const String & db_name_, const String & name_, - const DM::OptionTableInfoConstRef table_info_, + DM::OptionTableInfoConstRef table_info_, const ColumnsDescription & columns_, const ASTPtr & primary_expr_ast_, Timestamp tombstone, @@ -212,7 +212,7 @@ class StorageDeltaMerge const AlterCommands & commands, const String & database_name, const String & table_name, - const DB::DM::OptionTableInfoConstRef table_info_, + DB::DM::OptionTableInfoConstRef table_info_, const Context & context); DataTypePtr getPKTypeImpl() const override; @@ -227,11 +227,11 @@ class StorageDeltaMerge bool dataDirExist(); void shutdownImpl(); - /// Get Rough set filter from query - DM::RSOperatorPtr parseRoughSetFilter(const SelectQueryInfo & query_info, - const DM::ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger); + /// Get filters from query to construct rough set operation and push down filters. + DM::PushDownFilterPtr parsePushDownFilter(const SelectQueryInfo & query_info, + const DM::ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger); DM::RowKeyRanges parseMvccQueryInfo(const DB::MvccQueryInfo & mvcc_query_info, unsigned num_streams, diff --git a/dbms/src/Storages/StorageDisaggregatedRemote.cpp b/dbms/src/Storages/StorageDisaggregatedRemote.cpp index 1b65ddc307e..6deb3496849 100644 --- a/dbms/src/Storages/StorageDisaggregatedRemote.cpp +++ b/dbms/src/Storages/StorageDisaggregatedRemote.cpp @@ -332,17 +332,18 @@ DM::RSOperatorPtr StorageDisaggregated::buildRSOperator( const DM::ColumnDefinesPtr & columns_to_read) { if (!filter_conditions.hasValue()) - return DM::EMPTY_FILTER; + return DM::EMPTY_RS_OPERATOR; const bool enable_rs_filter = db_context.getSettingsRef().dt_enable_rough_set_filter; if (!enable_rs_filter) { LOG_DEBUG(log, "Rough set filter is disabled."); - return DM::EMPTY_FILTER; + return DM::EMPTY_RS_OPERATOR; } auto dag_query = std::make_unique( filter_conditions.conditions, + google::protobuf::RepeatedPtrField{}, // Not care now DAGPreparedSets{}, // Not care now NamesAndTypes{}, // Not care now db_context.getTimezoneInfo()); @@ -356,7 +357,7 @@ DM::RSOperatorPtr StorageDisaggregated::buildRSOperator( return DM::Attr{.col_name = "", .col_id = column_id, .type = DataTypePtr{}}; }; auto rs_operator = DM::FilterParser::parseDAGQuery(*dag_query, *columns_to_read, std::move(create_attr_by_column_id), log); - if (likely(rs_operator != DM::EMPTY_FILTER)) + if (likely(rs_operator != DM::EMPTY_RS_OPERATOR)) LOG_DEBUG(log, "Rough set filter: {}", rs_operator->toDebugString()); return rs_operator; } diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index e13f4cd6558..53c233a0950 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -12,13 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include #include #include #include +#include #include #include #include @@ -96,8 +96,10 @@ DM::RSOperatorPtr FilterParserTest::generateRsOperator(const String table_info_j { NamesAndTypes source_columns; std::tie(source_columns, std::ignore) = parseColumnsFromTableInfo(table_info); + const google::protobuf::RepeatedPtrField pushed_down_filters; dag_query = std::make_unique( conditions, + google::protobuf::RepeatedPtrField{}, // don't care pushed down filters DAGPreparedSets(), source_columns, timezone_info); @@ -429,7 +431,7 @@ try auto & timezone_info = ctx->getTimezoneInfo(); convertTimeZone(origin_time_stamp, converted_time, *timezone_info.timezone, time_zone_utc); - auto rs_operator = generateRsOperator(table_info_json, String("select * from default.t_111 where col_timestamp > cast_string_datetime('") + datetime + String("')")); + auto rs_operator = generateRsOperator(table_info_json, String("select * from default.t_111 where col_timestamp > cast_string_datetime('") + datetime + String("')"), timezone_info); EXPECT_EQ(rs_operator->name(), "greater"); EXPECT_EQ(rs_operator->getAttrs().size(), 1); EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_timestamp"); diff --git a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp new file mode 100644 index 00000000000..6ce86f4027e --- /dev/null +++ b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp @@ -0,0 +1,856 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB::tests +{ + +class ParsePushDownFilterTest : public ::testing::Test +{ +public: + static void SetUpTestCase() + { + try + { + registerFunctions(); + } + catch (DB::Exception &) + { + // Maybe another test has already registed, ignore exception here. + } + } + +protected: + LoggerPtr log = Logger::get(); + ContextPtr ctx = DB::tests::TiFlashTestEnv::getContext(); + TimezoneInfo default_timezone_info = DB::tests::TiFlashTestEnv::getContext()->getTimezoneInfo(); + DM::PushDownFilterPtr generatePushDownFilter(String table_info_json, const String & query, TimezoneInfo & timezone_info); +}; + + +DM::PushDownFilterPtr ParsePushDownFilterTest::generatePushDownFilter(const String table_info_json, const String & query, TimezoneInfo & timezone_info) +{ + const TiDB::TableInfo table_info(table_info_json, NullspaceID); + QueryTasks query_tasks; + std::tie(query_tasks, std::ignore) = compileQuery( + *ctx, + query, + [&](const String &, const String &) { + return table_info; + }, + getDAGProperties("")); + auto & dag_request = *query_tasks[0].dag_request; + DAGContext dag_context(dag_request, {}, NullspaceID, "", false, log); + ctx->setDAGContext(&dag_context); + // Don't care about regions information in this test + DAGQuerySource dag(*ctx); + auto query_block = *dag.getRootQueryBlock(); + google::protobuf::RepeatedPtrField empty_condition; + // Push down all filters + const google::protobuf::RepeatedPtrField & pushed_down_filters = query_block.children[0]->selection != nullptr ? query_block.children[0]->selection->selection().conditions() : empty_condition; + const google::protobuf::RepeatedPtrField & conditions = empty_condition; + + std::unique_ptr dag_query; + DM::ColumnDefines columns_to_read; + { + NamesAndTypes source_columns; + std::tie(source_columns, std::ignore) = parseColumnsFromTableInfo(table_info); + dag_query = std::make_unique( + conditions, + pushed_down_filters, + DAGPreparedSets(), + source_columns, + timezone_info); + for (const auto & column : table_info.columns) + { + columns_to_read.push_back(DM::ColumnDefine(column.id, column.name, getDataTypeByColumnInfo(column))); + } + } + + auto create_attr_by_column_id = [&columns_to_read](ColumnID column_id) -> DM::Attr { + auto iter = std::find_if( + columns_to_read.begin(), + columns_to_read.end(), + [column_id](const DM::ColumnDefine & d) -> bool { return d.id == column_id; }); + if (iter != columns_to_read.end()) + return DM::Attr{.col_name = iter->name, .col_id = iter->id, .type = iter->type}; + // Maybe throw an exception? Or check if `type` is nullptr before creating filter? + return DM::Attr{.col_name = "", .col_id = column_id, .type = DataTypePtr{}}; + }; + + auto rs_operator = DM::FilterParser::parseDAGQuery(*dag_query, columns_to_read, std::move(create_attr_by_column_id), log); + + // build push down filter + if (!dag_query->pushed_down_filters.empty()) + { + NamesAndTypes columns_to_read_name_and_type; + for (const auto & col : columns_to_read) + { + columns_to_read_name_and_type.emplace_back(col.name, col.type); + } + + std::unordered_set filter_column_names; + for (const auto & filter : dag_query->pushed_down_filters) + { + DB::getColumnNamesFromExpr(filter, columns_to_read_name_and_type, filter_column_names); + } + DM::ColumnDefines filter_columns; + filter_columns.reserve(filter_column_names.size()); + for (const auto & name : filter_column_names) + { + auto iter = std::find_if( + columns_to_read.begin(), + columns_to_read.end(), + [&name](const DM::ColumnDefine & d) -> bool { return d.name == name; }); + RUNTIME_CHECK(iter != columns_to_read.end()); + filter_columns.push_back(*iter); + } + + ColumnInfos table_scan_column_info; + table_scan_column_info.reserve(columns_to_read.size()); + const auto & table_infos = table_info.columns; + for (const auto & col : columns_to_read) + { + auto iter = std::find_if( + table_infos.begin(), + table_infos.end(), + [col](const ColumnInfo & c) -> bool { return c.id == col.id; }); + RUNTIME_CHECK(iter != table_infos.end()); + table_scan_column_info.push_back(*iter); + } + + std::vector need_cast_column; + need_cast_column.reserve(columns_to_read.size()); + for (const auto & col : table_scan_column_info) + { + if (!filter_column_names.contains(col.name)) + { + need_cast_column.push_back(ExtraCastAfterTSMode::None); + } + else + { + if (col.id != -1 && col.tp == TiDB::TypeTimestamp) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); + else if (col.id != -1 && col.tp == TiDB::TypeTime) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); + else + need_cast_column.push_back(ExtraCastAfterTSMode::None); + } + } + + std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, *ctx); + ExpressionActionsChain chain; + auto & step = analyzer->initAndGetLastStep(chain); + auto & actions = step.actions; + ExpressionActionsPtr extra_cast = nullptr; + if (auto [has_cast, casted_columns] = analyzer->buildExtraCastsAfterTS(actions, need_cast_column, table_scan_column_info); has_cast) + { + NamesWithAliases project_cols; + for (size_t i = 0; i < columns_to_read.size(); ++i) + { + if (filter_column_names.contains(columns_to_read[i].name)) + { + project_cols.emplace_back(casted_columns[i], columns_to_read[i].name); + } + } + actions->add(ExpressionAction::project(project_cols)); + + for (auto & col : filter_columns) + step.required_output.push_back(col.name); + + extra_cast = chain.getLastActions(); + assert(extra_cast); + chain.finalize(); + chain.clear(); + } + + // build filter expression actions + auto [before_where, filter_column_name, _] = ::DB::buildPushDownFilter(dag_query->pushed_down_filters, *analyzer); + + return std::make_shared(rs_operator, before_where, filter_columns, filter_column_name, extra_cast); + } + return std::make_shared(rs_operator); +} + +// Test cases for col and literal +TEST_F(ParsePushDownFilterTest, ColAndLiteral) +try +{ + const String table_info_json = R"json({ + "cols":[ + {"comment":"","default":null,"default_bit":null,"id":2,"name":{"L":"col_2","O":"col_2"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":8}} + ], + "pk_is_handle":false,"index_info":[],"is_common_handle":false, + "name":{"L":"t_111","O":"t_111"},"partition":null, + "comment":"Mocked.","id":30,"schema_version":-1,"state":0,"tiflash_replica":{"Count":0},"update_timestamp":1636471547239654 +})json"; + + { + // Equal between col and literal + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 = 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"666\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 1); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Greater between col and literal + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 > 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"greater\",\"col\":\"col_2\",\"value\":\"666\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 2); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // GreaterEqual between col and literal + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 >= 667", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater_equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"greater_equal\",\"col\":\"col_2\",\"value\":\"667\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 2); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Less between col and literal + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 < 777", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "less"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"less\",\"col\":\"col_2\",\"value\":\"777\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // LessEqual between col and literal + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 <= 776", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "less_equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"less_equal\",\"col\":\"col_2\",\"value\":\"776\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } +} +CATCH + +TEST_F(ParsePushDownFilterTest, LiteralAndCol) +try +{ + const String table_info_json = R"json({ + "cols":[ + {"comment":"","default":null,"default_bit":null,"id":2,"name":{"L":"col_2","O":"col_2"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":8}} + ], + "pk_is_handle":false,"index_info":[],"is_common_handle":false, + "name":{"L":"t_111","O":"t_111"},"partition":null, + "comment":"Mocked.","id":30,"schema_version":-1,"state":0,"tiflash_replica":{"Count":0},"update_timestamp":1636471547239654 +})json"; + // Test cases for literal and col (inverse direction) + { + // Equal between literal and col (take care of direction) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where 667 = col_2", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"667\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 1); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // NotEqual between literal and col (take care of direction) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where 667 != col_2", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "not_equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"not_equal\",\"col\":\"col_2\",\"value\":\"667\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Greater between literal and col (take care of direction) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where 667 < col_2", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"greater\",\"col\":\"col_2\",\"value\":\"667\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 1); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // GreaterEqual between literal and col (take care of direction) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where 667 <= col_2", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater_equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"greater_equal\",\"col\":\"col_2\",\"value\":\"667\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 2); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Less between literal and col (take care of direction) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where 777 > col_2", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "less"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"less\",\"col\":\"col_2\",\"value\":\"777\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // LessEqual between literal and col (take care of direction) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where 777 >= col_2", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "less_equal"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"less_equal\",\"col\":\"col_2\",\"value\":\"777\"}"); + + Block before_where_block = Block{toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439})}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } +} +CATCH + +// Test cases for Logic operator +TEST_F(ParsePushDownFilterTest, LogicOperator) +try +{ + const String table_info_json = R"json({ + "cols":[ + {"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"col_1","O":"col_1"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":254}}, + {"comment":"","default":null,"default_bit":null,"id":2,"name":{"L":"col_2","O":"col_2"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":8}}, + {"comment":"","default":null,"default_bit":null,"id":3,"name":{"L":"col_3","O":"col_3"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":8}} + ], + "pk_is_handle":false,"index_info":[],"is_common_handle":false, + "name":{"L":"t_111","O":"t_111"},"partition":null, + "comment":"Mocked.","id":30,"schema_version":-1,"state":0,"tiflash_replica":{"Count":0},"update_timestamp":1636471547239654 +})json"; + { + // Not + auto filter = generatePushDownFilter(table_info_json, "select col_1, col_2 from default.t_111 where NOT col_2=666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "not"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"not\",\"children\":[{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"666\"}]}"); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // And + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_1 = 'test1' and col_2 = 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "and"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + std::regex rx(R"(\{"op":"and","children":\[\{"op":"unsupported",.*\},\{"op":"equal","col":"col_2","value":"666"\}\]\})"); + EXPECT_TRUE(std::regex_search(rs_operator->toDebugString(), rx)); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 1); + EXPECT_EQ(filter->filter_columns.size(), 2); + } + + { + // OR + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 = 789 or col_2 = 777", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "or"); + EXPECT_EQ(rs_operator->getAttrs().size(), 2); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->getAttrs()[1].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[1].col_id, 2); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"or\",\"children\":[{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"789\"},{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"777\"}]}"); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 0); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + // More complicated + { + // And with "not supported" + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_1 = 'test1' and not col_2 = 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "and"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + std::regex rx(R"(\{"op":"and","children":\[\{"op":"unsupported",.*\},\{"op":"not","children":\[\{"op":"equal","col":"col_2","value":"666"\}\]\}\]\})"); + EXPECT_TRUE(std::regex_search(rs_operator->toDebugString(), rx)); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 1); + EXPECT_EQ(filter->filter_columns.size(), 2); + } + + { + // And with not + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 = 789 and not col_3 = 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "and"); + EXPECT_EQ(rs_operator->getAttrs().size(), 2); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->getAttrs()[1].col_name, "col_3"); + EXPECT_EQ(rs_operator->getAttrs()[1].col_id, 3); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"and\",\"children\":[{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"789\"},{\"op\":\"not\",\"children\":[{\"op\":\"equal\",\"col\":\"col_3\",\"value\":\"666\"}]}]}"); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 0); + EXPECT_EQ(filter->filter_columns.size(), 2); + } + + { + // And with or + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 = 789 and (col_3 = 666 or col_3 = 678)", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "and"); + EXPECT_EQ(rs_operator->getAttrs().size(), 3); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + EXPECT_EQ(rs_operator->getAttrs()[1].col_name, "col_3"); + EXPECT_EQ(rs_operator->getAttrs()[1].col_id, 3); + EXPECT_EQ(rs_operator->getAttrs()[2].col_name, "col_3"); + EXPECT_EQ(rs_operator->getAttrs()[2].col_id, 3); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"and\",\"children\":[{\"op\":\"equal\",\"col\":\"col_2\",\"value\":\"789\"},{\"op\":\"or\",\"children\":[{\"op\":\"equal\",\"col\":\"col_3\",\"value\":\"666\"},{\"op\":\"equal\",\"col\":\"col_3\",\"value\":\"678\"}]}]}"); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 789, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 1); + EXPECT_EQ(filter->filter_columns.size(), 2); + } + + { + // Or with "not supported" + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_1 = 'test1' or col_2 = 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "or"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + std::regex rx(R"(\{"op":"or","children":\[\{"op":"unsupported",.*\},\{"op":"equal","col":"col_2","value":"666"\}\]\})"); + EXPECT_TRUE(std::regex_search(rs_operator->toDebugString(), rx)); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 1, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 2); + EXPECT_EQ(filter->filter_columns.size(), 2); + } + + { + // Or with not + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_1 = 'test1' or not col_2 = 666", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "or"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_2"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 2); + std::regex rx(R"(\{"op":"or","children":\[\{"op":"unsupported",.*\},\{"op":"not","children":\[\{"op":"equal","col":"col_2","value":"666"\}\]\}\]\})"); + EXPECT_TRUE(std::regex_search(rs_operator->toDebugString(), rx)); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 666, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 2); + } + + { + // And between col and literal (not supported since And only support when child is ColumnExpr) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 and 1", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "and"); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"and\",\"children\":[{\"op\":\"unsupported\",\"reason\":\"child of logical and is not function\",\"content\":\"tp: ColumnRef val: \"\\200\\000\\000\\000\\000\\000\\000\\001\" field_type { tp: 8 flag: 4097 flen: 0 decimal: 0 collate: 0 }\",\"is_not\":\"0\"},{\"op\":\"unsupported\",\"reason\":\"child of logical and is not function\",\"content\":\"tp: Uint64 val: \"\\000\\000\\000\\000\\000\\000\\000\\001\" field_type { tp: 1 flag: 4129 flen: 0 decimal: 0 collate: 0 }\",\"is_not\":\"0\"}]}"); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 666, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 6); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + std::cout << " do query select * from default.t_111 where col_2 or 1 " << std::endl; + { + // Or between col and literal (not supported since Or only support when child is ColumnExpr) + auto filter = generatePushDownFilter(table_info_json, "select * from default.t_111 where col_2 or 1", default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "or"); + EXPECT_EQ(rs_operator->toDebugString(), "{\"op\":\"or\",\"children\":[{\"op\":\"unsupported\",\"reason\":\"child of logical operator is not function\",\"content\":\"tp: ColumnRef val: \"\\200\\000\\000\\000\\000\\000\\000\\001\" field_type { tp: 8 flag: 4097 flen: 0 decimal: 0 collate: 0 }\",\"is_not\":\"0\"},{\"op\":\"unsupported\",\"reason\":\"child of logical operator is not function\",\"content\":\"tp: Uint64 val: \"\\000\\000\\000\\000\\000\\000\\000\\001\" field_type { tp: 1 flag: 4129 flen: 0 decimal: 0 collate: 0 }\",\"is_not\":\"0\"}]}"); + + Block before_where_block = Block{{toVec("col_1", {"a", "b", "c", "test1", "d", "test1", "pingcap", "tiflash"}), + toVec("col_2", {0, 666, 0, 1, 121, 666, 667, 888439}), + toVec("col_3", {3, 121, 0, 121, 121, 666, 667, 888439})}}; + EXPECT_EQ(filter->extra_cast, nullptr); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + EXPECT_TRUE(col->isColumnConst()); // always true, so filter column is const column + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + // TODO: add is null and is not null test case + // after test framework support nullable column +} +CATCH + +// Test cases for date,datetime,timestamp column +TEST_F(ParsePushDownFilterTest, TimestampColumn) +try +{ + const String table_info_json = R"json({ + "cols":[ + {"comment":"","default":null,"default_bit":null,"id":4,"name":{"L":"col_timestamp","O":"col_time"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":5,"Elems":null,"Flag":1,"Flen":0,"Tp":7}}, + {"comment":"","default":null,"default_bit":null,"id":5,"name":{"L":"col_datetime","O":"col_datetime"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":5,"Elems":null,"Flag":1,"Flen":0,"Tp":12}}, + {"comment":"","default":null,"default_bit":null,"id":6,"name":{"L":"col_date","O":"col_date"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":5,"Elems":null,"Flag":1,"Flen":0,"Tp":14}} + ], + "pk_is_handle":false,"index_info":[],"is_common_handle":false, + "name":{"L":"t_111","O":"t_111"},"partition":null, + "comment":"Mocked.","id":30,"schema_version":-1,"state":0,"tiflash_replica":{"Count":0},"update_timestamp":1636471547239654 +})json"; + + String datetime = "1970-01-01 00:00:01.000000"; + ReadBufferFromMemory read_buffer(datetime.c_str(), datetime.size()); + UInt64 origin_time_stamp; + tryReadMyDateTimeText(origin_time_stamp, 6, read_buffer); + const auto & time_zone_utc = DateLUT::instance("UTC"); + UInt64 converted_time = origin_time_stamp; + std::cout << "origin_time_stamp: " << origin_time_stamp << std::endl; + // origin_time_stamp: 1802216106174185472 + + { + // Greater between TimeStamp col and Datetime literal, use local timezone + auto ctx = TiFlashTestEnv::getContext(); + auto & timezone_info = ctx->getTimezoneInfo(); + convertTimeZone(origin_time_stamp, converted_time, *timezone_info.timezone, time_zone_utc); + // converted_time: 0 + + auto filter = generatePushDownFilter(table_info_json, String("select * from default.t_111 where col_timestamp > cast_string_datetime('") + datetime + String("')"), timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_timestamp"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 4); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(converted_time) + String("\"}")); + + Block before_where_block = Block{{toVec("col_timestamp", {12, 1, 1802216106174185472, 1802216106174185472, 1, 43, 1802216106174185472, 888439}), + toVec("col_datetime", {1849259496301477883, 1849559496301477888, 0, 1, 1849259496301477888, 1849559496301477888, 667, 888439}), + toVec("col_date", {-1849559496301477888, 1849259496301477888, 0, 121, 121, 1849259496301477888, 667, 888439})}}; + EXPECT_TRUE(filter->extra_cast); + filter->extra_cast->execute(before_where_block); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 3); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Greater between TimeStamp col and Datetime literal, use Chicago timezone + auto ctx = TiFlashTestEnv::getContext(); + auto & timezone_info = ctx->getTimezoneInfo(); + timezone_info.resetByTimezoneName("America/Chicago"); + convertTimeZone(origin_time_stamp, converted_time, *timezone_info.timezone, time_zone_utc); + // converted_time: 1802216518491045888 + + auto filter = generatePushDownFilter(table_info_json, String("select * from default.t_111 where col_timestamp > cast_string_datetime('") + datetime + String("')"), timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_timestamp"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 4); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(converted_time) + String("\"}")); + + Block before_where_block = Block{{toVec("col_timestamp", {1849559496301477888, 1849560389654675456, 1949560389654675456, 1849259496301477888, 1849560389654675452, 1849559416301477888, 1849559496301477833, 888439}), + toVec("col_datetime", {1849259496301477883, 1849559496301477888, 0, 1, 1849259496301477888, 1849559496301477888, 667, 888439}), + toVec("col_date", {-1849559496301477888, 1849259496301477888, 0, 121, 121, 1849259496301477888, 667, 888439})}}; + EXPECT_TRUE(filter->extra_cast); + filter->extra_cast->execute(before_where_block); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Greater between TimeStamp col and Datetime literal, use Chicago timezone + auto ctx = TiFlashTestEnv::getContext(); + auto & timezone_info = ctx->getTimezoneInfo(); + timezone_info.resetByTimezoneOffset(28800); + convertTimeZoneByOffset(origin_time_stamp, converted_time, false, timezone_info.timezone_offset); + // converted_time: 0 + + auto filter = generatePushDownFilter(table_info_json, String("select * from default.t_111 where col_timestamp > cast_string_datetime('") + datetime + String("')"), timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_timestamp"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 4); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(converted_time) + String("\"}")); + + Block before_where_block = Block{{toVec("col_timestamp", {1849559496301477888, 1849560389654675456, 1949560389654675456, 1849259496301477888, 1849560389654675452, 1849559416301477888, 1849559496301477833, 888439}), + toVec("col_datetime", {1849259496301477883, 1849559496301477888, 0, 1, 1849259496301477888, 1849559496301477888, 667, 888439}), + toVec("col_date", {-1849559496301477888, 1849259496301477888, 0, 121, 121, 1849259496301477888, 667, 888439})}}; + EXPECT_TRUE(filter->extra_cast); + filter->extra_cast->execute(before_where_block); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 7); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Greater between Datetime col and Datetime literal + auto filter = generatePushDownFilter(table_info_json, String("select * from default.t_111 where col_datetime > cast_string_datetime('") + datetime + String("')"), default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_datetime"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 5); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_datetime\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); + + Block before_where_block = Block{{toVec("col_timestamp", {1849559496301477888, 1849560389654675456, 1949560389654675456, 1849259496301477888, 1849560389654675452, 1849559416301477888, 1849559496301477833, 888439}), + toVec("col_datetime", {1849259496301477883, 1849559496301477888, 0, 1, 1849259496301477888, 1849559496301477888, 667, 888439}), + toVec("col_date", {-1849559496301477888, 1849259496301477888, 0, 121, 121, 1849259496301477888, 667, 888439})}}; + EXPECT_TRUE(!filter->extra_cast); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 4); + EXPECT_EQ(filter->filter_columns.size(), 1); + } + + { + // Greater between Date col and Datetime literal + auto filter = generatePushDownFilter(table_info_json, String("select * from default.t_111 where col_date > cast_string_datetime('") + datetime + String("')"), default_timezone_info); + auto & rs_operator = filter->rs_operator; + EXPECT_EQ(rs_operator->name(), "greater"); + EXPECT_EQ(rs_operator->getAttrs().size(), 1); + EXPECT_EQ(rs_operator->getAttrs()[0].col_name, "col_date"); + EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 6); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_date\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); + + Block before_where_block = Block{{toVec("col_timestamp", {1849559496301477888, 1849560389654675456, 1949560389654675456, 1849259496301477888, 1849560389654675452, 1849559416301477888, 1849559496301477833, 888439}), + toVec("col_datetime", {1849259496301477883, 1849559496301477888, 0, 1, 1849259496301477888, 1849559496301477888, 667, 888439}), + toVec("col_date", {-1849559496301477888, 1849560046057291779, 0, 121, 1849560046057291798, 1849259496301477888, 667, 888439})}}; + EXPECT_TRUE(!filter->extra_cast); + filter->before_where->execute(before_where_block); + EXPECT_EQ(before_where_block.rows(), 8); + auto & col = before_where_block.getByName(filter->filter_column_name).column; + const auto * concrete_column = typeid_cast(&(*col)); + EXPECT_EQ(countBytesInFilter(concrete_column->getData()), 3); + EXPECT_EQ(filter->filter_columns.size(), 1); + } +} +CATCH + +} // namespace DB::tests