From cac69a508011891f9a21fffe72cca68ed94af4e8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 6 Mar 2023 14:37:11 +0800 Subject: [PATCH 01/25] support late materialization Signed-off-by: Lloyd-Pottiger --- contrib/tipb | 2 +- dbms/src/Debug/MockStorage.cpp | 4 +- .../Coprocessor/DAGExpressionAnalyzer.cpp | 39 +++-- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 12 +- dbms/src/Flash/Coprocessor/DAGQueryInfo.h | 4 + .../Coprocessor/DAGStorageInterpreter.cpp | 71 +-------- .../Flash/Coprocessor/DAGStorageInterpreter.h | 3 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 7 + dbms/src/Flash/Coprocessor/DAGUtils.h | 1 + .../Flash/Coprocessor/InterpreterUtils.cpp | 37 ++++- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 7 +- dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 1 + dbms/src/Flash/Coprocessor/TiDBTableScan.h | 11 ++ .../BitmapFilterBlockInputStream.cpp | 36 +---- .../BitmapFilterBlockInputStream.h | 11 +- .../DeltaMerge/ColumnFile/ColumnFileBig.cpp | 2 +- .../DeltaMerge/DMSegmentThreadInputStream.h | 4 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 3 +- .../DeltaMerge/Filter/PushDownFilter.h | 53 +++++++ .../Storages/DeltaMerge/Filter/RSOperator.h | 2 +- .../DeltaMerge/FilterParser/FilterParser.cpp | 4 +- .../DeltaMerge/FilterParser/FilterParser.h | 8 +- .../LateMaterializationBlockInputStream.cpp | 4 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 136 +++++++++++++++--- dbms/src/Storages/DeltaMerge/Segment.h | 13 +- .../Storages/DeltaMerge/SegmentReadTaskPool.h | 5 +- .../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 | 8 +- .../gtest_skippable_block_input_stream.cpp | 2 +- dbms/src/Storages/StorageDeltaMerge.cpp | 101 +++++++++---- dbms/src/Storages/StorageDeltaMerge.h | 18 +-- .../Storages/tests/gtest_filter_parser.cpp | 2 + 35 files changed, 400 insertions(+), 221 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h diff --git a/contrib/tipb b/contrib/tipb index cf83fcb61a3..e82242cc05d 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit cf83fcb61a3b749f38dcfe8a31e157fa33ca206e +Subproject commit e82242cc05dea24e4ed288cd53b22c4a3e5d550f diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 788c5c9c9c2..23c4877de91 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -154,6 +154,7 @@ BlockInputStreamPtr MockStorage::getStreamFromDeltaMerge(Context & context, Int6 auto scan_context = std::make_shared(); QueryProcessingStage::Enum stage; + const google::protobuf::RepeatedPtrField pushed_down_filters; SelectQueryInfo query_info; query_info.query = std::make_shared(); query_info.mvcc_query_info = std::make_unique(context.getSettingsRef().resolve_locks, std::numeric_limits::max(), scan_context); @@ -162,10 +163,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, + pushed_down_filters, 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 4c045567b55..7d8dd9a2a79 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -62,13 +62,6 @@ bool isUInt8Type(const DataTypePtr & type) return removeNullable(type)->getTypeId() == TypeIndex::UInt8; } -tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info) -{ - return dag_timezone_info.is_name_based - ? constructStringLiteralTiExpr(dag_timezone_info.timezone_name) - : constructInt64LiteralTiExpr(dag_timezone_info.timezone_offset); -} - String getAggFuncName( const tipb::Expr & expr, const tipb::Aggregation & agg, @@ -900,7 +893,6 @@ String DAGExpressionAnalyzer::appendTimeZoneCast( bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( const ExpressionActionsPtr & actions, - const std::vector & need_cast_column, const ColumnInfos & table_scan_columns) { bool has_cast = false; @@ -915,44 +907,47 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( // For Duration String fsp_col; static const String dur_func_name = "FunctionConvertDurationFromNanos"; - for (size_t i = 0; i < need_cast_column.size(); ++i) + + // After apply cast, some columns' name will be changed, and new column will be added. + // We add a projection to keep the original column names. + NamesWithAliases project_cols; + for (size_t i = 0; i < table_scan_columns.size(); ++i) { - if (!context.getTimezoneInfo().is_utc_timezone && need_cast_column[i] == ExtraCastAfterTSMode::AppendTimeZoneCast) + const auto & col = table_scan_columns[i]; + String casted_name = source_columns[i].name; + + if (!context.getTimezoneInfo().is_utc_timezone && col.id != -1 && col.tp == TiDB::TypeTimestamp) { - 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; } - if (need_cast_column[i] == ExtraCastAfterTSMode::AppendDurationCast) + if (col.id != -1 && col.tp == TiDB::TypeTime) { if (table_scan_columns[i].decimal > 6) throw Exception("fsp must <= 6", ErrorCodes::LOGICAL_ERROR); 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); source_columns[i].type = actions->getSampleBlock().getByName(casted_name).type; has_cast = true; } + + project_cols.emplace_back(casted_name, source_columns[i].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; } bool DAGExpressionAnalyzer::appendExtraCastsAfterTS( ExpressionActionsChain & chain, - const std::vector & need_cast_column, - const TiDBTableScan & table_scan) + const DB::ColumnInfos & table_scan_columns) { auto & step = initAndGetLastStep(chain); + auto & actions = step.actions; - bool has_cast = buildExtraCastsAfterTS(step.actions, need_cast_column, table_scan.getColumns()); + auto has_cast = buildExtraCastsAfterTS(actions, table_scan_columns); for (auto & col : source_columns) step.required_output.push_back(col.name); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index a63cefb97cd..9e6b83cf560 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -148,8 +148,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable // Int64 to duration. bool appendExtraCastsAfterTS( ExpressionActionsChain & chain, - const std::vector & need_cast_column, - const TiDBTableScan & table_scan); + const DB::ColumnInfos & table_scan_columns); /// return true if some actions is needed bool appendJoinKeyAndJoinFilters( @@ -263,6 +262,10 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & expr_name); + bool buildExtraCastsAfterTS( + const ExpressionActionsPtr & actions, + const ColumnInfos & table_scan_columns); + /** * when force_uint8 is false, alignReturnType align the data type in tiflash with the data type in dag request, otherwise * always convert the return type to uint8 or nullable(uint8) @@ -278,11 +281,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); - std::pair buildJoinKey( const ExpressionActionsPtr & actions, const google::protobuf::RepeatedPtrField & keys, diff --git a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h index 5d95ce7dcd6..e8db3b1f670 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -28,15 +28,19 @@ 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_) + , pushed_down_filters(pushed_down_filters_) , dag_sets(std::move(dag_sets_)) , source_columns(source_columns_) , timezone_info(timezone_info_){}; // 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; diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index ab963e9d9a0..4f731da3c08 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -169,49 +169,6 @@ bool hasRegionToRead(const DAGContext & dag_context, const TiDBTableScan & table return has_region_to_read; } -// add timezone cast for timestamp type, this is used to support session level timezone -// -std::tuple addExtraCastsAfterTs( - DAGExpressionAnalyzer & analyzer, - const std::vector & need_cast_column, - const TiDBTableScan & table_scan) -{ - bool has_need_cast_column = false; - for (auto b : need_cast_column) - has_need_cast_column |= (b != ExtraCastAfterTSMode::None); - if (!has_need_cast_column) - return {false, nullptr, 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)) - { - ExpressionActionsPtr extra_cast = chain.getLastActions(); - 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}; - } - else - { - return {false, nullptr, nullptr}; - } -} - void injectFailPointForLocalRead([[maybe_unused]] const SelectQueryInfo & query_info) { // Inject failpoint to throw RegionException for testing @@ -377,7 +334,7 @@ void DAGStorageInterpreter::prepare() assert(storages_with_structure_lock.find(logical_table_id) != storages_with_structure_lock.end()); storage_for_logical_table = storages_with_structure_lock[logical_table_id].storage; - std::tie(required_columns, source_columns, is_need_add_cast_column) = getColumnsForTableScan(); + std::tie(required_columns, source_columns) = getColumnsForTableScan(); analyzer = std::make_unique(std::move(source_columns), context); } @@ -387,8 +344,8 @@ 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); - if (has_cast) + auto extra_cast = ::DB::addExtraCastsAfterTs(*analyzer, table_scan.getColumns()); + if (extra_cast.has_value()) { assert(remote_read_streams_start_index <= pipeline.streams.size()); size_t i = 0; @@ -396,16 +353,9 @@ void DAGStorageInterpreter::executeCastAfterTableScan( while (i < remote_read_streams_start_index) { auto & stream = pipeline.streams[i++]; - stream = std::make_shared(stream, extra_cast, log->identifier()); + stream = std::make_shared(stream, extra_cast.value(), 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"); - } } } @@ -583,6 +533,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()); @@ -928,12 +879,10 @@ std::unordered_map DAG return storages_with_lock; } -std::tuple> DAGStorageInterpreter::getColumnsForTableScan() +std::pair DAGStorageInterpreter::getColumnsForTableScan() { Names required_columns_tmp; NamesAndTypes source_columns_tmp; - std::vector need_cast_column; - need_cast_column.reserve(table_scan.getColumnSize()); String handle_column_name = MutableSupport::tidb_pk_column_name; if (auto pk_handle_col = storage_for_logical_table->getTableInfo().getPKHandleColumn()) handle_column_name = pk_handle_col->get().name; @@ -971,15 +920,9 @@ 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 - need_cast_column.push_back(ExtraCastAfterTSMode::None); } - return {required_columns_tmp, source_columns_tmp, need_cast_column}; + return {required_columns_tmp, source_columns_tmp}; } // Build remote requests from `region_retry_from_local_region` and `table_regions_info.remote_regions` diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index c9be8e00201..0d6110b9470 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -79,7 +79,7 @@ class DAGStorageInterpreter std::unordered_map getAndLockStorages(Int64 query_schema_version); - std::tuple> getColumnsForTableScan(); + std::pair getColumnsForTableScan(); std::vector buildRemoteRequests(const DM::ScanContextPtr & scan_context); @@ -103,7 +103,6 @@ class DAGStorageInterpreter void executeImpl(DAGPipeline & pipeline); private: - std::vector is_need_add_cast_column; /// it shouldn't be hash map because duplicated region id may occur if merge regions to retry of dag. RegionRetryList region_retry_from_local_region; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 83563c47338..802a78ec40f 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -726,6 +726,13 @@ std::unordered_map getFuncNameToSigMap() } } // namespace +tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info) +{ + return dag_timezone_info.is_name_based + ? constructStringLiteralTiExpr(dag_timezone_info.timezone_name) + : constructInt64LiteralTiExpr(dag_timezone_info.timezone_offset); +} + bool isScalarFunctionExpr(const tipb::Expr & expr) { return expr.tp() == tipb::ExprType::ScalarFunc; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 1fe665e9c89..48767b5e599 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -31,6 +31,7 @@ namespace DB { class DAGContext; +tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info); bool isLiteralExpr(const tipb::Expr & expr); Field decodeLiteral(const tipb::Expr & expr); bool isFunctionExpr(const tipb::Expr & expr); diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index a6d85e36ed4..1f0b7e6e38b 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -175,15 +175,44 @@ void executeCreatingSets( } } +// add timezone cast for timestamp type, this is used to support session level timezone +std::optional addExtraCastsAfterTs( + DAGExpressionAnalyzer & analyzer, + const DB::ColumnInfos & table_scan_columns) +{ + bool has_need_cast_column = false; + for (const auto & col : table_scan_columns) + { + has_need_cast_column |= (col.id != -1 && (col.tp == TiDB::TypeTimestamp || col.tp == TiDB::TypeTime)); + } + if (!has_need_cast_column) + return std::nullopt; + + ExpressionActionsChain chain; + // execute timezone cast or duration cast if needed for local table scan + if (analyzer.appendExtraCastsAfterTS(chain, table_scan_columns)) + { + ExpressionActionsPtr extra_cast = chain.getLastActions(); + assert(extra_cast); + chain.finalize(); + chain.clear(); + return extra_cast; + } + else + { + return std::nullopt; + } +} + 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 +236,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. diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 3d2ebd33afc..eb3e66c77bb 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -68,8 +68,13 @@ void executeCreatingSets( size_t max_streams, const LoggerPtr & log); +// add timezone cast for timestamp type, this is used to support session level timezone +std::optional addExtraCastsAfterTs( + DAGExpressionAnalyzer & analyzer, + const DB::ColumnInfos & table_scan_columns); + std::tuple buildPushDownFilter( - const FilterConditions & filter_conditions, + const google::protobuf::RepeatedPtrField & conditions, DAGExpressionAnalyzer & analyzer); void executePushedDownFilter( 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/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 6f32de937b9..7f107913421 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp @@ -40,7 +40,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 6cf66ca0bf2..a23d55e818d 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 5f07d6c7fd4..19664dab2dc 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -980,7 +980,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, diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 0f5de36e872..d0f05936153 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -315,7 +316,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/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h new file mode 100644 index 00000000000..89e87bb15e8 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -0,0 +1,53 @@ +// Copyright 2022 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 "RSOperator.h" + +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_) + : rs_operator(rs_operator_) + , beofre_where(beofre_where_) + , filter_columns(std::move(filter_columns_)) + , filter_column_name(std::move(filter_column_name_)) + {} + + explicit PushDownFilter(const RSOperatorPtr & rs_operator_) + : rs_operator(rs_operator_) + , beofre_where(nullptr) + , filter_columns({}) + {} + + RSOperatorPtr rs_operator; + ExpressionActionsPtr beofre_where; + ColumnDefines filter_columns; + String filter_column_name; +}; + +} // namespace DB::DM \ No newline at end of file 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..95798aebd37 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,7 +392,7 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, FilterParser::AttrCreatorByColumnID && creator, const LoggerPtr & log) { - RSOperatorPtr op = EMPTY_FILTER; + RSOperatorPtr op = EMPTY_RS_OPERATOR; if (dag_info.filters.empty()) return op; diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h index 36c53ca0a85..eeabe53821d 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,11 @@ struct DAGQueryInfo; namespace DM { +namespace cop +{ +ColumnID getColumnIDForColumnExpr(const tipb::Expr & expr, const ColumnDefines & columns_to_read); +} // namespace cop + class RSOperator; using RSOperatorPtr = std::shared_ptr; @@ -42,7 +48,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..0d7f3b1f0d8 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp @@ -125,8 +125,8 @@ Block LateMaterializationBlockInputStream::readImpl() 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/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 85bbfcb0ac6..6155c0f3913 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -33,7 +34,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -572,16 +575,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: @@ -710,7 +713,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, @@ -845,7 +848,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); @@ -1171,7 +1174,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, @@ -1196,7 +1199,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, @@ -1456,7 +1459,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, @@ -1485,7 +1488,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, @@ -1665,7 +1668,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, @@ -2150,7 +2153,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(), @@ -2201,7 +2204,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(), @@ -2236,7 +2239,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(), @@ -2473,7 +2476,7 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(BitmapFilterPtr && bitma 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, @@ -2485,7 +2488,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, @@ -2500,6 +2503,89 @@ 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) +{ + 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); + + 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); + + filter_column_stream = std::make_shared(filter_column_stream, filter->beofre_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()); + } + + 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); + + return std::make_shared(columns_to_read, filter_column_stream, rest_column_stream, bitmap_filter, dm_context.tracing_id); +} + RowKeyRanges Segment::shrinkRowKeyRanges(const RowKeyRanges & read_ranges) { RowKeyRanges real_ranges; @@ -2516,7 +2602,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) { @@ -2529,17 +2615,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->beofre_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 67a6db771a4..31724a042f3 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -165,7 +165,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); @@ -631,10 +631,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 db1bbda7012..2a3538fe8da 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -15,6 +15,7 @@ #pragma once #include #include +#include #include #include #include @@ -163,7 +164,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_, @@ -260,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 53577eeacb7..6a495f10d8d 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -56,7 +56,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(), @@ -256,7 +256,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 0d4a087da76..557e8c270f7 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 @@ -548,7 +548,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 ee8ea8455f6..a97a021c74a 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -132,7 +132,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..b3af265de95 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp @@ -521,7 +521,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 +543,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 +563,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 +591,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); 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..4e03542a42d 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 @@ -139,7 +139,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, diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 8a76f1d90da..85d50ba671f 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 @@ -702,12 +705,13 @@ DM::RowKeyRanges StorageDeltaMerge::parseMvccQueryInfo(const DB::MvccQueryInfo & } -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) { @@ -727,13 +731,50 @@ 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 (!query_info.dag_query->pushed_down_filters.empty()) + { + std::unordered_set filter_column_ids; + for (const auto & filter : query_info.dag_query->filters) + { + filter_column_ids.insert(DB::DM::cop::getColumnIDForColumnExpr(filter, columns_to_read)); + } + ColumnDefines filter_columns; + for (const auto id : filter_column_ids) + { + auto iter = std::find_if( + columns_to_read.begin(), + columns_to_read.end(), + [id](const ColumnDefine & d) -> bool { return d.id == id; }); + RUNTIME_CHECK(iter != columns_to_read.end()); + filter_columns.push_back(*iter); + } + + for (const auto & col : filter_columns) + { + // do not support push down filter on datetime and time + RUNTIME_CHECK(col.id == 1 || (col.type->getTypeId() != TypeIndex::MyDateTime && col.type->getTypeId() != TypeIndex::MyTime)); + } + + 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); + } + + // build filter expression actions + std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, context); + auto [before_where, filter_column_name, _] = ::DB::buildPushDownFilter(query_info.dag_query->pushed_down_filters, *analyzer); + + return std::make_shared(rs_operator, before_where, filter_columns, filter_column_name); + } + return std::make_shared(rs_operator); } BlockInputStreams StorageDeltaMerge::read( @@ -774,7 +815,7 @@ BlockInputStreams StorageDeltaMerge::read( auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, 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; @@ -785,7 +826,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, @@ -852,7 +893,7 @@ UInt64 StorageDeltaMerge::onSyncGc(Int64 limit, const GCOptions & gc_options) { if (storeInited()) { - return _store->onSyncGc(limit, gc_options); + return store->onSyncGc(limit, gc_options); } return 0; } @@ -941,7 +982,7 @@ DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() { if (storeInited()) { - return _store; + return store; } return nullptr; } @@ -1175,7 +1216,7 @@ try std::lock_guard lock(store_mutex); // Avoid concurrent init store and DDL. if (storeInited()) { - _store->applyAlters(commands, table_info, max_column_id_used, context); + store->applyAlters(commands, table_info, max_column_id_used, context); } else { @@ -1214,12 +1255,12 @@ ColumnDefines StorageDeltaMerge::getStoreColumnDefines() const { if (storeInited()) { - return _store->getTableColumns(); + return store->getTableColumns(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return _store->getTableColumns(); + return store->getTableColumns(); } ColumnDefines cols; cols.emplace_back(table_column_info->handle_column_define); @@ -1259,13 +1300,13 @@ void StorageDeltaMerge::rename( } if (storeInited()) { - _store->rename(new_path_to_db, new_database_name, new_table_name); + store->rename(new_path_to_db, new_database_name, new_table_name); return; } std::lock_guard lock(store_mutex); if (storeInited()) { - _store->rename(new_path_to_db, new_database_name, new_table_name); + store->rename(new_path_to_db, new_database_name, new_table_name); } else { @@ -1278,12 +1319,12 @@ String StorageDeltaMerge::getTableName() const { if (storeInited()) { - return _store->getTableName(); + return store->getTableName(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return _store->getTableName(); + return store->getTableName(); } return table_column_info->table_name; } @@ -1292,12 +1333,12 @@ String StorageDeltaMerge::getDatabaseName() const { if (storeInited()) { - return _store->getDatabaseName(); + return store->getDatabaseName(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return _store->getDatabaseName(); + return store->getDatabaseName(); } return table_column_info->db_name; } @@ -1421,7 +1462,7 @@ BlockInputStreamPtr StorageDeltaMerge::status() StoreStats stat; if (storeInited()) { - stat = _store->getStoreStats(); + stat = store->getStoreStats(); } #define INSERT_INT(NAME) \ @@ -1527,7 +1568,7 @@ void StorageDeltaMerge::shutdownImpl() return; if (storeInited()) { - _store->shutdown(); + store->shutdown(); } } @@ -1553,12 +1594,12 @@ DataTypePtr StorageDeltaMerge::getPKTypeImpl() const { if (storeInited()) { - return _store->getPKDataType(); + return store->getPKDataType(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return _store->getPKDataType(); + return store->getPKDataType(); } return table_column_info->handle_column_define.type; } @@ -1567,12 +1608,12 @@ SortDescription StorageDeltaMerge::getPrimarySortDescription() const { if (storeInited()) { - return _store->getPrimarySortDescription(); + return store->getPrimarySortDescription(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return _store->getPrimarySortDescription(); + return store->getPrimarySortDescription(); } SortDescription desc; desc.emplace_back(table_column_info->handle_column_define.name, /* direction_= */ 1, /* nulls_direction_= */ 1); @@ -1583,12 +1624,12 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore() { if (storeInited()) { - return _store; + return store; } std::lock_guard lock(store_mutex); - if (_store == nullptr) + if (store == nullptr) { - _store = std::make_shared( + store = std::make_shared( global_context, data_path_contains_database_name, table_column_info->db_name, @@ -1603,7 +1644,7 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore() table_column_info.reset(nullptr); store_inited.store(true, std::memory_order_release); } - return _store; + return store; } bool StorageDeltaMerge::initStoreIfDataDirExist() diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index f98d7b703d5..8c03155998c 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 @@ -172,7 +172,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, @@ -188,7 +188,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; @@ -203,11 +203,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, @@ -237,7 +237,7 @@ class StorageDeltaMerge std::unique_ptr table_column_info; // After create DeltaMergeStore object, it is deprecated. std::atomic store_inited; - DM::DeltaMergeStorePtr _store; + DM::DeltaMergeStorePtr store; Strings pk_column_names; // TODO: remove it. Only use for debug from ch-client. bool is_common_handle = false; diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index 0e1265bcfd1..c0fb423bbab 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -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, + pushed_down_filters, DAGPreparedSets(), source_columns, timezone_info); From 03d9df612576fdd8c04b421cca043aff8c8b764a Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 7 Mar 2023 15:02:21 +0800 Subject: [PATCH 02/25] fix Signed-off-by: Lloyd-Pottiger --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 12 +- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 8 +- .../Coprocessor/DAGStorageInterpreter.cpp | 29 ++++ dbms/src/Flash/Coprocessor/DAGUtils.cpp | 7 - dbms/src/Flash/Coprocessor/DAGUtils.h | 1 - .../Flash/Coprocessor/InterpreterUtils.cpp | 29 ---- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 5 - .../DeltaMerge/tests/gtest_segment_bitmap.cpp | 140 +--------------- .../tests/gtest_segment_test_basic.cpp | 22 +++ .../tests/gtest_segment_test_basic.h | 1 + .../DeltaMerge/tests/gtest_segment_util.cpp | 155 ++++++++++++++++++ .../DeltaMerge/tests/gtest_segment_util.h | 56 +++++++ .../gtest_skippable_block_input_stream.cpp | 92 +---------- 13 files changed, 280 insertions(+), 277 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 7d8dd9a2a79..8b8b532725a 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -62,6 +62,13 @@ bool isUInt8Type(const DataTypePtr & type) return removeNullable(type)->getTypeId() == TypeIndex::UInt8; } +tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info) +{ + return dag_timezone_info.is_name_based + ? constructStringLiteralTiExpr(dag_timezone_info.timezone_name) + : constructInt64LiteralTiExpr(dag_timezone_info.timezone_offset); +} + String getAggFuncName( const tipb::Expr & expr, const tipb::Aggregation & agg, @@ -910,7 +917,7 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( // After apply cast, some columns' name will be changed, and new column will be added. // We add a projection to keep the original column names. - NamesWithAliases project_cols; + NamesWithAliases project_to_origin_names; for (size_t i = 0; i < table_scan_columns.size(); ++i) { const auto & col = table_scan_columns[i]; @@ -934,8 +941,9 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( has_cast = true; } - project_cols.emplace_back(casted_name, source_columns[i].name); + project_to_origin_names.emplace_back(casted_name, source_columns[i].name); } + actions->add(ExpressionAction::project(project_to_origin_names)); return has_cast; } diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 9e6b83cf560..55c5bdc9de7 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -262,10 +262,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & expr_name); - bool buildExtraCastsAfterTS( - const ExpressionActionsPtr & actions, - const ColumnInfos & table_scan_columns); - /** * when force_uint8 is false, alignReturnType align the data type in tiflash with the data type in dag request, otherwise * always convert the return type to uint8 or nullable(uint8) @@ -281,6 +277,10 @@ class DAGExpressionAnalyzer : private boost::noncopyable const String & expr_name, bool force_uint8); + bool buildExtraCastsAfterTS( + const ExpressionActionsPtr & actions, + const ColumnInfos & table_scan_columns); + std::pair buildJoinKey( const ExpressionActionsPtr & actions, const google::protobuf::RepeatedPtrField & keys, diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 7cab4e9d42a..5776a2124d9 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -178,6 +178,35 @@ bool hasRegionToRead(const DAGContext & dag_context, const TiDBTableScan & table return has_region_to_read; } +// add timezone cast for timestamp type, this is used to support session level timezone +std::optional addExtraCastsAfterTs( + DAGExpressionAnalyzer & analyzer, + const DB::ColumnInfos & table_scan_columns) +{ + bool has_need_cast_column = false; + for (const auto & col : table_scan_columns) + { + has_need_cast_column |= (col.id != -1 && (col.tp == TiDB::TypeTimestamp || col.tp == TiDB::TypeTime)); + } + if (!has_need_cast_column) + return std::nullopt; + + ExpressionActionsChain chain; + // execute timezone cast or duration cast if needed for local table scan + if (analyzer.appendExtraCastsAfterTS(chain, table_scan_columns)) + { + ExpressionActionsPtr extra_cast = chain.getLastActions(); + assert(extra_cast); + chain.finalize(); + chain.clear(); + return extra_cast; + } + else + { + return std::nullopt; + } +} + void injectFailPointForLocalRead([[maybe_unused]] const SelectQueryInfo & query_info) { // Inject failpoint to throw RegionException for testing diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 802a78ec40f..83563c47338 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -726,13 +726,6 @@ std::unordered_map getFuncNameToSigMap() } } // namespace -tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info) -{ - return dag_timezone_info.is_name_based - ? constructStringLiteralTiExpr(dag_timezone_info.timezone_name) - : constructInt64LiteralTiExpr(dag_timezone_info.timezone_offset); -} - bool isScalarFunctionExpr(const tipb::Expr & expr) { return expr.tp() == tipb::ExprType::ScalarFunc; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 48767b5e599..1fe665e9c89 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -31,7 +31,6 @@ namespace DB { class DAGContext; -tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info); bool isLiteralExpr(const tipb::Expr & expr); Field decodeLiteral(const tipb::Expr & expr); bool isFunctionExpr(const tipb::Expr & expr); diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 1f0b7e6e38b..d25eed2e2f5 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -175,35 +175,6 @@ void executeCreatingSets( } } -// add timezone cast for timestamp type, this is used to support session level timezone -std::optional addExtraCastsAfterTs( - DAGExpressionAnalyzer & analyzer, - const DB::ColumnInfos & table_scan_columns) -{ - bool has_need_cast_column = false; - for (const auto & col : table_scan_columns) - { - has_need_cast_column |= (col.id != -1 && (col.tp == TiDB::TypeTimestamp || col.tp == TiDB::TypeTime)); - } - if (!has_need_cast_column) - return std::nullopt; - - ExpressionActionsChain chain; - // execute timezone cast or duration cast if needed for local table scan - if (analyzer.appendExtraCastsAfterTS(chain, table_scan_columns)) - { - ExpressionActionsPtr extra_cast = chain.getLastActions(); - assert(extra_cast); - chain.finalize(); - chain.clear(); - return extra_cast; - } - else - { - return std::nullopt; - } -} - std::tuple buildPushDownFilter( const google::protobuf::RepeatedPtrField & conditions, DAGExpressionAnalyzer & analyzer) diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index eb3e66c77bb..1d13e50319f 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -68,11 +68,6 @@ void executeCreatingSets( size_t max_streams, const LoggerPtr & log); -// add timezone cast for timestamp type, this is used to support session level timezone -std::optional addExtraCastsAfterTs( - DAGExpressionAnalyzer & analyzer, - const DB::ColumnInfos & table_scan_columns); - std::tuple buildPushDownFilter( const google::protobuf::RepeatedPtrField & conditions, DAGExpressionAnalyzer & analyzer); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp index b3af265de95..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 { @@ -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 f08abb624ad..7375cc7328e 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -764,6 +764,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); @@ -786,6 +787,27 @@ std::vector SegmentTestBasic::readSegment(PageIdU64 segment_id, bool need return blks; } +std::vector SegmentTestBasic::lateMaterializationReadSegment(PageIdU64 segment_id, const RowKeyRanges & ranges, const PushDownFilterPtr & filter) +{ + auto [segment, snapshot] = getSegmentForRead(segment_id); + ColumnDefines columns_to_read = {getExtraHandleColumnDefine(options.is_common_handle), + getVersionColumnDefine()}; + auto stream = segment->getBitmapFilterInputStream( + *dm_context, + columns_to_read, + snapshot, + ranges.empty() ? RowKeyRanges{segment->getRowKeyRange()} : ranges, + filter, + std::numeric_limits::max(), + DEFAULT_BLOCK_SIZE); + std::vector blks; + for (auto blk = stream->read(); blk; blk = stream->read()) + { + blks.push_back(blk); + } + return blks; +} + ColumnPtr SegmentTestBasic::getSegmentRowId(PageIdU64 segment_id, const RowKeyRanges & ranges) { LOG_INFO(logger_op, "getSegmentRowId, segment_id={}", segment_id); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h index 4608c9e9a70..ae641220ea3 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h @@ -96,6 +96,7 @@ class SegmentTestBasic : public DB::base::TiFlashStorageTestBasic void printFinishedOperations() const; std::vector readSegment(PageIdU64 segment_id, bool need_row_id, const RowKeyRanges & ranges); + std::vector lateMaterializationReadSegment(PageIdU64 segment_id, const RowKeyRanges & ranges, const PushDownFilterPtr & filter); ColumnPtr getSegmentRowId(PageIdU64 segment_id, const RowKeyRanges & ranges); ColumnPtr getSegmentHandle(PageIdU64 segment_id, const RowKeyRanges & ranges); void writeSegmentWithDeleteRange(PageIdU64 segment_id, Int64 begin, Int64 end); 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..742c04388c2 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp @@ -0,0 +1,155 @@ +// Copyright 2022 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 "gtest_segment_util.h" + +#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 \ No newline at end of file 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..70146487b3a --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h @@ -0,0 +1,56 @@ +// Copyright 2022 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 \ No newline at end of file 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 4e03542a42d..b9654be438f 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: From a02039dbd61d2887ef408cf8720a1869b1d11cfe Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 7 Mar 2023 16:43:53 +0800 Subject: [PATCH 03/25] fix rs operator Signed-off-by: Lloyd-Pottiger --- .../DeltaMerge/FilterParser/FilterParser.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index 95798aebd37..fad9dba470b 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -393,22 +393,32 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, const LoggerPtr & log) { RSOperatorPtr op = EMPTY_RS_OPERATOR; - if (dag_info.filters.empty()) + 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; + children.reserve(dag_info.filters.size() + dag_info.pushed_down_filters.size()); for (int i = 0; i < dag_info.filters.size(); ++i) { const auto & filter = dag_info.filters[i]; children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); } + for (int i = 0; i < dag_info.pushed_down_filters.size(); ++i) + { + const auto & filter = dag_info.pushed_down_filters[i]; + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); + } op = createAnd(children); } return op; From e614c754278fd46d4872bb76666ef7f7a0cbd968 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 8 Mar 2023 11:21:45 +0800 Subject: [PATCH 04/25] fix Signed-off-by: Lloyd-Pottiger --- dbms/src/Core/Block.cpp | 6 ++++-- dbms/src/Core/Block.h | 2 ++ .../DeltaMerge/FilterParser/FilterParser.cpp | 18 ++++++++++++++++++ .../DeltaMerge/FilterParser/FilterParser.h | 6 ++---- .../LateMaterializationBlockInputStream.cpp | 4 ++-- dbms/src/Storages/StorageDeltaMerge.cpp | 9 +++++---- 6 files changed, 33 insertions(+), 12 deletions(-) diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 62174b84eb7..b8ca70d92af 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); + } } } diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 73184ab48e5..5e58fe2c125 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -173,6 +173,8 @@ using BlocksList = std::list; using BucketBlocksListMap = std::map; /// 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. Block hstackBlocks(Blocks && blocks, const Block & header); /// join blocks by rows diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index fad9dba470b..36cf6f7c265 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -424,6 +424,24 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, return op; } +void FilterParser::parseFilterColumnsFromDAGQuery(const tipb::Expr & expr, const ColumnDefines & columns_to_read, std::unordered_set & col_id_set) +{ + if (expr.children_size() == 0) + { + if (likely(isColumnExpr(expr))) + { + col_id_set.insert(cop::getColumnIDForColumnExpr(expr, columns_to_read)); + } + } + else + { + for (const auto & child : expr.children()) + { + parseFilterColumnsFromDAGQuery(child, columns_to_read, col_id_set); + } + } +} + std::unordered_map FilterParser::scalar_func_rs_filter_map{ /* {tipb::ScalarFuncSig::CastIntAsInt, "cast"}, diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h index eeabe53821d..657382f4a74 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h @@ -36,10 +36,6 @@ struct DAGQueryInfo; namespace DM { -namespace cop -{ -ColumnID getColumnIDForColumnExpr(const tipb::Expr & expr, const ColumnDefines & columns_to_read); -} // namespace cop class RSOperator; using RSOperatorPtr = std::shared_ptr; @@ -83,6 +79,8 @@ class FilterParser }; static std::unordered_map scalar_func_rs_filter_map; + + static void parseFilterColumnsFromDAGQuery(const tipb::Expr & expr, const ColumnDefines & columns_to_read, std::unordered_set & col_id_set); }; } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp index 0d7f3b1f0d8..a0b2ae32a5d 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp @@ -102,11 +102,11 @@ 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) { col.column = col.column->filter(*filter, passed_count); } diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 65d9843230c..312a84d3997 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -737,14 +737,15 @@ DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryIn LOG_DEBUG(tracing_logger, "Rough set filter is disabled."); // build push down filter - if (!query_info.dag_query->pushed_down_filters.empty()) + if (likely(query_info.dag_query) && !query_info.dag_query->pushed_down_filters.empty()) { std::unordered_set filter_column_ids; - for (const auto & filter : query_info.dag_query->filters) + for (const auto & filter : query_info.dag_query->pushed_down_filters) { - filter_column_ids.insert(DB::DM::cop::getColumnIDForColumnExpr(filter, columns_to_read)); + FilterParser::parseFilterColumnsFromDAGQuery(filter, columns_to_read, filter_column_ids); } ColumnDefines filter_columns; + filter_columns.reserve(filter_column_ids.size()); for (const auto id : filter_column_ids) { auto iter = std::find_if( @@ -758,7 +759,7 @@ DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryIn for (const auto & col : filter_columns) { // do not support push down filter on datetime and time - RUNTIME_CHECK(col.id == 1 || (col.type->getTypeId() != TypeIndex::MyDateTime && col.type->getTypeId() != TypeIndex::MyTime)); + RUNTIME_CHECK(col.id == -1 || (col.type->getTypeId() != TypeIndex::MyDateTime && col.type->getTypeId() != TypeIndex::MyTime)); } NamesAndTypes columns_to_read_name_and_type; From 7783c961db268e0ed0f546dd19a0ec7a317f813b Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 8 Mar 2023 14:58:27 +0800 Subject: [PATCH 05/25] rewrite zone Signed-off-by: Lloyd-Pottiger --- dbms/src/Debug/MockStorage.cpp | 6 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGQueryInfo.h | 14 ++--- .../Coprocessor/DAGStorageInterpreter.cpp | 5 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 57 +++++++++++++++++++ dbms/src/Flash/Coprocessor/DAGUtils.h | 7 +++ .../Flash/Coprocessor/InterpreterUtils.cpp | 29 ++++++++++ dbms/src/Flash/Coprocessor/InterpreterUtils.h | 10 ++++ dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 7 ++- dbms/src/Flash/Coprocessor/TiDBTableScan.h | 2 + dbms/src/Flash/Planner/PhysicalPlan.cpp | 2 +- .../tests/gtest_storage_disaggregated.cpp | 2 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 4 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 2 +- .../DeltaMerge/FilterParser/FilterParser.cpp | 48 +++------------- dbms/src/Storages/StorageDeltaMerge.cpp | 4 +- .../Storages/tests/gtest_filter_parser.cpp | 18 ++---- 17 files changed, 143 insertions(+), 76 deletions(-) diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index d1979d337b8..a23614e2c40 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -153,8 +153,6 @@ std::tuple MockStorage::prepareFor column_names.push_back(column_info.first); auto scan_context = std::make_shared(); - const google::protobuf::RepeatedPtrField pushed_down_filters; - SelectQueryInfo query_info; query_info.query = std::make_shared(); query_info.keep_order = false; @@ -169,12 +167,12 @@ BlockInputStreamPtr MockStorage::getStreamFromDeltaMerge(Context & context, Int6 if (filter_conditions && filter_conditions->hasValue()) { auto analyzer = std::make_unique(names_and_types_map_for_delta_merge[table_id], context); + const google::protobuf::RepeatedPtrField pushed_down_filters; query_info.dag_query = std::make_unique( filter_conditions->conditions, pushed_down_filters, analyzer->getPreparedSets(), - analyzer->getCurrentInputColumns(), - context.getTimezoneInfo()); + analyzer->getCurrentInputColumns()); 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 diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index cdf48757383..de90231bdf0 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -620,7 +620,7 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) } else if (query_block.isTableScanSource()) { - TiDBTableScan table_scan(query_block.source, query_block.source_name, dagContext()); + TiDBTableScan table_scan(query_block.source, context.getTimezoneInfo(), query_block.source_name, dagContext()); if (unlikely(context.isTest())) handleMockTableScan(table_scan, pipeline); else diff --git a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h index e8db3b1f670..005b1b4f050 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -27,25 +27,21 @@ namespace DB struct DAGQueryInfo { DAGQueryInfo( - const google::protobuf::RepeatedPtrField & filters_, + 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_) + const NamesAndTypes & source_columns_) + : filters(std::move(filters_)) , pushed_down_filters(pushed_down_filters_) , dag_sets(std::move(dag_sets_)) - , source_columns(source_columns_) - , timezone_info(timezone_info_){}; + , source_columns(source_columns_){}; // filters in dag request - const google::protobuf::RepeatedPtrField & filters; + 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; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index c519da6100b..a3d90c9b64e 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -588,11 +588,10 @@ std::unordered_map DAGStorageInterpreter::generateSele /// to avoid null point exception query_info.query = dagContext().dummy_ast; query_info.dag_query = std::make_unique( - filter_conditions.conditions, + rewiteExprWithTimezone(context.getTimezoneInfo(), filter_conditions.conditions, table_scan.getColumns()), table_scan.getPushedDownFilters(), analyzer->getPreparedSets(), - analyzer->getCurrentInputColumns(), - context.getTimezoneInfo()); + analyzer->getCurrentInputColumns()); query_info.req_id = fmt::format("{} table_id={}", log->identifier(), table_id); query_info.keep_order = table_scan.keepOrder(); query_info.is_fast_scan = table_scan.isFastScan(); diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 83563c47338..dbc5df52653 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -986,6 +986,63 @@ String exprToString(const tipb::Expr & expr, const std::vector return fmt_buf.toString(); } +std::vector getColumnsForExpr(const tipb::Expr & expr) +{ + if (isLiteralExpr(expr)) + { + return {}; + } + else if (isColumnExpr(expr)) + { + return {decodeDAGInt64(expr.val())}; + } + else if (isScalarFunctionExpr(expr)) + { + std::vector cols; + for (const auto & child : expr.children()) + { + auto child_col = getColumnsForExpr(child); + cols.insert(cols.end(), child_col.begin(), child_col.end()); + } + return cols; + } + else + { + throw TiFlashException("Should not reach here: not a column or literal expression", Errors::Coprocessor::Internal); + } +} + +tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info) +{ + tipb::Expr ret_expr = expr; + if (expr.tp() == tipb::ExprType::MysqlTime && expr.field_type().tp() == TiDB::TypeDatetime) + { + // for example: + // when timezone is +08:00 + // 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 + static const auto & time_zone_utc = DateLUT::instance("UTC"); + UInt64 from_time = decodeDAGUInt64(expr.val()); + UInt64 result_time = from_time; + if (timezone_info.is_name_based) + convertTimeZone(from_time, result_time, *timezone_info.timezone, time_zone_utc); + else if (timezone_info.timezone_offset != 0) + convertTimeZoneByOffset(from_time, result_time, false, timezone_info.timezone_offset); + WriteBufferFromOwnString ss; + encodeDAGUInt64(result_time, ss); + ret_expr.set_val(ss.releaseStr()); + } + else if (isScalarFunctionExpr(expr)) + { + ret_expr.clear_children(); + ret_expr.mutable_children()->Reserve(expr.children().size()); + for (const auto & child : expr.children()) + { + ret_expr.mutable_children()->Add(rewriteTimeStampLiteral(child, timezone_info)); + } + } + return ret_expr; +} + const String & getTypeName(const tipb::Expr & expr) { return tipb::ExprType_Name(expr.tp()); diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 1fe665e9c89..9b7b272f626 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -47,6 +47,13 @@ String getJoinExecTypeName(const tipb::JoinExecType & tp); bool isColumnExpr(const tipb::Expr & expr); String getColumnNameForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); NameAndTypePair getColumnNameAndTypeForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); +// get column index from tipb::Expr +std::vector getColumnsForExpr(const tipb::Expr & expr); +// rewrite timestamp literal to UTC time +// for example: +// when timezone is +08:00 +// 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 +tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info); const String & getTypeName(const tipb::Expr & expr); String exprToString(const tipb::Expr & expr, const std::vector & input_col); bool exprHasValidFieldType(const tipb::Expr & expr); diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index d25eed2e2f5..59875cce1df 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -238,4 +238,33 @@ void executeGeneratedColumnPlaceholder( stream->setExtraInfo("generated column placeholder above table scan"); } } + +google::protobuf::RepeatedPtrField rewiteExprWithTimezone( + const TimezoneInfo & timezone_info, + const google::protobuf::RepeatedPtrField & conditions, + const DB::ColumnInfos & table_scan_columns) +{ + if (timezone_info.is_utc_timezone || conditions.empty()) + { + return std::move(conditions); + } + + google::protobuf::RepeatedPtrField rewrote_conditions; + for (const auto & condition : conditions) + { + const auto col_idxs = getColumnsForExpr(condition); + tipb::Expr expr = condition; + for (const auto idx : col_idxs) + { + if (!timezone_info.is_utc_timezone && table_scan_columns[idx].id != -1 && table_scan_columns[idx].tp == TiDB::TP::TypeTimestamp) + { + expr = ::DB::rewriteTimeStampLiteral(expr, timezone_info); + break; + } + } + rewrote_conditions.Add(std::move(expr)); + } + return rewrote_conditions; +} + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 1d13e50319f..9df2275c019 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -84,4 +84,14 @@ void executeGeneratedColumnPlaceholder( const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline); +// Check timezone and all condition, if timezone is not UTC +// timestamp literal will be rewrote to UTC time +// for example: +// when timezone is +08:00 +// 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 +google::protobuf::RepeatedPtrField rewiteExprWithTimezone( + const TimezoneInfo & timezone_info, + const google::protobuf::RepeatedPtrField & conditions, + const DB::ColumnInfos & table_scan_columns); + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index 68ef961c471..2b63f2fbdfd 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -13,19 +13,24 @@ // limitations under the License. #include +#include #include namespace DB { TiDBTableScan::TiDBTableScan( const tipb::Executor * table_scan_, + const TimezoneInfo & timezone_info_, const String & executor_id_, const DAGContext & dag_context) : table_scan(table_scan_) + , timezone_info(timezone_info_) , 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())) + // Rewrite timezone expr to avoid add cast for timezone. + // And time functions will not be pushed down, so it is unnecessary to add cast before executing pushed down filters + , pushed_down_filters(is_partition_table_scan ? std::move(rewiteExprWithTimezone(timezone_info, table_scan->partition_table_scan().pushed_down_filter_conditions(), columns)) : std::move(rewiteExprWithTimezone(timezone_info, table_scan->tbl_scan().pushed_down_filter_conditions(), columns))) // 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 9ea5e16ec26..604564ac4d9 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.h +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -29,6 +29,7 @@ class TiDBTableScan public: TiDBTableScan( const tipb::Executor * table_scan_, + const TimezoneInfo & timezone_info_, const String & executor_id_, const DAGContext & dag_context); bool isPartitionTableScan() const @@ -78,6 +79,7 @@ class TiDBTableScan private: const tipb::Executor * table_scan; + const TimezoneInfo & timezone_info; String executor_id; bool is_partition_table_scan; const ColumnInfos columns; diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 45bd8d00203..3e457535469 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -97,7 +97,7 @@ void PhysicalPlan::build(const tipb::DAGRequest * dag_request) void PhysicalPlan::buildTableScan(const String & executor_id, const tipb::Executor * executor) { - TiDBTableScan table_scan(executor, executor_id, dagContext()); + TiDBTableScan table_scan(executor, context.getTimezoneInfo(), executor_id, dagContext()); if (unlikely(context.isTest())) pushBack(PhysicalMockTableScan::build(context, executor_id, log, table_scan)); else diff --git a/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp b/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp index 88d19ce866e..ba536031cad 100644 --- a/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp +++ b/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp @@ -68,7 +68,7 @@ try auto dag_context = std::make_shared(*dag_req, meta, true); auto * ori_dag_context = TiFlashTestEnv::getGlobalContext().getDAGContext(); TiFlashTestEnv::getGlobalContext().setDAGContext(dag_context.get()); - TiDBTableScan tidb_table_scan(&table_scan, table_scan.executor_id(), *dag_context); + TiDBTableScan tidb_table_scan(&table_scan, context.context.getTimezoneInfo(), table_scan.executor_id(), *dag_context); FilterConditions filter_conditions; StorageDisaggregated storage(TiFlashTestEnv::getGlobalContext(), tidb_table_scan, filter_conditions); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 617a3ad92e9..eb1f5e31121 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -48,6 +48,8 @@ #include #include +#include "Storages/DeltaMerge/Filter/PushDownFilter.h" + namespace ProfileEvents { extern const Event DMWriteBlock; @@ -1072,7 +1074,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 fb020befa83..b3bb9bdd7c0 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -340,7 +340,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/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index 36cf6f7c265..3df30120787 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -105,7 +105,6 @@ inline RSOperatorPtr parseTiCompareExpr( // const FilterParser::RSFilterType filter_type, const ColumnDefines & columns_to_read, const FilterParser::AttrCreatorByColumnID & creator, - const TimezoneInfo & timezone_info, const LoggerPtr & /*log*/) { if (unlikely(expr.children_size() != 2)) @@ -115,17 +114,10 @@ inline RSOperatorPtr parseTiCompareExpr( // false); /// Only support `column` `op` `literal` now. - Attr attr; Field value; OperandType left = OperandType::Unknown; OperandType right = OperandType::Unknown; - bool is_timestamp_column = false; - for (const auto & child : expr.children()) - { - if (isColumnExpr(child)) - is_timestamp_column = (child.field_type().tp() == TiDB::TypeTimestamp); - } for (int32_t child_idx = 0; child_idx < expr.children_size(); child_idx++) { const auto & child = expr.children(child_idx); @@ -155,28 +147,6 @@ inline RSOperatorPtr parseTiCompareExpr( // left = OperandType::Literal; else if (child_idx == 1) right = OperandType::Literal; - - if (is_timestamp_column) - { - auto literal_type = child.field_type().tp(); - if (unlikely(literal_type != TiDB::TypeTimestamp && literal_type != TiDB::TypeDatetime)) - return createUnsupported(expr.ShortDebugString(), - "Compare timestamp column with literal type(" + DB::toString(literal_type) - + ") is not supported", - false); - // convert literal value from timezone specified in cop request to UTC - if (literal_type == TiDB::TypeDatetime && !timezone_info.is_utc_timezone) - { - static const auto & time_zone_utc = DateLUT::instance("UTC"); - UInt64 from_time = value.get(); - UInt64 result_time = from_time; - if (timezone_info.is_name_based) - convertTimeZone(from_time, result_time, *timezone_info.timezone, time_zone_utc); - else if (timezone_info.timezone_offset != 0) - convertTimeZoneByOffset(from_time, result_time, false, timezone_info.timezone_offset); - value = Field(result_time); - } - } } } @@ -246,7 +216,6 @@ inline RSOperatorPtr parseTiCompareExpr( // RSOperatorPtr parseTiExpr(const tipb::Expr & expr, const ColumnDefines & columns_to_read, const FilterParser::AttrCreatorByColumnID & creator, - const TimezoneInfo & timezone_info, const LoggerPtr & log) { assert(isFunctionExpr(expr)); @@ -278,7 +247,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, { const auto & child = expr.children(0); if (likely(isFunctionExpr(child))) - op = createNot(parseTiExpr(child, columns_to_read, creator, timezone_info, log)); + op = createNot(parseTiExpr(child, columns_to_read, creator, log)); else op = createUnsupported(child.ShortDebugString(), "child of logical not is not function", false); } @@ -293,7 +262,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, { const auto & child = expr.children(i); if (likely(isFunctionExpr(child))) - children.emplace_back(parseTiExpr(child, columns_to_read, creator, timezone_info, log)); + children.emplace_back(parseTiExpr(child, columns_to_read, creator, log)); else children.emplace_back(createUnsupported(child.ShortDebugString(), "child of logical operator is not function", false)); } @@ -310,7 +279,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, case FilterParser::RSFilterType::GreaterEqual: case FilterParser::RSFilterType::Less: case FilterParser::RSFilterType::LessEqual: - op = parseTiCompareExpr(expr, filter_type, columns_to_read, creator, timezone_info, log); + op = parseTiCompareExpr(expr, filter_type, columns_to_read, creator, log); break; case FilterParser::RSFilterType::IsNull: @@ -375,11 +344,10 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, inline RSOperatorPtr tryParse(const tipb::Expr & filter, const ColumnDefines & columns_to_read, const FilterParser::AttrCreatorByColumnID & creator, - const TimezoneInfo & timezone_info, const LoggerPtr & log) { if (isFunctionExpr(filter)) - return cop::parseTiExpr(filter, columns_to_read, creator, timezone_info, log); + return cop::parseTiExpr(filter, columns_to_read, creator, log); else return createUnsupported(filter.ShortDebugString(), "child of logical and is not function", false); } @@ -398,11 +366,11 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, 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); + op = cop::tryParse(dag_info.filters[0], columns_to_read, creator, 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); + op = cop::tryParse(dag_info.pushed_down_filters[0], columns_to_read, creator, log); } else { @@ -412,12 +380,12 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, for (int i = 0; i < dag_info.filters.size(); ++i) { const auto & filter = dag_info.filters[i]; - children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); } for (int i = 0; i < dag_info.pushed_down_filters.size(); ++i) { const auto & filter = dag_info.pushed_down_filters[i]; - children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); } op = createAnd(children); } diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 4a2974bb657..92c6c0d0494 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -871,7 +871,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; @@ -883,7 +883,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, diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index c0fb423bbab..bbf852ded81 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -98,11 +99,10 @@ DM::RSOperatorPtr FilterParserTest::generateRsOperator(const String table_info_j std::tie(source_columns, std::ignore) = parseColumnsFromTableInfo(table_info); const google::protobuf::RepeatedPtrField pushed_down_filters; dag_query = std::make_unique( - conditions, + rewiteExprWithTimezone(timezone_info, conditions, table_info.columns), pushed_down_filters, DAGPreparedSets(), - source_columns, - timezone_info); + source_columns); for (const auto & column : table_info.columns) { columns_to_read.push_back(DM::ColumnDefine(column.id, column.name, getDataTypeByColumnInfo(column))); @@ -422,21 +422,17 @@ try 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; { // 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); auto rs_operator = generateRsOperator(table_info_json, String("select * from default.t_111 where col_timestamp > cast_string_datetime('") + datetime + String("')")); 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("\"}")); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); } { @@ -444,14 +440,13 @@ try 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); 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"); 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("\"}")); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); } { @@ -459,14 +454,13 @@ try auto ctx = TiFlashTestEnv::getContext(); auto & timezone_info = ctx.getTimezoneInfo(); timezone_info.resetByTimezoneOffset(28800); - convertTimeZoneByOffset(origin_time_stamp, converted_time, false, timezone_info.timezone_offset); 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"); 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("\"}")); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); } { From 27621f621a1b16cda87c468b645aecd5919fee0f Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 8 Mar 2023 15:55:35 +0800 Subject: [PATCH 06/25] tidy Signed-off-by: Lloyd-Pottiger --- dbms/src/Core/Block.cpp | 4 ++-- .../Storages/DeltaMerge/FilterParser/FilterParser.cpp | 10 ++++------ 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index b8ca70d92af..4aa9a4d4849 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -664,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/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index 3df30120787..5181269e4ac 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -377,15 +377,13 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, /// By default, multiple conditions with operator "and" RSOperators children; children.reserve(dag_info.filters.size() + dag_info.pushed_down_filters.size()); - for (int i = 0; i < dag_info.filters.size(); ++i) + for (const auto & filter : dag_info.filters) { - const auto & filter = dag_info.filters[i]; - children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); } - for (int i = 0; i < dag_info.pushed_down_filters.size(); ++i) + for (const auto & filter : dag_info.pushed_down_filters) { - const auto & filter = dag_info.pushed_down_filters[i]; - children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); } op = createAnd(children); } From 49ef1c8b23f4f62d43f53fceacde7bc2940b51d8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 8 Mar 2023 16:16:58 +0800 Subject: [PATCH 07/25] format Signed-off-by: Lloyd-Pottiger --- dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index 5181269e4ac..57d4f9e267a 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -379,11 +379,11 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, 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, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); } for (const auto & filter : dag_info.pushed_down_filters) { - children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); } op = createAnd(children); } From 8eb19f7bcccd3bc92da1b237dac1713ae8cf16b2 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 8 Mar 2023 18:59:03 +0800 Subject: [PATCH 08/25] fix Signed-off-by: Lloyd-Pottiger --- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 3 +- .../tests/gtest_segment_test_basic.cpp | 21 ---------- .../tests/gtest_segment_test_basic.h | 1 - dbms/src/Storages/StorageDeltaMerge.cpp | 42 +++++++++---------- dbms/src/Storages/StorageDeltaMerge.h | 2 +- 5 files changed, 23 insertions(+), 46 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index eb1f5e31121..5d55e271347 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -48,8 +49,6 @@ #include #include -#include "Storages/DeltaMerge/Filter/PushDownFilter.h" - namespace ProfileEvents { extern const Event DMWriteBlock; 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 365934fb740..60fe4a5fb3c 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -787,27 +787,6 @@ std::vector SegmentTestBasic::readSegment(PageIdU64 segment_id, bool need return blks; } -std::vector SegmentTestBasic::lateMaterializationReadSegment(PageIdU64 segment_id, const RowKeyRanges & ranges, const PushDownFilterPtr & filter) -{ - auto [segment, snapshot] = getSegmentForRead(segment_id); - ColumnDefines columns_to_read = {getExtraHandleColumnDefine(options.is_common_handle), - getVersionColumnDefine()}; - auto stream = segment->getBitmapFilterInputStream( - *dm_context, - columns_to_read, - snapshot, - ranges.empty() ? RowKeyRanges{segment->getRowKeyRange()} : ranges, - filter, - std::numeric_limits::max(), - DEFAULT_BLOCK_SIZE); - std::vector blks; - for (auto blk = stream->read(); blk; blk = stream->read()) - { - blks.push_back(blk); - } - return blks; -} - ColumnPtr SegmentTestBasic::getSegmentRowId(PageIdU64 segment_id, const RowKeyRanges & ranges) { LOG_INFO(logger_op, "getSegmentRowId, segment_id={}", segment_id); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h index f907e673548..d4037b195a5 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h @@ -96,7 +96,6 @@ class SegmentTestBasic : public DB::base::TiFlashStorageTestBasic void printFinishedOperations() const; std::vector readSegment(PageIdU64 segment_id, bool need_row_id, const RowKeyRanges & ranges); - std::vector lateMaterializationReadSegment(PageIdU64 segment_id, const RowKeyRanges & ranges, const PushDownFilterPtr & filter); ColumnPtr getSegmentRowId(PageIdU64 segment_id, const RowKeyRanges & ranges); ColumnPtr getSegmentHandle(PageIdU64 segment_id, const RowKeyRanges & ranges); void writeSegmentWithDeleteRange(PageIdU64 segment_id, Int64 begin, Int64 end); diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 92c6c0d0494..d00d00ef5df 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -987,7 +987,7 @@ UInt64 StorageDeltaMerge::onSyncGc(Int64 limit, const GCOptions & gc_options) { if (storeInited()) { - return store->onSyncGc(limit, gc_options); + return _store->onSyncGc(limit, gc_options); } return 0; } @@ -1076,7 +1076,7 @@ DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() { if (storeInited()) { - return store; + return _store; } return nullptr; } @@ -1310,7 +1310,7 @@ try std::lock_guard lock(store_mutex); // Avoid concurrent init store and DDL. if (storeInited()) { - store->applyAlters(commands, table_info, max_column_id_used, context); + _store->applyAlters(commands, table_info, max_column_id_used, context); } else { @@ -1349,12 +1349,12 @@ ColumnDefines StorageDeltaMerge::getStoreColumnDefines() const { if (storeInited()) { - return store->getTableColumns(); + return _store->getTableColumns(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return store->getTableColumns(); + return _store->getTableColumns(); } ColumnDefines cols; cols.emplace_back(table_column_info->handle_column_define); @@ -1394,13 +1394,13 @@ void StorageDeltaMerge::rename( } if (storeInited()) { - store->rename(new_path_to_db, new_database_name, new_table_name); + _store->rename(new_path_to_db, new_database_name, new_table_name); return; } std::lock_guard lock(store_mutex); if (storeInited()) { - store->rename(new_path_to_db, new_database_name, new_table_name); + _store->rename(new_path_to_db, new_database_name, new_table_name); } else { @@ -1413,12 +1413,12 @@ String StorageDeltaMerge::getTableName() const { if (storeInited()) { - return store->getTableName(); + return _store->getTableName(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return store->getTableName(); + return _store->getTableName(); } return table_column_info->table_name; } @@ -1427,12 +1427,12 @@ String StorageDeltaMerge::getDatabaseName() const { if (storeInited()) { - return store->getDatabaseName(); + return _store->getDatabaseName(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return store->getDatabaseName(); + return _store->getDatabaseName(); } return table_column_info->db_name; } @@ -1556,7 +1556,7 @@ BlockInputStreamPtr StorageDeltaMerge::status() StoreStats stat; if (storeInited()) { - stat = store->getStoreStats(); + stat = _store->getStoreStats(); } #define INSERT_INT(NAME) \ @@ -1662,7 +1662,7 @@ void StorageDeltaMerge::shutdownImpl() return; if (storeInited()) { - store->shutdown(); + _store->shutdown(); } } @@ -1688,12 +1688,12 @@ DataTypePtr StorageDeltaMerge::getPKTypeImpl() const { if (storeInited()) { - return store->getPKDataType(); + return _store->getPKDataType(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return store->getPKDataType(); + return _store->getPKDataType(); } return table_column_info->handle_column_define.type; } @@ -1702,12 +1702,12 @@ SortDescription StorageDeltaMerge::getPrimarySortDescription() const { if (storeInited()) { - return store->getPrimarySortDescription(); + return _store->getPrimarySortDescription(); } std::lock_guard lock(store_mutex); if (storeInited()) { - return store->getPrimarySortDescription(); + return _store->getPrimarySortDescription(); } SortDescription desc; desc.emplace_back(table_column_info->handle_column_define.name, /* direction_= */ 1, /* nulls_direction_= */ 1); @@ -1718,12 +1718,12 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore() { if (storeInited()) { - return store; + return _store; } std::lock_guard lock(store_mutex); - if (store == nullptr) + if (_store == nullptr) { - store = std::make_shared( + _store = std::make_shared( global_context, data_path_contains_database_name, table_column_info->db_name, @@ -1738,7 +1738,7 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore() table_column_info.reset(nullptr); store_inited.store(true, std::memory_order_release); } - return store; + return _store; } bool StorageDeltaMerge::initStoreIfDataDirExist() diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index b3b4bd31af4..e08d2ab9209 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -255,7 +255,7 @@ class StorageDeltaMerge std::unique_ptr table_column_info; // After create DeltaMergeStore object, it is deprecated. std::atomic store_inited; - DM::DeltaMergeStorePtr store; + DM::DeltaMergeStorePtr _store; // NOLINT(readability-identifier-naming) Strings pk_column_names; // TODO: remove it. Only use for debug from ch-client. bool is_common_handle = false; From 7f7dbbeec0285952e5974a3a2db666d973ff6a04 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Thu, 9 Mar 2023 10:38:19 +0800 Subject: [PATCH 09/25] fix Signed-off-by: Lloyd-Pottiger --- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 2 +- dbms/src/Flash/Coprocessor/InterpreterUtils.cpp | 3 ++- dbms/src/Storages/tests/gtest_filter_parser.cpp | 15 ++++++++++----- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index dbc5df52653..c122a78d56a 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1015,7 +1015,7 @@ std::vector getColumnsForExpr(const tipb::Expr & expr) tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info) { tipb::Expr ret_expr = expr; - if (expr.tp() == tipb::ExprType::MysqlTime && expr.field_type().tp() == TiDB::TypeDatetime) + if (isLiteralExpr(expr) && (expr.field_type().tp() == TiDB::TypeTimestamp || expr.field_type().tp() == TiDB::TypeDatetime)) { // for example: // when timezone is +08:00 diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 59875cce1df..895d68b2d4e 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -250,13 +250,14 @@ google::protobuf::RepeatedPtrField rewiteExprWithTimezone( } google::protobuf::RepeatedPtrField rewrote_conditions; + rewrote_conditions.Reserve(conditions.size()); for (const auto & condition : conditions) { const auto col_idxs = getColumnsForExpr(condition); tipb::Expr expr = condition; for (const auto idx : col_idxs) { - if (!timezone_info.is_utc_timezone && table_scan_columns[idx].id != -1 && table_scan_columns[idx].tp == TiDB::TP::TypeTimestamp) + if (table_scan_columns[idx].id != -1 && table_scan_columns[idx].tp == TiDB::TP::TypeTimestamp) { expr = ::DB::rewriteTimeStampLiteral(expr, timezone_info); break; diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index bbf852ded81..528c9b84140 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -74,7 +74,6 @@ TimezoneInfo FilterParserTest::default_timezone_info; DM::RSOperatorPtr FilterParserTest::generateRsOperator(const String table_info_json, const String & query, TimezoneInfo & timezone_info = default_timezone_info) { const TiDB::TableInfo table_info(table_info_json); - QueryTasks query_tasks; std::tie(query_tasks, std::ignore) = compileQuery( ctx, @@ -422,17 +421,21 @@ try 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; { // 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); - 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"); EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 4); - EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(converted_time) + String("\"}")); } { @@ -440,13 +443,14 @@ try 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); 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"); EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 4); - EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(converted_time) + String("\"}")); } { @@ -454,13 +458,14 @@ try auto ctx = TiFlashTestEnv::getContext(); auto & timezone_info = ctx.getTimezoneInfo(); timezone_info.resetByTimezoneOffset(28800); + convertTimeZoneByOffset(origin_time_stamp, converted_time, false, timezone_info.timezone_offset); 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"); EXPECT_EQ(rs_operator->getAttrs()[0].col_id, 4); - EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(origin_time_stamp) + String("\"}")); + EXPECT_EQ(rs_operator->toDebugString(), String("{\"op\":\"greater\",\"col\":\"col_timestamp\",\"value\":\"") + toString(converted_time) + String("\"}")); } { From 5199f2cdbf5e31257bb93def58aae646fa813128 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Thu, 9 Mar 2023 18:31:33 +0800 Subject: [PATCH 10/25] fix bugs Signed-off-by: Lloyd-Pottiger --- contrib/tipb | 2 +- dbms/src/DataStreams/FilterBlockInputStream.h | 2 + .../src/DataStreams/FilterTransformAction.cpp | 12 +-- dbms/src/DataStreams/FilterTransformAction.h | 3 +- .../DeltaMerge/BitmapFilter/BitmapFilter.cpp | 14 +++ .../DeltaMerge/BitmapFilter/BitmapFilter.h | 2 + .../LateMaterializationBlockInputStream.cpp | 30 ++++--- .../LateMaterializationBlockInputStream.h | 4 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 2 +- .../gtest_skippable_block_input_stream.cpp | 88 ++++++++++++++++++- 10 files changed, 135 insertions(+), 24 deletions(-) diff --git a/contrib/tipb b/contrib/tipb index e82242cc05d..c4b2c4dd451 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit e82242cc05dea24e4ed288cd53b22c4a3e5d550f +Subproject commit c4b2c4dd451da06a40fab65a1e4409e7187c936b 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/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp index eb76380fd6c..aff95c3420b 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp @@ -95,6 +95,20 @@ 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) + { + return; + } + else + { + 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/LateMaterializationBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp index a0b2ae32a5d..f4ad972864e 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,22 +94,30 @@ 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); } + // remove the filter column to make sure the rows() return the correct value. + filter_column_block.erase(filter_column_name); } else if (filter_out_count > 0) { // 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(); + RUNTIME_CHECK(rest_column_block.rows() == filter->size()); for (auto & col : rest_column_block) { col.column = col.column->filter(*filter, passed_count); } for (auto & col : filter_column_block) { + if (col.name == filter_column_name) + continue; col.column = col.column->filter(*filter, passed_count); } + filter_column_block.erase(filter_column_name); } else { @@ -119,7 +127,7 @@ 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={}", + "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(), diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h index 3897e0447c2..529e03ddbe2 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,7 @@ class LateMaterializationBlockInputStream : public IProfilingBlockInputStream private: Block header; - + 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 +59,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 721f034c2fc..e6946bcb55b 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -2583,7 +2583,7 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit segment_snap->stable->getDMFilesRows(), dm_context.tracing_id); - return std::make_shared(columns_to_read, filter_column_stream, rest_column_stream, bitmap_filter, dm_context.tracing_id); + 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) 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 b9654be438f..1e27fc758da 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 @@ -90,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{}); @@ -124,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{}); @@ -142,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; @@ -190,6 +259,7 @@ try { testSkipBlockCase("d_mem:[0, 1000)"); testReadWithFilterCase("d_mem:[0, 1000)"); + testLateMaterializationCase("d_mem:[0, 1000)"); } CATCH @@ -198,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 @@ -206,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 @@ -214,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 @@ -222,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 @@ -230,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 @@ -238,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 @@ -246,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 @@ -254,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 @@ -262,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 @@ -270,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 @@ -278,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 @@ -286,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 @@ -295,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 @@ -304,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 From a9a12046d22fac1d6027a6ecec0d9b931b66a799 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Thu, 9 Mar 2023 19:30:20 +0800 Subject: [PATCH 11/25] format Signed-off-by: Lloyd-Pottiger --- contrib/tiflash-proxy | 2 +- dbms/src/Debug/MockStorage.cpp | 2 +- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- .../DeltaMerge/tests/gtest_skippable_block_input_stream.cpp | 6 +++--- dbms/src/Storages/StorageDisaggregatedRemote.cpp | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 2d3b2b033cd..6793d377dac 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 2d3b2b033cd8b2e06894ed14a6334b248f324a3d +Subproject commit 6793d377dacdfe921c4a6cb428fe6c6bd0dc1a0e diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 089b9d5df6c..90ba48378f5 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -170,7 +170,7 @@ BlockInputStreamPtr MockStorage::getStreamFromDeltaMerge(Context & context, Int6 const google::protobuf::RepeatedPtrField pushed_down_filters; query_info.dag_query = std::make_unique( filter_conditions->conditions, - google::protobuf::RepeatedPtrField {}, // Not care now + google::protobuf::RepeatedPtrField{}, // Not care now analyzer->getPreparedSets(), analyzer->getCurrentInputColumns(), context.getTimezoneInfo()); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 0431763ac93..3c414324a0d 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1017,7 +1017,7 @@ 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->beofre_where || read_mode != ReadMode::Bitmap , "Push down filters needs bitmap"); + RUNTIME_CHECK_MSG(!filter || !filter->beofre_where || read_mode != ReadMode::Bitmap, "Push down filters needs bitmap"); auto read_task_pool = std::make_shared( physical_table_id, dm_context, 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 1e27fc758da..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 @@ -173,7 +173,7 @@ class SkippableBlockInputStreamTest : public SegmentTestBasic 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) @@ -198,8 +198,8 @@ class SkippableBlockInputStreamTest : public SegmentTestBasic col.column = col.column->filter(filter, -1); } ASSERT_BLOCK_EQ(eblk, blk); - } - else + } + else { auto blk = stream->read(); ASSERT_EQ(offset, blk.startOffset()); diff --git a/dbms/src/Storages/StorageDisaggregatedRemote.cpp b/dbms/src/Storages/StorageDisaggregatedRemote.cpp index 0391da15494..2806c35781d 100644 --- a/dbms/src/Storages/StorageDisaggregatedRemote.cpp +++ b/dbms/src/Storages/StorageDisaggregatedRemote.cpp @@ -321,7 +321,7 @@ DM::RSOperatorPtr StorageDisaggregated::buildRSOperator( auto dag_query = std::make_unique( filter_conditions.conditions, - google::protobuf::RepeatedPtrField {}, // Not care now + google::protobuf::RepeatedPtrField{}, // Not care now DAGPreparedSets{}, // Not care now NamesAndTypes{}, // Not care now db_context.getTimezoneInfo()); From 96fd74555d1367fc41bfc1f88ec65fe3790d5b9b Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Thu, 9 Mar 2023 19:37:13 +0800 Subject: [PATCH 12/25] clean Signed-off-by: Lloyd-Pottiger --- dbms/src/Debug/MockStorage.cpp | 1 - dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp | 3 +-- dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 1 - dbms/src/Storages/tests/gtest_filter_parser.cpp | 1 - 4 files changed, 1 insertion(+), 5 deletions(-) diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 90ba48378f5..9554ad2fe84 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -167,7 +167,6 @@ BlockInputStreamPtr MockStorage::getStreamFromDeltaMerge(Context & context, Int6 if (filter_conditions && filter_conditions->hasValue()) { auto analyzer = std::make_unique(names_and_types_map_for_delta_merge[table_id], context); - const google::protobuf::RepeatedPtrField pushed_down_filters; query_info.dag_query = std::make_unique( filter_conditions->conditions, google::protobuf::RepeatedPtrField{}, // Not care now diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index cd335069e2c..f6d8b188eb0 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -953,9 +953,8 @@ bool DAGExpressionAnalyzer::appendExtraCastsAfterTS( const DB::ColumnInfos & table_scan_columns) { auto & step = initAndGetLastStep(chain); - auto & actions = step.actions; - auto has_cast = buildExtraCastsAfterTS(actions, table_scan_columns); + auto has_cast = buildExtraCastsAfterTS(step.actions, table_scan_columns); for (auto & col : source_columns) step.required_output.push_back(col.name); diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index 3a48bde2880..bfd7f52d669 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -13,7 +13,6 @@ // limitations under the License. #include -#include #include namespace DB diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index 30db90f6be6..39d302ff9cb 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include From 6f0002f4392a896a5c819fb906d27c2539b162dc Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 10 Mar 2023 10:58:22 +0800 Subject: [PATCH 13/25] fix rewrite Signed-off-by: Lloyd-Pottiger --- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 62 +++++++++---------- dbms/src/Flash/Coprocessor/DAGUtils.h | 4 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 18 ++---- dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 2 - 4 files changed, 36 insertions(+), 50 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index c122a78d56a..5fa84d5c70d 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -986,36 +987,12 @@ String exprToString(const tipb::Expr & expr, const std::vector return fmt_buf.toString(); } -std::vector getColumnsForExpr(const tipb::Expr & expr) -{ - if (isLiteralExpr(expr)) - { - return {}; - } - else if (isColumnExpr(expr)) - { - return {decodeDAGInt64(expr.val())}; - } - else if (isScalarFunctionExpr(expr)) - { - std::vector cols; - for (const auto & child : expr.children()) - { - auto child_col = getColumnsForExpr(child); - cols.insert(cols.end(), child_col.begin(), child_col.end()); - } - return cols; - } - else - { - throw TiFlashException("Should not reach here: not a column or literal expression", Errors::Coprocessor::Internal); - } -} - -tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info) +tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info, const NamesAndTypes & table_scan_columns, bool is_child) { tipb::Expr ret_expr = expr; - if (isLiteralExpr(expr) && (expr.field_type().tp() == TiDB::TypeTimestamp || expr.field_type().tp() == TiDB::TypeDatetime)) + // is_child means the expr is a child of a scalar function + // to avoid rewrite a dependent literal like `where '2019-01-01 00:00:00'` + if (is_child && isLiteralExpr(expr) && (expr.field_type().tp() == TiDB::TypeTimestamp || expr.field_type().tp() == TiDB::TypeDatetime)) { // for example: // when timezone is +08:00 @@ -1033,11 +1010,34 @@ tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & } else if (isScalarFunctionExpr(expr)) { - ret_expr.clear_children(); - ret_expr.mutable_children()->Reserve(expr.children().size()); + if (expr.sig() == tipb::ScalarFuncSig::LogicalAnd || expr.sig() == tipb::ScalarFuncSig::LogicalOr) + { + ret_expr.clear_children(); + ret_expr.mutable_children()->Reserve(expr.children().size()); + for (const auto & child : expr.children()) + { + ret_expr.mutable_children()->Add(rewriteTimeStampLiteral(child, timezone_info, table_scan_columns)); + } + return ret_expr; + } + bool is_timestamp_column = false; for (const auto & child : expr.children()) { - ret_expr.mutable_children()->Add(rewriteTimeStampLiteral(child, timezone_info)); + if (isColumnExpr(child)) + { + is_timestamp_column = (child.field_type().tp() == TiDB::TypeTimestamp); + is_timestamp_column &= getColumnNameForColumnExpr(child, table_scan_columns) != EXTRA_HANDLE_COLUMN_NAME; + break; + } + } + if (is_timestamp_column) + { + ret_expr.clear_children(); + ret_expr.mutable_children()->Reserve(expr.children().size()); + for (const auto & child : expr.children()) + { + ret_expr.mutable_children()->Add(rewriteTimeStampLiteral(child, timezone_info, table_scan_columns, true)); + } } } return ret_expr; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 9b7b272f626..7f7f5bb8ef1 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -47,13 +47,11 @@ String getJoinExecTypeName(const tipb::JoinExecType & tp); bool isColumnExpr(const tipb::Expr & expr); String getColumnNameForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); NameAndTypePair getColumnNameAndTypeForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); -// get column index from tipb::Expr -std::vector getColumnsForExpr(const tipb::Expr & expr); // rewrite timestamp literal to UTC time // for example: // when timezone is +08:00 // 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 -tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info); +tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info, const NamesAndTypes & table_scan_columns, bool is_child = false); const String & getTypeName(const tipb::Expr & expr); String exprToString(const tipb::Expr & expr, const std::vector & input_col); bool exprHasValidFieldType(const tipb::Expr & expr); diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 24c94367d6c..6b5ab9a8e08 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -245,26 +245,16 @@ google::protobuf::RepeatedPtrField rewiteExprWithTimezone( const google::protobuf::RepeatedPtrField & conditions, const NamesAndTypes & table_scan_columns) { - if (timezone_info.is_utc_timezone || conditions.empty()) - { + if (timezone_info.is_utc_timezone) return std::move(conditions); - } + if (conditions.empty()) + return {}; google::protobuf::RepeatedPtrField rewrote_conditions; rewrote_conditions.Reserve(conditions.size()); for (const auto & condition : conditions) { - const auto col_idxs = getColumnsForExpr(condition); - tipb::Expr expr = condition; - for (const auto idx : col_idxs) - { - if (table_scan_columns[idx].name != EXTRA_HANDLE_COLUMN_NAME && table_scan_columns[idx].type->getTypeId() == TypeIndex::MyTimeStamp) - { - expr = ::DB::rewriteTimeStampLiteral(expr, timezone_info); - break; - } - } - rewrote_conditions.Add(std::move(expr)); + rewrote_conditions.Add(DB::rewriteTimeStampLiteral(condition, timezone_info, table_scan_columns)); } return rewrote_conditions; } diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index bfd7f52d669..68ef961c471 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -25,8 +25,6 @@ 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()))) - // Rewrite timezone expr to avoid add cast for timezone. - // And time functions will not be pushed down, so it is unnecessary to add cast before executing pushed down filters , 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. From c5a093c334af4321a1c16aee78bd887e62db2ca9 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 10 Mar 2023 13:12:43 +0800 Subject: [PATCH 14/25] rebase tipb to master Signed-off-by: Lloyd-Pottiger --- contrib/tipb | 2 +- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/tipb b/contrib/tipb index c4b2c4dd451..5362260ee6f 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit c4b2c4dd451da06a40fab65a1e4409e7187c936b +Subproject commit 5362260ee6f7720ffeca48e8797ef76c31ac0f98 diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 3c414324a0d..f78c88ce6f5 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1017,7 +1017,7 @@ 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->beofre_where || read_mode != ReadMode::Bitmap, "Push down filters needs bitmap"); + RUNTIME_CHECK_MSG(!filter || !filter->beofre_where || read_mode == ReadMode::Bitmap, "Push down filters needs bitmap"); auto read_task_pool = std::make_shared( physical_table_id, dm_context, @@ -1025,7 +1025,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, From 6f6e9270b0af23b4799c2c841adf098ed645f51e Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 10 Mar 2023 13:58:16 +0800 Subject: [PATCH 15/25] fix typo Signed-off-by: Lloyd-Pottiger remove useless check Signed-off-by: Lloyd-Pottiger --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h | 6 +++--- dbms/src/Storages/DeltaMerge/Segment.cpp | 4 ++-- dbms/src/Storages/StorageDeltaMerge.cpp | 6 ------ 4 files changed, 6 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index f78c88ce6f5..cd4bbde75c6 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1017,7 +1017,7 @@ 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->beofre_where || read_mode == ReadMode::Bitmap, "Push down filters needs bitmap"); + 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, diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index 89e87bb15e8..d0bf82e5adb 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -33,19 +33,19 @@ class PushDownFilter : public std::enable_shared_from_this const ColumnDefines & filter_columns_, const String filter_column_name_) : rs_operator(rs_operator_) - , beofre_where(beofre_where_) + , before_where(beofre_where_) , filter_columns(std::move(filter_columns_)) , filter_column_name(std::move(filter_column_name_)) {} explicit PushDownFilter(const RSOperatorPtr & rs_operator_) : rs_operator(rs_operator_) - , beofre_where(nullptr) + , before_where(nullptr) , filter_columns({}) {} RSOperatorPtr rs_operator; - ExpressionActionsPtr beofre_where; + ExpressionActionsPtr before_where; ColumnDefines filter_columns; String filter_column_name; }; diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index ad45108842a..202be2239a2 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -2568,7 +2568,7 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit segment_snap->stable->getDMFilesRows(), dm_context.tracing_id); - filter_column_stream = std::make_shared(filter_column_stream, filter->beofre_where, filter->filter_column_name, dm_context.tracing_id); + 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()) { @@ -2645,7 +2645,7 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream(const DMContext & dm_con max_version, expected_block_size); - if (filter && filter->beofre_where) + if (filter && filter->before_where) { // if has filter conditions pushed down, use late materialization return getLateMaterializationStream( diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index d00d00ef5df..ba0a7dac6bb 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -756,12 +756,6 @@ DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryIn filter_columns.push_back(*iter); } - for (const auto & col : filter_columns) - { - // do not support push down filter on datetime and time - RUNTIME_CHECK(col.id == -1 || (col.type->getTypeId() != TypeIndex::MyDateTime && col.type->getTypeId() != TypeIndex::MyTime)); - } - NamesAndTypes columns_to_read_name_and_type; for (const auto & col : columns_to_read) { From f2abda35ea6a25cdb676120cbd6ed3f8e24210e0 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 10 Mar 2023 17:07:28 +0800 Subject: [PATCH 16/25] add more comments Signed-off-by: Lloyd-Pottiger --- dbms/src/Core/Block.h | 13 +++++++++++-- dbms/src/Flash/Coprocessor/DAGQueryInfo.h | 4 ++-- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 3 +++ dbms/src/Flash/Coprocessor/InterpreterUtils.cpp | 2 +- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 4 ++-- .../src/Storages/DeltaMerge/Filter/PushDownFilter.h | 8 ++++++-- .../LateMaterializationBlockInputStream.cpp | 5 +---- .../LateMaterializationBlockInputStream.h | 2 ++ dbms/src/Storages/DeltaMerge/Segment.cpp | 7 +++++++ dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h | 1 - 10 files changed, 35 insertions(+), 14 deletions(-) diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 5e58fe2c125..ff7bea3cb91 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -172,12 +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/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h index 2202d9286cc..1c5864e9596 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -34,8 +34,8 @@ struct DAGQueryInfo const NamesAndTypes & source_columns_, const TimezoneInfo & timezone_info) : source_columns(source_columns_) - , filters(rewiteExprWithTimezone(timezone_info, filters_, source_columns)) - , pushed_down_filters(rewiteExprWithTimezone(timezone_info, pushed_down_filters_, source_columns)) + , filters(rewiteExprsWithTimezone(timezone_info, filters_, source_columns)) + , pushed_down_filters(rewiteExprsWithTimezone(timezone_info, pushed_down_filters_, source_columns)) , dag_sets(std::move(dag_sets_)){}; const NamesAndTypes & source_columns; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 5fa84d5c70d..05682b4fd97 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1020,6 +1020,8 @@ tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & } return ret_expr; } + // If the column is timestamp type, and not handle column + // we should rewrite the literal to UTC time bool is_timestamp_column = false; for (const auto & child : expr.children()) { @@ -1040,6 +1042,7 @@ tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & } } } + // ignore other type of expr return ret_expr; } diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 6b5ab9a8e08..0c9a6a7efbf 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -240,7 +240,7 @@ void executeGeneratedColumnPlaceholder( } } -google::protobuf::RepeatedPtrField rewiteExprWithTimezone( +google::protobuf::RepeatedPtrField rewiteExprsWithTimezone( const TimezoneInfo & timezone_info, const google::protobuf::RepeatedPtrField & conditions, const NamesAndTypes & table_scan_columns) diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 4dc9cf48845..4eafe8b41e2 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -84,12 +84,12 @@ void executeGeneratedColumnPlaceholder( const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline); -// Check timezone and all condition, if timezone is not UTC +// Check timezone and all filter conditions, if timezone is not UTC // timestamp literal will be rewrote to UTC time // for example: // when timezone is +08:00 // 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 -google::protobuf::RepeatedPtrField rewiteExprWithTimezone( +google::protobuf::RepeatedPtrField rewiteExprsWithTimezone( const TimezoneInfo & timezone_info, const google::protobuf::RepeatedPtrField & conditions, const NamesAndTypes & table_scan_columns); diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index d0bf82e5adb..0c15f0728af 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -34,8 +34,8 @@ class PushDownFilter : public std::enable_shared_from_this const String filter_column_name_) : rs_operator(rs_operator_) , before_where(beofre_where_) - , filter_columns(std::move(filter_columns_)) , filter_column_name(std::move(filter_column_name_)) + , filter_columns(std::move(filter_columns_)) {} explicit PushDownFilter(const RSOperatorPtr & rs_operator_) @@ -44,10 +44,14 @@ class PushDownFilter : public std::enable_shared_from_this , filter_columns({}) {} + // Rough set operator RSOperatorPtr rs_operator; + // Filter expression actions and the name of the tmp filter column + // Used construct the FilterBlockInputStream ExpressionActionsPtr before_where; - ColumnDefines filter_columns; String filter_column_name; + // The columns needed by the filter expression + ColumnDefines filter_columns; }; } // namespace DB::DM \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp index f4ad972864e..1941bf85df0 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp @@ -98,8 +98,6 @@ Block LateMaterializationBlockInputStream::readImpl() continue; col.column = col.column->filter(*filter, passed_count); } - // remove the filter column to make sure the rows() return the correct value. - filter_column_block.erase(filter_column_name); } else if (filter_out_count > 0) { @@ -117,7 +115,6 @@ Block LateMaterializationBlockInputStream::readImpl() continue; col.column = col.column->filter(*filter, passed_count); } - filter_column_block.erase(filter_column_name); } else { @@ -126,7 +123,7 @@ 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(), + 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(), diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h index 529e03ddbe2..8a884ca63a1 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.h @@ -50,6 +50,8 @@ 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; diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 202be2239a2..42f278e1358 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -2499,9 +2499,11 @@ 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); + SkippableBlockInputStreamPtr stable_stream = segment_snap->stable->getInputStream( dm_context, columns_to_read, @@ -2538,10 +2540,12 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit 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, @@ -2568,6 +2572,7 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit segment_snap->stable->getDMFilesRows(), dm_context.tracing_id); + // 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()) @@ -2584,6 +2589,7 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit 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, @@ -2609,6 +2615,7 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit 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); } diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h index 00a859de19c..2bdc816cad0 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include From 45aebc32b0e9443990bf10bccf61b6136345dd2b Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 10 Mar 2023 21:35:00 +0800 Subject: [PATCH 17/25] do some optimization Signed-off-by: Lloyd-Pottiger --- .../DeltaMerge/BitmapFilter/BitmapFilter.cpp | 6 +-- .../Storages/DeltaMerge/File/DMFileReader.cpp | 51 +++++++++++++------ .../DeltaMerge/Filter/PushDownFilter.h | 3 +- .../LateMaterializationBlockInputStream.cpp | 1 - 4 files changed, 38 insertions(+), 23 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp index aff95c3420b..e578310ec72 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.cpp @@ -99,11 +99,7 @@ void BitmapFilter::rangeAnd(IColumn::Filter & f, UInt32 start, UInt32 limit) con { RUNTIME_CHECK(start + limit <= filter.size() && f.size() == limit); auto begin = filter.cbegin() + start; - if (all_match) - { - return; - } - else + if (!all_match) { std::transform(f.begin(), f.end(), begin, f.begin(), [](const UInt8 a, const bool b) { return a != 0 && b; }); } diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index 4ac17be5312..aadd36eb66b 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 6, 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 index 0c15f0728af..669c8a29741 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -15,8 +15,7 @@ #pragma once #include - -#include "RSOperator.h" +#include namespace DB::DM { diff --git a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp index 1941bf85df0..270b92ca15f 100644 --- a/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/LateMaterializationBlockInputStream.cpp @@ -104,7 +104,6 @@ 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(); - RUNTIME_CHECK(rest_column_block.rows() == filter->size()); for (auto & col : rest_column_block) { col.column = col.column->filter(*filter, passed_count); From 1fbf0f605dd440b6d055fe94403067523bbe781d Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 13 Mar 2023 18:37:10 +0800 Subject: [PATCH 18/25] add ut for ParsePushDownFilter Signed-off-by: Lloyd-Pottiger --- .../tests/gtests_parse_push_down_filter.cpp | 769 ++++++++++++++++++ 1 file changed, 769 insertions(+) create mode 100644 dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp 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..f62c222f44e --- /dev/null +++ b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp @@ -0,0 +1,769 @@ +// Copyright 2022 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 + + +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(); + Context 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); + 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, {}, "", 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()) + { + std::unordered_set filter_column_ids; + for (const auto & filter : dag_query->pushed_down_filters) + { + DM::FilterParser::parseFilterColumnsFromDAGQuery(filter, columns_to_read, filter_column_ids); + } + DM::ColumnDefines filter_columns; + filter_columns.reserve(filter_column_ids.size()); + for (const auto id : filter_column_ids) + { + auto iter = std::find_if( + columns_to_read.begin(), + columns_to_read.end(), + [id](const DM::ColumnDefine & d) -> bool { return d.id == id; }); + RUNTIME_CHECK(iter != columns_to_read.end()); + filter_columns.push_back(*iter); + } + + 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); + } + + // build filter expression actions + std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, ctx); + 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); + } + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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})}}; + 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 = "2021-10-26 17:00:00.00000"; + 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; + + { + // 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: 1849559496301477888 + + 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, 1849559496301477889, 1849559496301977881, 1849259496301477888, 1849559493301477888, 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})}}; + 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); + } + + { + // 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: 1849560389654675456 + + 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})}}; + 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 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: 1849559496301477888 + + 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})}}; + 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); + } + + { + // origin_time_stamp: 1849560046057291776 + // 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})}}; + 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); + } + + { + // origin_time_stamp: 1849560046057291776 + // 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})}}; + 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); + } +} +CATCH + +} // namespace DB::DM::tests From e7250226637afb25eb993ec9eb2281fabc9b37cb Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 14 Mar 2023 10:58:18 +0800 Subject: [PATCH 19/25] format Signed-off-by: Lloyd-Pottiger --- dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp index 2ecdc260aed..7e460a5ced9 100644 --- a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp +++ b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp @@ -766,4 +766,4 @@ try } CATCH -} // namespace DB::DM::tests +} // namespace DB::tests From e28c2374c349db0c551ef804db0aeab6512a200c Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 14 Mar 2023 19:55:25 +0800 Subject: [PATCH 20/25] extra cast Signed-off-by: Lloyd-Pottiger --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 42 +- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 14 +- dbms/src/Flash/Coprocessor/DAGQueryInfo.h | 15 +- .../Coprocessor/DAGStorageInterpreter.cpp | 60 +- .../Flash/Coprocessor/DAGStorageInterpreter.h | 5 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 80 +- dbms/src/Flash/Coprocessor/DAGUtils.h | 8 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 19 - dbms/src/Flash/Coprocessor/InterpreterUtils.h | 9 - .../DeltaMerge/Filter/PushDownFilter.h | 6 +- .../DeltaMerge/FilterParser/FilterParser.cpp | 68 +- .../DeltaMerge/FilterParser/FilterParser.h | 2 - dbms/src/Storages/DeltaMerge/Segment.cpp | 12 +- dbms/src/Storages/DeltaMerge/Segment.h | 4 +- dbms/src/Storages/StorageDeltaMerge.cpp | 81 +- dbms/src/Storages/StorageDeltaMerge.h | 2 +- .../tests/gtests_parse_push_down_filter.cpp | 1538 ++++++++--------- 17 files changed, 1007 insertions(+), 958 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index f6d8b188eb0..4d88a34aafd 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -898,12 +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); @@ -914,22 +916,16 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( // For Duration String fsp_col; static const String dur_func_name = "FunctionConvertDurationFromNanos"; - - // After apply cast, some columns' name will be changed, and new column will be added. - // We add a projection to keep the original column names. - NamesWithAliases project_to_origin_names; - for (size_t i = 0; i < table_scan_columns.size(); ++i) + for (size_t i = 0; i < need_cast_column.size(); ++i) { - const auto & col = table_scan_columns[i]; String casted_name = source_columns[i].name; - - if (!context.getTimezoneInfo().is_utc_timezone && col.id != -1 && col.tp == TiDB::TypeTimestamp) + if (!context.getTimezoneInfo().is_utc_timezone && need_cast_column[i] == ExtraCastAfterTSMode::AppendTimeZoneCast) { casted_name = appendTimeZoneCast(tz_col, source_columns[i].name, timezone_func_name, actions); has_cast = true; } - if (col.id != -1 && col.tp == TiDB::TypeTime) + if (need_cast_column[i] == ExtraCastAfterTSMode::AppendDurationCast) { if (table_scan_columns[i].decimal > 6) throw Exception("fsp must <= 6", ErrorCodes::LOGICAL_ERROR); @@ -941,25 +937,35 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( has_cast = true; } - project_to_origin_names.emplace_back(casted_name, source_columns[i].name); + casted_columns.emplace_back(std::move(casted_name)); } - actions->add(ExpressionAction::project(project_to_origin_names)); - return has_cast; + return {has_cast, casted_columns}; } bool DAGExpressionAnalyzer::appendExtraCastsAfterTS( ExpressionActionsChain & chain, - const DB::ColumnInfos & table_scan_columns) + const std::vector & need_cast_column, + const TiDBTableScan & table_scan) { auto & step = initAndGetLastStep(chain); + auto & actions = step.actions; + + auto [has_cast, casted_columns] = buildExtraCastsAfterTS(actions, need_cast_column, table_scan.getColumns()); + + if (!has_cast) + return false; - auto has_cast = buildExtraCastsAfterTS(step.actions, table_scan_columns); + // rename casted columns to original name + 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( @@ -1560,4 +1566,4 @@ String DAGExpressionAnalyzer::buildTupleFunctionForGroupConcat( return applyFunction(func_name, argument_names, actions, nullptr); } -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 9b33b841993..313fd5b681a 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -148,7 +148,8 @@ class DAGExpressionAnalyzer : private boost::noncopyable // Int64 to duration. bool appendExtraCastsAfterTS( ExpressionActionsChain & chain, - const DB::ColumnInfos & table_scan_columns); + const std::vector & need_cast_column, + const TiDBTableScan & table_scan); /// return true if some actions is needed bool appendJoinKeyAndJoinFilters( @@ -203,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 @@ -284,10 +290,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const String & expr_name, bool force_uint8); - bool buildExtraCastsAfterTS( - const ExpressionActionsPtr & actions, - 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) @@ -347,4 +349,4 @@ class DAGExpressionAnalyzer : private boost::noncopyable friend class DAGExpressionAnalyzerHelper; }; -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h index 1c5864e9596..71f10d07fd1 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -32,19 +32,22 @@ struct DAGQueryInfo const google::protobuf::RepeatedPtrField & pushed_down_filters_, DAGPreparedSets dag_sets_, const NamesAndTypes & source_columns_, - const TimezoneInfo & timezone_info) + const TimezoneInfo & timezone_info_) : source_columns(source_columns_) - , filters(rewiteExprsWithTimezone(timezone_info, filters_, source_columns)) - , pushed_down_filters(rewiteExprsWithTimezone(timezone_info, pushed_down_filters_, source_columns)) - , dag_sets(std::move(dag_sets_)){}; + , filters(filters_) + , pushed_down_filters(pushed_down_filters_) + , dag_sets(std::move(dag_sets_)) + , timezone_info(timezone_info_){}; const NamesAndTypes & source_columns; // filters in dag request - const google::protobuf::RepeatedPtrField filters; + const google::protobuf::RepeatedPtrField & filters; // filters have been push down to storage engine in dag request - const google::protobuf::RepeatedPtrField pushed_down_filters; + 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 TimezoneInfo & timezone_info; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 67977c4cd2c..39a8bbbb0af 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -178,31 +178,31 @@ bool hasRegionToRead(const DAGContext & dag_context, const TiDBTableScan & table } // add timezone cast for timestamp type, this is used to support session level timezone -std::optional addExtraCastsAfterTs( +// +std::pair addExtraCastsAfterTs( DAGExpressionAnalyzer & analyzer, - const DB::ColumnInfos & table_scan_columns) + const std::vector & need_cast_column, + const TiDBTableScan & table_scan) { bool has_need_cast_column = false; - for (const auto & col : table_scan_columns) - { - has_need_cast_column |= (col.id != -1 && (col.tp == TiDB::TypeTimestamp || col.tp == TiDB::TypeTime)); - } + for (auto b : need_cast_column) + has_need_cast_column |= (b != ExtraCastAfterTSMode::None); if (!has_need_cast_column) - return std::nullopt; + return {false, nullptr}; ExpressionActionsChain chain; // execute timezone cast or duration cast if needed for local table scan - if (analyzer.appendExtraCastsAfterTS(chain, table_scan_columns)) + if (analyzer.appendExtraCastsAfterTS(chain, need_cast_column, table_scan)) { ExpressionActionsPtr extra_cast = chain.getLastActions(); assert(extra_cast); chain.finalize(); chain.clear(); - return extra_cast; + return {true, extra_cast}; } else { - return std::nullopt; + return {false, nullptr}; } } @@ -386,7 +386,7 @@ void DAGStorageInterpreter::prepare() assert(storages_with_structure_lock.find(logical_table_id) != storages_with_structure_lock.end()); storage_for_logical_table = storages_with_structure_lock[logical_table_id].storage; - std::tie(required_columns, source_columns) = getColumnsForTableScan(); + std::tie(required_columns, source_columns, is_need_add_cast_column) = getColumnsForTableScan(); analyzer = std::make_unique(std::move(source_columns), context); } @@ -396,8 +396,8 @@ void DAGStorageInterpreter::executeCastAfterTableScan( DAGPipeline & pipeline) { // execute timezone cast or duration cast if needed for local table scan - auto extra_cast = ::DB::addExtraCastsAfterTs(*analyzer, table_scan.getColumns()); - if (extra_cast.has_value()) + 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()); size_t i = 0; @@ -405,7 +405,7 @@ void DAGStorageInterpreter::executeCastAfterTableScan( while (i < remote_read_streams_start_index) { auto & stream = pipeline.streams[i++]; - stream = std::make_shared(stream, extra_cast.value(), log->identifier()); + stream = std::make_shared(stream, extra_cast, log->identifier()); stream->setExtraInfo("cast after local tableScan"); } } @@ -979,10 +979,14 @@ std::unordered_map DAG return storages_with_lock; } -std::pair DAGStorageInterpreter::getColumnsForTableScan() +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; if (auto pk_handle_col = storage_for_logical_table->getTableInfo().getPKHandleColumn()) handle_column_name = pk_handle_col->get().name; @@ -1022,7 +1026,29 @@ std::pair DAGStorageInterpreter::getColumnsForTableScan() required_columns_tmp.emplace_back(std::move(name)); } - return {required_columns_tmp, source_columns_tmp}; + 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}; } // Build remote requests from `region_retry_from_local_region` and `table_regions_info.remote_regions` @@ -1077,4 +1103,4 @@ TableLockHolders DAGStorageInterpreter::releaseAlterLocks() return drop_locks; } -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index 8292d79e484..17d471dbfd6 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -82,7 +82,7 @@ class DAGStorageInterpreter std::unordered_map getAndLockStorages(Int64 query_schema_version); - std::pair getColumnsForTableScan(); + std::tuple> getColumnsForTableScan(); std::vector buildRemoteRequests(const DM::ScanContextPtr & scan_context); @@ -106,6 +106,7 @@ class DAGStorageInterpreter void executeImpl(DAGPipeline & pipeline); private: + std::vector is_need_add_cast_column; /// it shouldn't be hash map because duplicated region id may occur if merge regions to retry of dag. RegionRetryList region_retry_from_local_region; @@ -140,4 +141,4 @@ class DAGStorageInterpreter std::vector> generated_column_infos; }; -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 05682b4fd97..ad4529c950f 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include @@ -987,65 +986,6 @@ String exprToString(const tipb::Expr & expr, const std::vector return fmt_buf.toString(); } -tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info, const NamesAndTypes & table_scan_columns, bool is_child) -{ - tipb::Expr ret_expr = expr; - // is_child means the expr is a child of a scalar function - // to avoid rewrite a dependent literal like `where '2019-01-01 00:00:00'` - if (is_child && isLiteralExpr(expr) && (expr.field_type().tp() == TiDB::TypeTimestamp || expr.field_type().tp() == TiDB::TypeDatetime)) - { - // for example: - // when timezone is +08:00 - // 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 - static const auto & time_zone_utc = DateLUT::instance("UTC"); - UInt64 from_time = decodeDAGUInt64(expr.val()); - UInt64 result_time = from_time; - if (timezone_info.is_name_based) - convertTimeZone(from_time, result_time, *timezone_info.timezone, time_zone_utc); - else if (timezone_info.timezone_offset != 0) - convertTimeZoneByOffset(from_time, result_time, false, timezone_info.timezone_offset); - WriteBufferFromOwnString ss; - encodeDAGUInt64(result_time, ss); - ret_expr.set_val(ss.releaseStr()); - } - else if (isScalarFunctionExpr(expr)) - { - if (expr.sig() == tipb::ScalarFuncSig::LogicalAnd || expr.sig() == tipb::ScalarFuncSig::LogicalOr) - { - ret_expr.clear_children(); - ret_expr.mutable_children()->Reserve(expr.children().size()); - for (const auto & child : expr.children()) - { - ret_expr.mutable_children()->Add(rewriteTimeStampLiteral(child, timezone_info, table_scan_columns)); - } - return ret_expr; - } - // If the column is timestamp type, and not handle column - // we should rewrite the literal to UTC time - bool is_timestamp_column = false; - for (const auto & child : expr.children()) - { - if (isColumnExpr(child)) - { - is_timestamp_column = (child.field_type().tp() == TiDB::TypeTimestamp); - is_timestamp_column &= getColumnNameForColumnExpr(child, table_scan_columns) != EXTRA_HANDLE_COLUMN_NAME; - break; - } - } - if (is_timestamp_column) - { - ret_expr.clear_children(); - ret_expr.mutable_children()->Reserve(expr.children().size()); - for (const auto & child : expr.children()) - { - ret_expr.mutable_children()->Add(rewriteTimeStampLiteral(child, timezone_info, table_scan_columns, true)); - } - } - } - // ignore other type of expr - return ret_expr; -} - const String & getTypeName(const tipb::Expr & expr) { return tipb::ExprType_Name(expr.tp()); @@ -1201,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()); @@ -1508,4 +1466,4 @@ tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) throw Exception(fmt::format("Unsupported function {}", name)); return func_name_sig_map[name]; } -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 7f7f5bb8ef1..b1286fea9f6 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -46,12 +46,8 @@ 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); -// rewrite timestamp literal to UTC time -// for example: -// when timezone is +08:00 -// 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 -tipb::Expr rewriteTimeStampLiteral(const tipb::Expr & expr, const TimezoneInfo & timezone_info, const NamesAndTypes & table_scan_columns, bool is_child = false); const String & getTypeName(const tipb::Expr & expr); String exprToString(const tipb::Expr & expr, const std::vector & input_col); bool exprHasValidFieldType(const tipb::Expr & expr); @@ -110,4 +106,4 @@ class UniqueNameGenerator tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 0c9a6a7efbf..8251d7c3ccc 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -240,23 +240,4 @@ void executeGeneratedColumnPlaceholder( } } -google::protobuf::RepeatedPtrField rewiteExprsWithTimezone( - const TimezoneInfo & timezone_info, - const google::protobuf::RepeatedPtrField & conditions, - const NamesAndTypes & table_scan_columns) -{ - if (timezone_info.is_utc_timezone) - return std::move(conditions); - if (conditions.empty()) - return {}; - - google::protobuf::RepeatedPtrField rewrote_conditions; - rewrote_conditions.Reserve(conditions.size()); - for (const auto & condition : conditions) - { - rewrote_conditions.Add(DB::rewriteTimeStampLiteral(condition, timezone_info, table_scan_columns)); - } - return rewrote_conditions; -} - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 4eafe8b41e2..dafc4ad02c2 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -84,14 +84,5 @@ void executeGeneratedColumnPlaceholder( const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline); -// Check timezone and all filter conditions, if timezone is not UTC -// timestamp literal will be rewrote to UTC time -// for example: -// when timezone is +08:00 -// 2019-01-01 00:00:00 +08:00 -> 2019-01-01 00:00:00 +00:00 -google::protobuf::RepeatedPtrField rewiteExprsWithTimezone( - const TimezoneInfo & timezone_info, - const google::protobuf::RepeatedPtrField & conditions, - const NamesAndTypes & table_scan_columns); } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index 669c8a29741..acb431ec359 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -30,17 +30,20 @@ class PushDownFilter : public std::enable_shared_from_this PushDownFilter(const RSOperatorPtr & rs_operator_, const ExpressionActionsPtr & beofre_where_, const ColumnDefines & filter_columns_, - const String filter_column_name_) + 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 @@ -51,6 +54,7 @@ class PushDownFilter : public std::enable_shared_from_this String filter_column_name; // The columns needed by the filter expression ColumnDefines filter_columns; + ExpressionActionsPtr extra_cast; }; } // namespace DB::DM \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index 57d4f9e267a..d4bd4e8e45e 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -105,6 +105,7 @@ inline RSOperatorPtr parseTiCompareExpr( // const FilterParser::RSFilterType filter_type, const ColumnDefines & columns_to_read, const FilterParser::AttrCreatorByColumnID & creator, + const TimezoneInfo & timezone_info, const LoggerPtr & /*log*/) { if (unlikely(expr.children_size() != 2)) @@ -114,10 +115,17 @@ inline RSOperatorPtr parseTiCompareExpr( // false); /// Only support `column` `op` `literal` now. + Attr attr; Field value; OperandType left = OperandType::Unknown; OperandType right = OperandType::Unknown; + bool is_timestamp_column = false; + for (const auto & child : expr.children()) + { + if (isColumnExpr(child)) + is_timestamp_column = (child.field_type().tp() == TiDB::TypeTimestamp); + } for (int32_t child_idx = 0; child_idx < expr.children_size(); child_idx++) { const auto & child = expr.children(child_idx); @@ -147,6 +155,28 @@ inline RSOperatorPtr parseTiCompareExpr( // left = OperandType::Literal; else if (child_idx == 1) right = OperandType::Literal; + + if (is_timestamp_column) + { + auto literal_type = child.field_type().tp(); + if (unlikely(literal_type != TiDB::TypeTimestamp && literal_type != TiDB::TypeDatetime)) + return createUnsupported(expr.ShortDebugString(), + "Compare timestamp column with literal type(" + DB::toString(literal_type) + + ") is not supported", + false); + // convert literal value from timezone specified in cop request to UTC + if (literal_type == TiDB::TypeDatetime && !timezone_info.is_utc_timezone) + { + static const auto & time_zone_utc = DateLUT::instance("UTC"); + UInt64 from_time = value.get(); + UInt64 result_time = from_time; + if (timezone_info.is_name_based) + convertTimeZone(from_time, result_time, *timezone_info.timezone, time_zone_utc); + else if (timezone_info.timezone_offset != 0) + convertTimeZoneByOffset(from_time, result_time, false, timezone_info.timezone_offset); + value = Field(result_time); + } + } } } @@ -216,6 +246,7 @@ inline RSOperatorPtr parseTiCompareExpr( // RSOperatorPtr parseTiExpr(const tipb::Expr & expr, const ColumnDefines & columns_to_read, const FilterParser::AttrCreatorByColumnID & creator, + const TimezoneInfo & timezone_info, const LoggerPtr & log) { assert(isFunctionExpr(expr)); @@ -247,7 +278,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, { const auto & child = expr.children(0); if (likely(isFunctionExpr(child))) - op = createNot(parseTiExpr(child, columns_to_read, creator, log)); + op = createNot(parseTiExpr(child, columns_to_read, creator, timezone_info, log)); else op = createUnsupported(child.ShortDebugString(), "child of logical not is not function", false); } @@ -262,7 +293,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, { const auto & child = expr.children(i); if (likely(isFunctionExpr(child))) - children.emplace_back(parseTiExpr(child, columns_to_read, creator, log)); + children.emplace_back(parseTiExpr(child, columns_to_read, creator, timezone_info, log)); else children.emplace_back(createUnsupported(child.ShortDebugString(), "child of logical operator is not function", false)); } @@ -279,7 +310,7 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, case FilterParser::RSFilterType::GreaterEqual: case FilterParser::RSFilterType::Less: case FilterParser::RSFilterType::LessEqual: - op = parseTiCompareExpr(expr, filter_type, columns_to_read, creator, log); + op = parseTiCompareExpr(expr, filter_type, columns_to_read, creator, timezone_info, log); break; case FilterParser::RSFilterType::IsNull: @@ -344,10 +375,11 @@ RSOperatorPtr parseTiExpr(const tipb::Expr & expr, inline RSOperatorPtr tryParse(const tipb::Expr & filter, const ColumnDefines & columns_to_read, const FilterParser::AttrCreatorByColumnID & creator, + const TimezoneInfo & timezone_info, const LoggerPtr & log) { if (isFunctionExpr(filter)) - return cop::parseTiExpr(filter, columns_to_read, creator, log); + return cop::parseTiExpr(filter, columns_to_read, creator, timezone_info, log); else return createUnsupported(filter.ShortDebugString(), "child of logical and is not function", false); } @@ -366,11 +398,11 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, if (dag_info.filters.size() == 1 && dag_info.pushed_down_filters.empty()) { - op = cop::tryParse(dag_info.filters[0], columns_to_read, creator, log); + 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, log); + op = cop::tryParse(dag_info.pushed_down_filters[0], columns_to_read, creator, dag_info.timezone_info, log); } else { @@ -379,35 +411,17 @@ RSOperatorPtr FilterParser::parseDAGQuery(const DAGQueryInfo & dag_info, 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, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); } for (const auto & filter : dag_info.pushed_down_filters) { - children.emplace_back(cop::tryParse(filter, columns_to_read, creator, log)); + children.emplace_back(cop::tryParse(filter, columns_to_read, creator, dag_info.timezone_info, log)); } op = createAnd(children); } return op; } -void FilterParser::parseFilterColumnsFromDAGQuery(const tipb::Expr & expr, const ColumnDefines & columns_to_read, std::unordered_set & col_id_set) -{ - if (expr.children_size() == 0) - { - if (likely(isColumnExpr(expr))) - { - col_id_set.insert(cop::getColumnIDForColumnExpr(expr, columns_to_read)); - } - } - else - { - for (const auto & child : expr.children()) - { - parseFilterColumnsFromDAGQuery(child, columns_to_read, col_id_set); - } - } -} - std::unordered_map FilterParser::scalar_func_rs_filter_map{ /* {tipb::ScalarFuncSig::CastIntAsInt, "cast"}, @@ -970,4 +984,4 @@ std::unordered_map FilterParser } // namespace DM -} // namespace DB +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h index 657382f4a74..c6a12eaaa29 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.h @@ -79,8 +79,6 @@ class FilterParser }; static std::unordered_map scalar_func_rs_filter_map; - - static void parseFilterColumnsFromDAGQuery(const tipb::Expr & expr, const ColumnDefines & columns_to_read, std::unordered_set & col_id_set); }; } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index d2b3fe2dab7..a2a18f74384 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -1840,7 +1841,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); @@ -1849,7 +1850,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)}, @@ -2572,6 +2573,13 @@ BlockInputStreamPtr Segment::getLateMaterializationStream(BitmapFilterPtr && bit 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"); diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index 5cea3834816..cd59d57943a 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -458,8 +458,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. diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 5c23465d8f3..ca34cefec84 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -60,6 +60,8 @@ #include +#include "Functions/FunctionBinaryArithmetic.h" + namespace DB { namespace FailPoints @@ -739,34 +741,93 @@ DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryIn // build push down filter if (likely(query_info.dag_query) && !query_info.dag_query->pushed_down_filters.empty()) { - std::unordered_set filter_column_ids; + 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) { - FilterParser::parseFilterColumnsFromDAGQuery(filter, columns_to_read, filter_column_ids); + DB::getColumnNamesFromExpr(filter, columns_to_read_name_and_type, filter_column_names); } ColumnDefines filter_columns; - filter_columns.reserve(filter_column_ids.size()); - for (const auto id : filter_column_ids) + 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(), - [id](const ColumnDefine & d) -> bool { return d.id == id; }); + [&name](const ColumnDefine & d) -> bool { return d.name == name; }); RUNTIME_CHECK(iter != columns_to_read.end()); filter_columns.push_back(*iter); } - NamesAndTypes columns_to_read_name_and_type; + 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) { - columns_to_read_name_and_type.emplace_back(col.name, col.type); + 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::TypeDatetime) + need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); + else + need_cast_column.push_back(ExtraCastAfterTSMode::None); + } } - // build filter expression actions 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); + return std::make_shared(rs_operator, before_where, filter_columns, filter_column_name, extra_cast); } return std::make_shared(rs_operator); } @@ -1066,7 +1127,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 e08d2ab9209..0d9a40c79b7 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -164,7 +164,7 @@ class StorageDeltaMerge return getAndMaybeInitStore(); } - DM::DeltaMergeStorePtr getStoreIfInited(); + DM::DeltaMergeStorePtr getStoreIfInited() const; bool isCommonHandle() const override { return is_common_handle; } diff --git a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp index 7e460a5ced9..1605d00da54 100644 --- a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp +++ b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp @@ -1,769 +1,769 @@ -// Copyright 2022 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 - - -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()) - { - std::unordered_set filter_column_ids; - for (const auto & filter : dag_query->pushed_down_filters) - { - DM::FilterParser::parseFilterColumnsFromDAGQuery(filter, columns_to_read, filter_column_ids); - } - DM::ColumnDefines filter_columns; - filter_columns.reserve(filter_column_ids.size()); - for (const auto id : filter_column_ids) - { - auto iter = std::find_if( - columns_to_read.begin(), - columns_to_read.end(), - [id](const DM::ColumnDefine & d) -> bool { return d.id == id; }); - RUNTIME_CHECK(iter != columns_to_read.end()); - filter_columns.push_back(*iter); - } - - 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); - } - - // build filter expression actions - std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, *ctx); - 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); - } - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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})}}; - 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 = "2021-10-26 17:00:00.00000"; - 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; - - { - // 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: 1849559496301477888 - - 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, 1849559496301477889, 1849559496301977881, 1849259496301477888, 1849559493301477888, 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})}}; - 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); - } - - { - // 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: 1849560389654675456 - - 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})}}; - 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 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: 1849559496301477888 - - 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})}}; - 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); - } - - { - // origin_time_stamp: 1849560046057291776 - // 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})}}; - 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); - } - - { - // origin_time_stamp: 1849560046057291776 - // 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})}}; - 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); - } -} -CATCH - -} // namespace DB::tests +// // Copyright 2022 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 + + +// 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()) +// { +// std::unordered_set filter_column_ids; +// for (const auto & filter : dag_query->pushed_down_filters) +// { +// DM::FilterParser::parseFilterColumnsFromDAGQuery(filter, columns_to_read, filter_column_ids); +// } +// DM::ColumnDefines filter_columns; +// filter_columns.reserve(filter_column_ids.size()); +// for (const auto id : filter_column_ids) +// { +// auto iter = std::find_if( +// columns_to_read.begin(), +// columns_to_read.end(), +// [id](const DM::ColumnDefine & d) -> bool { return d.id == id; }); +// RUNTIME_CHECK(iter != columns_to_read.end()); +// filter_columns.push_back(*iter); +// } + +// 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); +// } + +// // build filter expression actions +// std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, *ctx); +// 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); +// } +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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})}}; +// 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 = "2021-10-26 17:00:00.00000"; +// 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; + +// { +// // 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: 1849559496301477888 + +// 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, 1849559496301477889, 1849559496301977881, 1849259496301477888, 1849559493301477888, 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})}}; +// 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); +// } + +// { +// // 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: 1849560389654675456 + +// 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})}}; +// 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 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: 1849559496301477888 + +// 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})}}; +// 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); +// } + +// { +// // origin_time_stamp: 1849560046057291776 +// // 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})}}; +// 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); +// } + +// { +// // origin_time_stamp: 1849560046057291776 +// // 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})}}; +// 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); +// } +// } +// CATCH + +// } // namespace DB::tests From d7fd787045a456157a8f55e890d9702e6e02b77f Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 14 Mar 2023 20:55:59 +0800 Subject: [PATCH 21/25] fix ut Signed-off-by: Lloyd-Pottiger --- dbms/src/Storages/StorageDeltaMerge.cpp | 2 +- .../tests/gtests_parse_push_down_filter.cpp | 1624 +++++++++-------- 2 files changed, 856 insertions(+), 770 deletions(-) diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index ca34cefec84..60b508b0228 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -789,7 +789,7 @@ DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryIn { if (col.id != -1 && col.tp == TiDB::TypeTimestamp) need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); - else if (col.id != -1 && col.tp == TiDB::TypeDatetime) + else if (col.id != -1 && col.tp == TiDB::TypeTime) need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); else need_cast_column.push_back(ExtraCastAfterTSMode::None); diff --git a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp index 1605d00da54..7d360bdc4ea 100644 --- a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp +++ b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp @@ -1,769 +1,855 @@ -// // Copyright 2022 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 - - -// 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()) -// { -// std::unordered_set filter_column_ids; -// for (const auto & filter : dag_query->pushed_down_filters) -// { -// DM::FilterParser::parseFilterColumnsFromDAGQuery(filter, columns_to_read, filter_column_ids); -// } -// DM::ColumnDefines filter_columns; -// filter_columns.reserve(filter_column_ids.size()); -// for (const auto id : filter_column_ids) -// { -// auto iter = std::find_if( -// columns_to_read.begin(), -// columns_to_read.end(), -// [id](const DM::ColumnDefine & d) -> bool { return d.id == id; }); -// RUNTIME_CHECK(iter != columns_to_read.end()); -// filter_columns.push_back(*iter); -// } - -// 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); -// } - -// // build filter expression actions -// std::unique_ptr analyzer = std::make_unique(columns_to_read_name_and_type, *ctx); -// 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); -// } -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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})}}; -// 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 = "2021-10-26 17:00:00.00000"; -// 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; - -// { -// // 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: 1849559496301477888 - -// 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, 1849559496301477889, 1849559496301977881, 1849259496301477888, 1849559493301477888, 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})}}; -// 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); -// } - -// { -// // 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: 1849560389654675456 - -// 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})}}; -// 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 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: 1849559496301477888 - -// 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})}}; -// 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); -// } - -// { -// // origin_time_stamp: 1849560046057291776 -// // 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})}}; -// 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); -// } - -// { -// // origin_time_stamp: 1849560046057291776 -// // 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})}}; -// 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); -// } -// } -// CATCH - -// } // namespace DB::tests +// Copyright 2022 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 + + +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 From 9334dc02f43ef21b0fc7bc4816b83cba0f9477bc Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 15 Mar 2023 10:06:40 +0800 Subject: [PATCH 22/25] remove useless header Signed-off-by: Lloyd-Pottiger --- dbms/src/Flash/Coprocessor/DAGQueryInfo.h | 1 - dbms/src/Flash/Coprocessor/InterpreterUtils.cpp | 1 - dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h | 3 ++- dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp | 2 +- dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h | 2 +- dbms/src/Storages/tests/gtest_filter_parser.cpp | 4 ++-- dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp | 3 ++- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h index 71f10d07fd1..07fbe9f97f5 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -17,7 +17,6 @@ #include #include #include -#include #include diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 8251d7c3ccc..1b87225cc76 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -24,7 +24,6 @@ #include #include #include -#include namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index acb431ec359..71edde88ce5 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// 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. @@ -54,6 +54,7 @@ class PushDownFilter : public std::enable_shared_from_this 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; }; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp index 742c04388c2..d3a0c4245e3 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// 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. diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h index 70146487b3a..546dbfb78d7 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// 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. diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index 3e835d32b0c..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 @@ -99,7 +99,7 @@ DM::RSOperatorPtr FilterParserTest::generateRsOperator(const String table_info_j const google::protobuf::RepeatedPtrField pushed_down_filters; dag_query = std::make_unique( conditions, - pushed_down_filters, + google::protobuf::RepeatedPtrField{}, // don't care pushed down filters DAGPreparedSets(), source_columns, timezone_info); diff --git a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp index 7d360bdc4ea..6ce86f4027e 100644 --- a/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp +++ b/dbms/src/Storages/tests/gtests_parse_push_down_filter.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// 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. @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include From d7920d38633b93a19b5ed97f3f9ca26b913c2ffb Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 15 Mar 2023 16:55:49 +0800 Subject: [PATCH 23/25] new line Signed-off-by: Lloyd-Pottiger --- dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h | 2 +- dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h | 2 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGUtils.h | 2 +- dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h | 2 +- dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp | 2 +- dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp | 2 +- dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 4d88a34aafd..c0d3582f5d7 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -1566,4 +1566,4 @@ String DAGExpressionAnalyzer::buildTupleFunctionForGroupConcat( return applyFunction(func_name, argument_names, actions, nullptr); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 313fd5b681a..047cc528437 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -349,4 +349,4 @@ class DAGExpressionAnalyzer : private boost::noncopyable friend class DAGExpressionAnalyzerHelper; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 39a8bbbb0af..d594b0e59e1 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -1103,4 +1103,4 @@ TableLockHolders DAGStorageInterpreter::releaseAlterLocks() return drop_locks; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index 17d471dbfd6..fa8e81ffd44 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -141,4 +141,4 @@ class DAGStorageInterpreter std::vector> generated_column_infos; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index ad4529c950f..65daaa6a825 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1466,4 +1466,4 @@ tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) throw Exception(fmt::format("Unsupported function {}", name)); return func_name_sig_map[name]; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index b1286fea9f6..4241637c279 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -106,4 +106,4 @@ class UniqueNameGenerator tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index 71edde88ce5..0d7e21e10cd 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -58,4 +58,4 @@ class PushDownFilter : public std::enable_shared_from_this ExpressionActionsPtr extra_cast; }; -} // namespace DB::DM \ No newline at end of file +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp index d4bd4e8e45e..47a8e4ef28e 100644 --- a/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp +++ b/dbms/src/Storages/DeltaMerge/FilterParser/FilterParser.cpp @@ -984,4 +984,4 @@ std::unordered_map FilterParser } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp index d3a0c4245e3..06781634266 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp @@ -152,4 +152,4 @@ 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 \ No newline at end of file +} // 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 index 546dbfb78d7..ae013618dda 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h @@ -53,4 +53,4 @@ 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 \ No newline at end of file +} // namespace DB::DM::tests From cd42cc70d8f94defaf082c93c8314f3e3d03a912 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Wed, 15 Mar 2023 18:20:17 +0800 Subject: [PATCH 24/25] address comments Signed-off-by: Lloyd-Pottiger --- dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp | 2 +- dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp | 3 +-- dbms/src/Storages/StorageDeltaMerge.cpp | 2 -- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index c5b66c9f420..64bc0d16155 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp @@ -311,7 +311,7 @@ Block DMFileReader::readWithFilter(const IColumn::Filter & filter) { // 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 6, it points to index 3, skip = 3 >= 3, so use_packs[0] = false, read_rows = 3, next_pack_id_cp = 1 + // 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 diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp index 06781634266..6709ea1e48f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.cpp @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "gtest_segment_util.h" - #include +#include #include diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 60b508b0228..9cb158930aa 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -60,8 +60,6 @@ #include -#include "Functions/FunctionBinaryArithmetic.h" - namespace DB { namespace FailPoints From 1e0dd9c604ad12e39a2485dd3ff3b8941ccd4d6e Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Thu, 16 Mar 2023 13:10:24 +0800 Subject: [PATCH 25/25] add some comments Signed-off-by: Lloyd-Pottiger --- dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index c0d3582f5d7..7cb65e121f7 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -933,6 +933,8 @@ std::pair> DAGExpressionAnalyzer::buildExtraCastsAfter tipb::Expr fsp_expr = constructInt64LiteralTiExpr(fsp); fsp_col = getActions(fsp_expr, actions); 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; } @@ -956,7 +958,11 @@ bool DAGExpressionAnalyzer::appendExtraCastsAfterTS( if (!has_cast) return false; - // rename casted columns to original name + // 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);