From 0025c0f95f492967443881f3f5b96e25005a5e6c Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 27 Feb 2020 02:40:43 +0800 Subject: [PATCH 01/54] implement join --- contrib/client-c | 2 +- contrib/tipb | 2 +- .../DataStreams/CoprocessorBlockInputStream.h | 50 +++++++++++++++++++ .../Flash/Coprocessor/DAGStringConverter.cpp | 2 + dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 8 ++- dbms/src/Storages/IManageableStorage.h | 5 ++ dbms/src/Storages/StorageMergeTree.cpp | 38 ++++++++++++++ dbms/src/Storages/StorageMergeTree.h | 3 ++ dbms/src/Storages/Transaction/TMTContext.h | 2 + dbms/src/Storages/Transaction/TiDB.cpp | 15 ++++++ dbms/src/Storages/Transaction/TiDB.h | 1 + 11 files changed, 125 insertions(+), 3 deletions(-) create mode 100644 dbms/src/DataStreams/CoprocessorBlockInputStream.h diff --git a/contrib/client-c b/contrib/client-c index 7f48a0caf75..549970fd24c 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 7f48a0caf75520bcab53a35682b7bc36d83ab574 +Subproject commit 549970fd24ca795e01fcbdaf67adab1b5faee434 diff --git a/contrib/tipb b/contrib/tipb index 568726749cb..3fc41406b29 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 568726749cb746b365cca389ceb4043ca6a2f250 +Subproject commit 3fc41406b29ef098b023062988b7e9e04fa3b505 diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h new file mode 100644 index 00000000000..46e13663dff --- /dev/null +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include + +#include +#include + +namespace DB { + +class CoprocessorBlockInputStream : public IProfilingBlockInputStream { +public: + CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_) + : req(req_), resp_iter(pingcap::coprocessor::Client::send(cluster_, &req)), schema(schema_) { + pingcap::Exception error = resp_iter.prepare(); + if (!error.empty()) + throw error; + } + + Block getHeader() const override { + return {}; + } + + String getName() const override { return "Coprocessor"; } + + Block readImpl() override { + auto [data, has_next] = resp_iter.next(); + + if (!has_next) + { + return {}; + } + + tipb::Chunk chunk; + chunk.ParseFromString(data); + + Block block = codec.decode(chunk, schema); + + return block; + } + +private: + pingcap::coprocessor::Request req; + pingcap::coprocessor::ResponseIter resp_iter; + DAGSchema schema; + + ArrowChunkCodec codec; +}; + +} \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp index 69e19ddd957..5645764cc78 100644 --- a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp @@ -152,6 +152,8 @@ void DAGStringConverter::buildString(const tipb::Executor & executor, std::strin return buildTopNString(executor.topn(), ss); case tipb::ExecType::TypeLimit: return buildLimitString(executor.limit(), ss); + case tipb::ExecType::TypeJoin: + throw Exception("join has not implemented"); } } diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 90b8deb83c8..42ba12c7d1d 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -432,7 +432,13 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); query_info.mvcc_query_info->concurrent = 0.0; - pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + if (ts.engine() == 0) { + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, + max_streams); + } else { + pipeline.streams = storage->remote_read(dag.getKeyRanges(), query_info, ts); + } + if (pipeline.streams.empty()) { diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index ebfdaab72de..0ee149f0195 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -5,6 +5,9 @@ #include #include #include +#include + +#include namespace TiDB { @@ -49,6 +52,8 @@ class IManageableStorage : public IStorage virtual const TiDB::TableInfo & getTableInfo() const = 0; + virtual BlockInputStreams remote_read(const std::vector> & key_ranges, const SelectQueryInfo & query_info, tipb::TableScan ts); + // Apply AlterCommands synced from TiDB should use `alterFromTiDB` instead of `alter(...)` virtual void alterFromTiDB( const AlterCommands & commands, const TiDB::TableInfo & table_info, const String & database_name, const Context & context) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d11162e5867..a71fce83184 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -26,12 +26,15 @@ #include #include #include +#include #include #include #include #include +#include +#include namespace DB { @@ -131,6 +134,41 @@ StorageMergeTree::~StorageMergeTree() shutdown(); } +BlockInputStreams StorageMergeTree::remote_read(const std::vector> & key_ranges , const SelectQueryInfo & query_info, tipb::TableScan ts) +{ + std::vector cop_key_ranges; + for (const auto & key_range : key_ranges) + { + cop_key_ranges.push_back(pingcap::coprocessor::KeyRange{key_range.first.data(), key_range.second.data()}); + } + + ::tipb::DAGRequest dag_req; + + auto * exec = dag_req.add_executors(); + exec->set_tp(tipb::ExecType::TypeTableScan); + exec->set_allocated_tbl_scan(&ts); + dag_req.set_encode_type(tipb::EncodeType::TypeChunk); + + DAGSchema schema; + for (int i =0 ; i < ts.columns_size(); i++) + { + dag_req.set_output_offsets(i, i); + auto id = ts.columns(i).column_id(); + const ColumnInfo & info = data.table_info->getColumnInfoByID(id); + schema.push_back(std::make_pair(info.name, info)); + } + + pingcap::coprocessor::Request req; + + req.data = dag_req.SerializeAsString(); + req.tp = pingcap::coprocessor::ReqType::DAG; + req.start_ts = query_info.mvcc_query_info->read_tso; + req.ranges = cop_key_ranges; + + BlockInputStreamPtr input = std::make_shared(context.getTMTContext().getKVCluster(), req, schema); + return {input}; +} + BlockInputStreams StorageMergeTree::read( const Names & column_names, const SelectQueryInfo & query_info, diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 92e98d577ac..f96c897df71 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -13,6 +13,8 @@ #include #include +#include + namespace TiDB { struct TableInfo; @@ -59,6 +61,7 @@ class StorageMergeTree : public ext::shared_ptr_helper, public bool hasColumn(const String & column_name) const override { return data.hasColumn(column_name); } + BlockInputStreams remote_read(const std::vector> & key_ranges , const SelectQueryInfo & query_info, tipb::TableScan ts) override; BlockInputStreams read(const Names & column_names, const SelectQueryInfo & query_info, const Context & context, diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index a1b9f7f8f06..da4be172e44 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -51,6 +51,8 @@ class TMTContext : private boost::noncopyable pingcap::pd::ClientPtr getPDClient() const; + pingcap::kv::Cluster * getKVCluster() {return cluster.get();} + IndexReaderPtr createIndexReader() const; void restore(); diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index c40193f0546..2cfd79e1e65 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -542,6 +542,21 @@ String TableInfo::getColumnName(const ColumnID id) const DB::ErrorCodes::LOGICAL_ERROR); } +ColumnInfo TableInfo::getColumnInfoByID(const ColumnID id) const +{ + for (auto & col : columns) + { + if (id == col.id) + { + return col; + } + } + + throw DB::Exception( + std::string(__PRETTY_FUNCTION__) + ": Invalidate column id " + std::to_string(id) + " for table " + db_name + "." + name, + DB::ErrorCodes::LOGICAL_ERROR); +} + std::optional> TableInfo::getPKHandleColumn() const { if (!pk_is_handle) diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index 657262a739e..955af266f64 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -287,6 +287,7 @@ struct TableInfo ColumnID getColumnID(const String & name) const; String getColumnName(const ColumnID id) const; + ColumnInfo getColumnInfoByID(const ColumnID id) const; std::optional> getPKHandleColumn() const; From 3e1abb253367131b4a4b198ecd0828e3a93c29be Mon Sep 17 00:00:00 2001 From: Fei Han Date: Fri, 28 Feb 2020 02:19:03 +0800 Subject: [PATCH 02/54] fix --- contrib/client-c | 2 +- .../DataStreams/CoprocessorBlockInputStream.h | 73 +++++++++++++++---- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 2 +- dbms/src/Storages/IManageableStorage.h | 48 ++++++++++-- dbms/src/Storages/StorageMergeTree.cpp | 39 ---------- dbms/src/Storages/StorageMergeTree.h | 3 - 6 files changed, 104 insertions(+), 63 deletions(-) diff --git a/contrib/client-c b/contrib/client-c index 549970fd24c..891ea87a882 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 549970fd24ca795e01fcbdaf67adab1b5faee434 +Subproject commit 891ea87a882fc2fa8a6c1762389200dbe24040a4 diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h index 46e13663dff..1606c649ed5 100644 --- a/dbms/src/DataStreams/CoprocessorBlockInputStream.h +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -2,49 +2,94 @@ #include #include - #include #include -namespace DB { +namespace DB +{ -class CoprocessorBlockInputStream : public IProfilingBlockInputStream { +class CoprocessorBlockInputStream : public IProfilingBlockInputStream +{ public: CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_) - : req(req_), resp_iter(pingcap::coprocessor::Client::send(cluster_, &req)), schema(schema_) { + : req(req_), + resp_iter(pingcap::coprocessor::Client::send(cluster_, &req)), + schema(schema_), + log(&Logger::get("pingcap/coprocessor")) + { pingcap::Exception error = resp_iter.prepare(); if (!error.empty()) + { + LOG_WARNING(log, "coprocessor client meets error: " << error.displayText()); throw error; + } } - Block getHeader() const override { - return {}; + Block getHeader() const override + { + ColumnsWithTypeAndName columns; + for (auto name_and_column : schema) + { + auto tp = getDataTypeByColumnInfo(name_and_column.second); + ColumnWithTypeAndName col(tp, name_and_column.first); + columns.emplace_back(col); + } + LOG_DEBUG(log, "header columns: " + std::to_string(columns.size())); + return Block(columns); } String getName() const override { return "Coprocessor"; } - Block readImpl() override { + Block readImpl() override + { + if (chunk_queue.empty()) + { + bool has_next = fetchNewData(); + if (!has_next) + return {}; + } + auto chunk = chunk_queue.front(); + chunk_queue.pop(); + return codec.decode(chunk, schema); + } + +private: + bool fetchNewData() + { + LOG_DEBUG(log, "fetch new data"); + auto [data, has_next] = resp_iter.next(); if (!has_next) { - return {}; + return false; } - tipb::Chunk chunk; - chunk.ParseFromString(data); - Block block = codec.decode(chunk, schema); + resp = std::make_shared(); + resp->ParseFromString(data); + int chunks_size = resp->chunks_size(); - return block; + if (chunks_size == 0) + return fetchNewData(); + + for (int i = 0; i < chunks_size; i++) + { + chunk_queue.push(resp->chunks(i)); + } + return true; } -private: pingcap::coprocessor::Request req; pingcap::coprocessor::ResponseIter resp_iter; DAGSchema schema; ArrowChunkCodec codec; + std::shared_ptr resp; + + std::queue chunk_queue; + + Logger * log; }; -} \ No newline at end of file +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 42ba12c7d1d..6598896fd24 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -436,7 +436,7 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); } else { - pipeline.streams = storage->remote_read(dag.getKeyRanges(), query_info, ts); + pipeline.streams = storage->remote_read(dag.getKeyRanges(), query_info, ts, context); } diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index 0ee149f0195..4ee56ce54c8 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -1,12 +1,15 @@ #pragma once +#include #include +#include #include #include -#include +#include #include +#include #include - +#include #include namespace TiDB @@ -52,12 +55,47 @@ class IManageableStorage : public IStorage virtual const TiDB::TableInfo & getTableInfo() const = 0; - virtual BlockInputStreams remote_read(const std::vector> & key_ranges, const SelectQueryInfo & query_info, tipb::TableScan ts); + virtual BlockInputStreams remote_read(const std::vector> & key_ranges, + const SelectQueryInfo & query_info, const tipb::TableScan & ts, Context & context) + { + std::vector cop_key_ranges; + for (const auto & key_range : key_ranges) + { + cop_key_ranges.push_back( + pingcap::coprocessor::KeyRange{static_cast(key_range.first), static_cast(key_range.second)}); + } + + ::tipb::DAGRequest dag_req; + + auto * exec = dag_req.add_executors(); + exec->set_tp(tipb::ExecType::TypeTableScan); + exec->set_allocated_tbl_scan(new tipb::TableScan(ts)); + dag_req.set_encode_type(tipb::EncodeType::TypeChunk); + + DAGSchema schema; + for (int i = 0; i < ts.columns_size(); i++) + { + dag_req.add_output_offsets(i); + auto id = ts.columns(i).column_id(); + const ColumnInfo & info = getTableInfo().getColumnInfoByID(id); + schema.push_back(std::make_pair(info.name, info)); + } + + pingcap::coprocessor::Request req; + + dag_req.SerializeToString(&req.data); + req.tp = pingcap::coprocessor::ReqType::DAG; + req.start_ts = query_info.mvcc_query_info->read_tso; + req.ranges = cop_key_ranges; + + BlockInputStreamPtr input = std::make_shared(context.getTMTContext().getKVCluster(), req, schema); + return {input}; + }; // Apply AlterCommands synced from TiDB should use `alterFromTiDB` instead of `alter(...)` virtual void alterFromTiDB( - const AlterCommands & commands, const TiDB::TableInfo & table_info, const String & database_name, const Context & context) - = 0; + const AlterCommands & commands, const TiDB::TableInfo & table_info, const String & database_name, const Context & context) + = 0; PKType getPKType() const { diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index a71fce83184..aea198b3eec 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -26,15 +26,11 @@ #include #include #include -#include #include #include #include #include -#include - -#include namespace DB { @@ -134,41 +130,6 @@ StorageMergeTree::~StorageMergeTree() shutdown(); } -BlockInputStreams StorageMergeTree::remote_read(const std::vector> & key_ranges , const SelectQueryInfo & query_info, tipb::TableScan ts) -{ - std::vector cop_key_ranges; - for (const auto & key_range : key_ranges) - { - cop_key_ranges.push_back(pingcap::coprocessor::KeyRange{key_range.first.data(), key_range.second.data()}); - } - - ::tipb::DAGRequest dag_req; - - auto * exec = dag_req.add_executors(); - exec->set_tp(tipb::ExecType::TypeTableScan); - exec->set_allocated_tbl_scan(&ts); - dag_req.set_encode_type(tipb::EncodeType::TypeChunk); - - DAGSchema schema; - for (int i =0 ; i < ts.columns_size(); i++) - { - dag_req.set_output_offsets(i, i); - auto id = ts.columns(i).column_id(); - const ColumnInfo & info = data.table_info->getColumnInfoByID(id); - schema.push_back(std::make_pair(info.name, info)); - } - - pingcap::coprocessor::Request req; - - req.data = dag_req.SerializeAsString(); - req.tp = pingcap::coprocessor::ReqType::DAG; - req.start_ts = query_info.mvcc_query_info->read_tso; - req.ranges = cop_key_ranges; - - BlockInputStreamPtr input = std::make_shared(context.getTMTContext().getKVCluster(), req, schema); - return {input}; -} - BlockInputStreams StorageMergeTree::read( const Names & column_names, const SelectQueryInfo & query_info, diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index f96c897df71..92e98d577ac 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -13,8 +13,6 @@ #include #include -#include - namespace TiDB { struct TableInfo; @@ -61,7 +59,6 @@ class StorageMergeTree : public ext::shared_ptr_helper, public bool hasColumn(const String & column_name) const override { return data.hasColumn(column_name); } - BlockInputStreams remote_read(const std::vector> & key_ranges , const SelectQueryInfo & query_info, tipb::TableScan ts) override; BlockInputStreams read(const Names & column_names, const SelectQueryInfo & query_info, const Context & context, From 77e66932d96e5366acef779606f48d53623d6f0c Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 4 Mar 2020 16:35:46 +0800 Subject: [PATCH 03/54] update tipb --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index 3fc41406b29..bac275f4141 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 3fc41406b29ef098b023062988b7e9e04fa3b505 +Subproject commit bac275f41413725e1a3b6379a168b31e93a32612 From 4095a44cf07bd7dfeb804a878d3e744273a238b8 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 4 Mar 2020 17:32:45 +0800 Subject: [PATCH 04/54] update tipb --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index bac275f4141..aa600db6e32 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit bac275f41413725e1a3b6379a168b31e93a32612 +Subproject commit aa600db6e32e9612d031bbbce2b51a744b132ba5 From 92c07860366e2d1fb31c10638ebf8faea10fb053 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 2 Mar 2020 10:12:44 +0800 Subject: [PATCH 05/54] save work: refine dag interpreter, introduce dag interpreter query block --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 1 + dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 173 ++++ dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 41 + dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 48 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 6 +- .../Flash/Coprocessor/DAGStringConverter.cpp | 1 + dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 39 +- dbms/src/Flash/Coprocessor/InterpreterDAG.h | 37 +- .../Coprocessor/InterpreterDAGQueryBlock.cpp | 885 ++++++++++++++++++ .../Coprocessor/InterpreterDAGQueryBlock.h | 131 +++ 10 files changed, 1309 insertions(+), 53 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp create mode 100644 dbms/src/Flash/Coprocessor/DAGQueryBlock.h create mode 100644 dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp create mode 100644 dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 03a86ccebe1..4952db0c3e0 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -290,6 +290,7 @@ bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( if (tz_col.length() == 0) tz_col = getActions(tz_expr, actions); String casted_name = appendTimeZoneCast(tz_col, source_columns[i].name, func_name, actions); + // todo only append the original colum if keep_session_timezone is false source_columns.emplace_back(source_columns[i].name, source_columns[i].type); source_columns[i].name = casted_name; ret = true; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp new file mode 100644 index 00000000000..ee85621822a --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -0,0 +1,173 @@ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +#include "DAGQueryBlock.h" +#include "DAGUtils.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +extern const int LOGICAL_ERROR; +extern const int COP_BAD_DAG_REQUEST; +} // namespace ErrorCodes + +class Context; +class TiFlashMetrics; +using TiFlashMetricsPtr = std::shared_ptr; + +bool isSourceNode(const tipb::Executor * root) +{ + return root->tp() == tipb::ExecType::TypeJoin || root->tp() == tipb::ExecType::TypeTableScan; +} + +const static String SOURCE_NAME("source"); +const static String SEL_NAME("selection"); +const static String AGG_NAME("aggregation"); +const static String TOPN_NAME("topN"); +const static String LIMIT_NAME("limit"); + +static void assignOrThrowException(const tipb::Executor ** to, const tipb::Executor * from, const String & name) +{ + if (*to != nullptr) + { + throw Exception("Duplicated " + name + " in DAG request"); + } + *to = from; +} + +void collectOutPutFieldTypesFromAgg(std::vector & field_type, const tipb::Aggregation & agg) +{ + for (auto & expr : agg.agg_func()) + { + if (!exprHasValidFieldType(expr)) + { + throw Exception("Agg expression without valid field type", ErrorCodes::COP_BAD_DAG_REQUEST); + } + field_type.push_back(expr.field_type()); + } + for (auto & expr : agg.group_by()) + { + if (!exprHasValidFieldType(expr)) + { + throw Exception("Group by expression without valid field type", ErrorCodes::COP_BAD_DAG_REQUEST); + } + field_type.push_back(expr.field_type()); + } +} + +DAGQueryBlock::DAGQueryBlock(const tipb::Executor * root) +{ + const tipb::Executor * current = root; + while (isSourceNode(current)) + { + switch (current->tp()) + { + case tipb::ExecType::TypeSelection: + assignOrThrowException(&selection, current, SEL_NAME); + current = ¤t->selection().child(); + break; + case tipb::ExecType::TypeAggregation: + assignOrThrowException(&aggregation, current, AGG_NAME); + collectOutPutFieldTypesFromAgg(output_field_types, current->aggregation()); + current = ¤t->aggregation().child(); + break; + case tipb::ExecType::TypeStreamAgg: + assignOrThrowException(&aggregation, current, AGG_NAME); + collectOutPutFieldTypesFromAgg(output_field_types, current->stream_agg()); + current = ¤t->stream_agg().child(); + break; + case tipb::ExecType::TypeLimit: + assignOrThrowException(&limitOrTopN, current, LIMIT_NAME); + current = ¤t->limit().child(); + break; + case tipb::ExecType::TypeTopN: + assignOrThrowException(&limitOrTopN, current, TOPN_NAME); + current = ¤t->topn().child(); + break; + case tipb::ExecType::TypeIndexScan: + throw Exception("Unsupported executor in DAG request: " + current->DebugString(), ErrorCodes::NOT_IMPLEMENTED); + default: + throw Exception("Should not reach here", ErrorCodes::LOGICAL_ERROR); + } + } + assignOrThrowException(&source, current, SOURCE_NAME); + if (source->tp() == tipb::ExecType::TypeJoin) + { + // todo need to figure out left and right side of the join + children.push_back(std::make_shared(&source->join().probe_exec())); + children.push_back(std::make_shared(&source->join().build_exec())); + if (output_field_types.empty()) + { + for (auto & field_type : children[0]->output_field_types) + output_field_types.push_back(field_type); + for (auto & field_type : children[1]->output_field_types) + output_field_types.push_back(field_type); + } + } + else + { + if (output_field_types.empty()) + { + for (auto & ci : source->tbl_scan().columns()) + { + tipb::FieldType field_type; + field_type.set_tp(ci.tp()); + field_type.set_flag(ci.flag()); + field_type.set_flen(ci.columnlen()); + field_type.set_decimal(ci.decimal()); + output_field_types.push_back(field_type); + } + } + } +} + +DAGQueryBlock::DAGQueryBlock(std::vector & executors) +{ + for (size_t i = 0; i < executors.size(); i++) + { + switch (executors[i]->tp()) + { + case tipb::ExecType::TypeTableScan: + assignOrThrowException(&source, executors[i], SOURCE_NAME); + break; + case tipb::ExecType::TypeSelection: + assignOrThrowException(&selection, executors[i], SEL_NAME); + break; + case tipb::ExecType::TypeStreamAgg: + assignOrThrowException(&aggregation, executors[i], AGG_NAME); + collectOutPutFieldTypesFromAgg(output_field_types, executors[i]->stream_agg()); + break; + case tipb::ExecType::TypeAggregation: + assignOrThrowException(&aggregation, executors[i], AGG_NAME); + collectOutPutFieldTypesFromAgg(output_field_types, executors[i]->aggregation()); + break; + case tipb::ExecType::TypeTopN: + assignOrThrowException(&limitOrTopN, executors[i], TOPN_NAME); + break; + case tipb::ExecType::TypeLimit: + assignOrThrowException(&limitOrTopN, executors[i], LIMIT_NAME); + break; + default: + throw Exception("Unsupported executor in DAG request: " + executors[i]->DebugString(), ErrorCodes::NOT_IMPLEMENTED); + } + } + if (output_field_types.empty()) + { + for (auto & ci : source->tbl_scan().columns()) + { + tipb::FieldType field_type; + field_type.set_tp(ci.tp()); + field_type.set_flag(ci.flag()); + field_type.set_flen(ci.columnlen()); + field_type.set_decimal(ci.decimal()); + output_field_types.push_back(field_type); + } + } +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h new file mode 100644 index 00000000000..362a57d718b --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -0,0 +1,41 @@ +#pragma once + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class TiFlashMetrics; +using TiFlashMetricsPtr = std::shared_ptr; + +/// DAGQueryBlock is a dag query from single source, +/// which means the query block contains a source node(tablescan or join) +/// and some of the optional node.(selection/aggregation/project/limit/topN) +class DAGQueryBlock +{ +public: + explicit DAGQueryBlock(const tipb::Executor * root); + explicit DAGQueryBlock(std::vector & executors); + const tipb::Executor * source = nullptr; + const tipb::Executor * selection = nullptr; + const tipb::Executor * aggregation = nullptr; + const tipb::Executor * limitOrTopN = nullptr; + // todo use unique_ptr instead + std::vector> children; + std::vector output_field_types; + // kinds of project + std::vector output_offsets; + bool is_final_query_block; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index cd81c87e719..6608b57924d 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -20,14 +20,14 @@ const String DAGQuerySource::AGG_NAME("aggregation"); const String DAGQuerySource::TOPN_NAME("topN"); const String DAGQuerySource::LIMIT_NAME("limit"); -static void assignOrThrowException(Int32 & index, Int32 value, const String & name) -{ - if (index != -1) - { - throw Exception("Duplicated " + name + " in DAG request"); - } - index = value; -} +//static void assignOrThrowException(Int32 & index, Int32 value, const String & name) +//{ +// if (index != -1) +// { +// throw Exception("Duplicated " + name + " in DAG request"); +// } +// index = value; +//} DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, const std::vector> & key_ranges_, @@ -41,6 +41,7 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, Re dag_request(dag_request_), metrics(context.getTiFlashMetrics()) { + /* for (int i = 0; i < dag_request.executors_size(); i++) { switch (dag_request.executors(i).tp()) @@ -72,7 +73,32 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, Re "Unsupported executor in DAG request: " + dag_request.executors(i).DebugString(), ErrorCodes::NOT_IMPLEMENTED); } } - analyzeResultFieldTypes(); + */ + if (dag_request.has_executor_tree()) + { + query_block_tree = std::make_shared(&dag_request.executor_tree()); + } else { + std::vector executors; + for(const tipb::Executor & executor : dag_request.executors()) + executors.push_back(&executor); + query_block_tree = std::make_shared(executors); + } + query_block_tree->is_final_query_block = true; + for(Int32 i : dag_request.output_offsets()) + query_block_tree->output_offsets.push_back(i); + if (query_block_tree->aggregation != nullptr) + { + for(auto & field_type : query_block_tree->output_field_types) + result_field_types.push_back(field_type); + } + else + { + for (UInt32 i : dag_request.output_offsets()) + { + result_field_types.push_back(query_block_tree->output_field_types[i]); + } + } + //analyzeResultFieldTypes(); analyzeDAGEncodeType(); } @@ -107,6 +133,7 @@ std::unique_ptr DAGQuerySource::interpreter(Context &, QueryProces return std::make_unique(context, *this); } +/* bool fillExecutorOutputFieldTypes(const tipb::Executor & executor, std::vector & output_field_types) { tipb::FieldType field_type; @@ -145,7 +172,9 @@ bool fillExecutorOutputFieldTypes(const tipb::Executor & executor, std::vector executor_output; @@ -171,5 +200,6 @@ void DAGQuerySource::analyzeResultFieldTypes() result_field_types.push_back(executor_output[i]); } } + */ } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index 7e6f142db8d..bd7f2192051 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -10,6 +10,7 @@ #include #include #include +#include "DAGQueryBlock.h" namespace DB { @@ -86,6 +87,8 @@ class DAGQuerySource : public IQuerySource ASTPtr getAST() const { return ast; }; tipb::EncodeType getEncodeType() const { return encode_type; } + + std::shared_ptr getQueryBlock() const { return query_block_tree; } protected: void assertValid(Int32 index, const String & name) const @@ -96,7 +99,7 @@ class DAGQuerySource : public IQuerySource } } - void analyzeResultFieldTypes(); + //void analyzeResultFieldTypes(); void analyzeDAGEncodeType(); protected: @@ -120,6 +123,7 @@ class DAGQuerySource : public IQuerySource std::vector result_field_types; tipb::EncodeType encode_type; + std::shared_ptr query_block_tree; ASTPtr ast; }; diff --git a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp index 5645764cc78..7cb3ad33394 100644 --- a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp @@ -139,6 +139,7 @@ void DAGStringConverter::buildString(const tipb::Executor & executor, std::strin { case tipb::ExecType::TypeTableScan: return buildTSString(executor.tbl_scan(), ss); + case tipb::ExecType::TypeJoin: case tipb::ExecType::TypeIndexScan: // index scan not supported throw Exception("IndexScan is not supported", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 6598896fd24..a038b4affaf 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -223,7 +223,7 @@ bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, return ret; } -bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, +static bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, const ImutRegionRangePtr & region_key_range, Int32 handle_col_id, tipb::Expr & handle_filter) { if (key_ranges.empty()) @@ -511,7 +511,7 @@ bool InterpreterDAG::addTimeZoneCastAfterTS(std::vector & is_ts_column, Pi return false; } -InterpreterDAG::AnalysisResult InterpreterDAG::analyzeExpressions() +AnalysisResult InterpreterDAG::analyzeExpressions() { AnalysisResult res; ExpressionActionsChain chain; @@ -848,15 +848,37 @@ void InterpreterDAG::executeLimit(Pipeline & pipeline) } } -BlockIO InterpreterDAG::execute() +BlockIO InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info) { - Pipeline pipeline; - executeImpl(pipeline); - executeUnion(pipeline); + if (!query_block.children.empty()) + { + BlockInputStreams input_streams; + for (auto & child : query_block.children) + { + BlockIO child_stream = executeQueryBlock(*child, region_info); + input_streams.push_back(child_stream.in); + } + InterpreterDAGQueryBlock query_block_interpreter(context, input_streams, query_block, + keep_session_timezone_info, region_info, dag.getDAGRequest()); + return query_block_interpreter.execute(); + } + else + { + InterpreterDAGQueryBlock query_block_interpreter(context, {}, query_block, + keep_session_timezone_info, region_info, dag.getDAGRequest()); + return query_block_interpreter.execute(); + } +} - BlockIO res; - res.in = pipeline.firstStream(); +BlockIO InterpreterDAG::execute() +{ + /// region_info should based on the source executor, however + /// tidb does not support multi-table dag request yet, so + /// it is ok to use the same region_info for the whole dag request + RegionInfo region_info(dag.getRegionID(), dag.getRegionVersion(), dag.getRegionConfVersion(), dag.getKeyRanges()); + BlockIO res = executeQueryBlock(*dag.getQueryBlock(), region_info); + /* LOG_DEBUG( log, __PRETTY_FUNCTION__ << " Convert DAG request to BlockIO, adding " << analyzer->getImplicitCastCount() << " implicit cast"); if (log->debug()) @@ -873,6 +895,7 @@ BlockIO InterpreterDAG::execute() LOG_DEBUG(log, __PRETTY_FUNCTION__ << " Failed to convert DAG request to sql text"); } } + */ return res; } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index 59dcde3cd11..8faf6abc0d5 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -36,42 +37,8 @@ class InterpreterDAG : public IInterpreter BlockIO execute(); private: - struct Pipeline - { - BlockInputStreams streams; - - BlockInputStreamPtr & firstStream() { return streams.at(0); } - - template - void transform(Transform && transform) - { - for (auto & stream : streams) - transform(stream); - } - - bool hasMoreThanOneStream() const { return streams.size() > 1; } - }; - - struct AnalysisResult - { - bool has_where = false; - bool need_aggregate = false; - bool has_order_by = false; - - ExpressionActionsPtr before_where; - ExpressionActionsPtr before_aggregation; - ExpressionActionsPtr before_order_and_select; - ExpressionActionsPtr final_projection; - - String filter_column_name; - Strings order_column_names; - /// Columns from the SELECT list, before renaming them to aliases. - Names selected_columns; - - Names aggregation_keys; - AggregateDescriptions aggregate_descriptions; - }; + BlockIO executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info); void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp new file mode 100644 index 00000000000..240824f78f4 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp @@ -0,0 +1,885 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNKNOWN_TABLE; +extern const int TOO_MANY_COLUMNS; +extern const int SCHEMA_VERSION_ERROR; +extern const int UNKNOWN_EXCEPTION; +extern const int COP_BAD_DAG_REQUEST; +} // namespace ErrorCodes + +InterpreterDAGQueryBlock::InterpreterDAGQueryBlock(Context & context_, const BlockInputStreams & input_streams_, + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info_, const tipb::DAGRequest & rqst_) + : context(context_), + input_streams(input_streams_), + query_block(query_block_), + keep_session_timezone_info(keep_session_timezone_info_), + region_info(region_info_), + rqst(rqst_), + log(&Logger::get("InterpreterDAGQueryBlock")) +{ + if (query_block.selection != nullptr) + { + for (auto & condition : query_block.selection->selection().conditions()) + conditions.push_back(&condition); + } +} + +template +void constructHandleColRefExpr(tipb::Expr & expr, Int64 col_index) +{ + expr.set_tp(tipb::ExprType::ColumnRef); + std::stringstream ss; + encodeDAGInt64(col_index, ss); + expr.set_val(ss.str()); + auto * field_type = expr.mutable_field_type(); + field_type->set_tp(TiDB::TypeLongLong); + // handle col is always not null + field_type->set_flag(TiDB::ColumnFlagNotNull); + if constexpr (!std::is_signed_v) + { + field_type->set_flag(TiDB::ColumnFlagUnsigned); + } +} + +template +void constructIntLiteralExpr(tipb::Expr & expr, HandleType value) +{ + constexpr bool is_signed = std::is_signed_v; + if constexpr (is_signed) + { + expr.set_tp(tipb::ExprType::Int64); + } + else + { + expr.set_tp(tipb::ExprType::Uint64); + } + std::stringstream ss; + if constexpr (is_signed) + { + encodeDAGInt64(value, ss); + } + else + { + encodeDAGUInt64(value, ss); + } + expr.set_val(ss.str()); + auto * field_type = expr.mutable_field_type(); + field_type->set_tp(TiDB::TypeLongLong); + field_type->set_flag(TiDB::ColumnFlagNotNull); + if constexpr (!is_signed) + { + field_type->set_flag(TiDB::ColumnFlagUnsigned); + } +} + +template +void constructBoundExpr(Int32 handle_col_id, tipb::Expr & expr, TiKVHandle::Handle bound, bool is_left_bound) +{ + auto * left = expr.add_children(); + constructHandleColRefExpr(*left, handle_col_id); + auto * right = expr.add_children(); + constructIntLiteralExpr(*right, bound.handle_id); + expr.set_tp(tipb::ExprType::ScalarFunc); + if (is_left_bound) + expr.set_sig(tipb::ScalarFuncSig::GEInt); + else + expr.set_sig(tipb::ScalarFuncSig::LTInt); + auto * field_type = expr.mutable_field_type(); + field_type->set_tp(TiDB::TypeLongLong); + field_type->set_flag(TiDB::ColumnFlagNotNull); + field_type->set_flag(TiDB::ColumnFlagUnsigned); +} + +template +void constructExprBasedOnRange(Int32 handle_col_id, tipb::Expr & expr, HandleRange range) +{ + if (range.second == TiKVHandle::Handle::max) + { + constructBoundExpr(handle_col_id, expr, range.first, true); + } + else + { + auto * left = expr.add_children(); + constructBoundExpr(handle_col_id, *left, range.first, true); + auto * right = expr.add_children(); + constructBoundExpr(handle_col_id, *right, range.second, false); + expr.set_tp(tipb::ExprType::ScalarFunc); + expr.set_sig(tipb::ScalarFuncSig::LogicalAnd); + auto * field_type = expr.mutable_field_type(); + field_type->set_tp(TiDB::TypeLongLong); + field_type->set_flag(TiDB::ColumnFlagNotNull); + field_type->set_flag(TiDB::ColumnFlagUnsigned); + } +} + +template +bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, const std::vector> & region_ranges, + Int32 handle_col_id, tipb::Expr & handle_filter) +{ + if (ranges.empty()) + { + // generate an always false filter + constructInt64LiteralTiExpr(handle_filter, 0); + return false; + } + std::sort(ranges.begin(), ranges.end(), + [](const HandleRange & a, const HandleRange & b) { return a.first < b.first; }); + + std::vector> merged_ranges; + HandleRange merged_range; + merged_range.first = ranges[0].first; + merged_range.second = ranges[0].second; + + for (size_t i = 1; i < ranges.size(); i++) + { + if (merged_range.second >= ranges[i].first) + merged_range.second = merged_range.second >= ranges[i].second ? merged_range.second : ranges[i].second; + else + { + if (merged_range.second > merged_range.first) + merged_ranges.emplace_back(std::make_pair(merged_range.first, merged_range.second)); + merged_range.first = ranges[i].first; + merged_range.second = ranges[i].second; + } + } + if (merged_range.second > merged_range.first) + merged_ranges.emplace_back(std::make_pair(merged_range.first, merged_range.second)); + + bool ret = true; + for (const auto & region_range : region_ranges) + { + bool covered = false; + for (const auto & range : merged_ranges) + { + if (region_range.first >= range.first && region_range.second <= range.second) + { + covered = true; + break; + } + } + if (!covered && region_range.second > region_range.first) + { + ret = false; + break; + } + } + if (!ret) + { + if (merged_ranges.empty()) + { + constructInt64LiteralTiExpr(handle_filter, 0); + } + else if (merged_ranges.size() == 1) + { + constructExprBasedOnRange(handle_col_id, handle_filter, merged_ranges[0]); + } + else + { + for (const auto & range : merged_ranges) + { + auto * filter = handle_filter.add_children(); + constructExprBasedOnRange(handle_col_id, *filter, range); + } + handle_filter.set_tp(tipb::ExprType::ScalarFunc); + handle_filter.set_sig(tipb::ScalarFuncSig::LogicalOr); + auto * field_type = handle_filter.mutable_field_type(); + field_type->set_tp(TiDB::TypeLongLong); + field_type->set_flag(TiDB::ColumnFlagNotNull); + field_type->set_flag(TiDB::ColumnFlagUnsigned); + } + } + return ret; +} + +bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, + const ImutRegionRangePtr & region_key_range, Int32 handle_col_id, tipb::Expr & handle_filter) +{ + if (key_ranges.empty()) + { + constructInt64LiteralTiExpr(handle_filter, 0); + return false; + } + + std::vector> handle_ranges; + for (auto & range : key_ranges) + { + TiKVRange::Handle start = TiKVRange::getRangeHandle(range.first, table_id); + TiKVRange::Handle end = TiKVRange::getRangeHandle(range.second, table_id); + handle_ranges.emplace_back(std::make_pair(start, end)); + } + + std::vector> region_handle_ranges; + auto & raw_keys = region_key_range->rawKeys(); + TiKVRange::Handle region_start = TiKVRange::getRangeHandle(raw_keys.first, table_id); + TiKVRange::Handle region_end = TiKVRange::getRangeHandle(raw_keys.second, table_id); + region_handle_ranges.emplace_back(std::make_pair(region_start, region_end)); + + if (pk_is_uint64) + { + std::vector> update_handle_ranges; + for (auto & range : handle_ranges) + { + const auto [n, new_range] = CHTableHandle::splitForUInt64TableHandle(range); + + for (int i = 0; i < n; i++) + { + update_handle_ranges.emplace_back(new_range[i]); + } + } + std::vector> update_region_handle_ranges; + for (auto & range : region_handle_ranges) + { + const auto [n, new_range] = CHTableHandle::splitForUInt64TableHandle(range); + + for (int i = 0; i < n; i++) + { + update_region_handle_ranges.emplace_back(new_range[i]); + } + } + return checkRangeAndGenExprIfNeeded(update_handle_ranges, update_region_handle_ranges, handle_col_id, handle_filter); + } + else + return checkRangeAndGenExprIfNeeded(handle_ranges, region_handle_ranges, handle_col_id, handle_filter); +} + +RegionException::RegionReadStatus InterpreterDAGQueryBlock::getRegionReadStatus(const RegionPtr & current_region) +{ + if (!current_region) + return RegionException::NOT_FOUND; + if (current_region->version() != region_info.region_version || current_region->confVer() != region_info.region_conf_version) + return RegionException::VERSION_ERROR; + if (current_region->isPendingRemove()) + return RegionException::PENDING_REMOVE; + return RegionException::OK; +} + +// the flow is the same as executeFetchcolumns +void InterpreterDAGQueryBlock::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) +{ + if (!ts.has_table_id()) + { + // do not have table id + throw Exception("Table id not specified in table scan executor", ErrorCodes::COP_BAD_DAG_REQUEST); + } + TableID table_id = ts.table_id(); + + const Settings & settings = context.getSettingsRef(); + + if (settings.schema_version == DEFAULT_UNSPECIFIED_SCHEMA_VERSION) + { + storage = context.getTMTContext().getStorages().get(table_id); + if (storage == nullptr) + { + throw Exception("Table " + std::to_string(table_id) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + } + table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + } + else + { + getAndLockStorageWithSchemaVersion(table_id, settings.schema_version); + } + + Names required_columns; + std::vector source_columns; + std::vector is_ts_column; + String handle_column_name = MutableSupport::tidb_pk_column_name; + if (auto pk_handle_col = storage->getTableInfo().getPKHandleColumn()) + handle_column_name = pk_handle_col->get().name; + + for (Int32 i = 0; i < ts.columns().size(); i++) + { + auto const & ci = ts.columns(i); + ColumnID cid = ci.column_id(); + + if (cid == -1) + { + // Column ID -1 return the handle column + required_columns.push_back(handle_column_name); + auto pair = storage->getColumns().getPhysical(handle_column_name); + source_columns.push_back(pair); + is_ts_column.push_back(false); + handle_col_id = i; + continue; + } + + String name = storage->getTableInfo().getColumnName(cid); + required_columns.push_back(name); + if (name == handle_column_name) + handle_col_id = i; + auto pair = storage->getColumns().getPhysical(name); + source_columns.emplace_back(std::move(pair)); + is_ts_column.push_back(ci.tp() == TiDB::TypeTimestamp); + } + + if (handle_col_id == -1) + handle_col_id = required_columns.size(); + + auto current_region = context.getTMTContext().getKVStore()->getRegion(region_info.region_id); + auto region_read_status = getRegionReadStatus(current_region); + if (region_read_status != RegionException::OK) + { + std::vector region_ids; + region_ids.push_back(region_info.region_id); + LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << region_info.region_id); + throw RegionException(std::move(region_ids), region_read_status); + } + const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; + if (!checkKeyRanges(region_info.key_ranges, table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) + { + // need to add extra filter on handle column + filter_on_handle = true; + conditions.push_back(&handle_filter_expr); + } + + bool has_handle_column = (handle_col_id != (Int32)required_columns.size()); + + if (filter_on_handle && !has_handle_column) + { + // if need to add filter on handle column, and + // the handle column is not selected in ts, add + // the handle column + required_columns.push_back(handle_column_name); + auto pair = storage->getColumns().getPhysical(handle_column_name); + source_columns.push_back(pair); + is_ts_column.push_back(false); + } + + analyzer = std::make_unique(std::move(source_columns), context); + + if (query_block.aggregation == nullptr && query_block.is_final_query_block) + { + // only add final project if the query block is the final query block + int extra_col_size = (filter_on_handle && !has_handle_column) ? 1 : 0; + for (auto i : query_block.output_offsets) + { + if ((size_t)i >= required_columns.size() - extra_col_size) + { + // array index out of bound + throw Exception("Output offset index is out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); + } + // do not have alias + final_project.emplace_back(required_columns[i], ""); + } + } + // todo handle alias column + if (settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read) + { + throw Exception("Limit for number of columns to read exceeded. " + "Requested: " + + toString(required_columns.size()) + ", maximum: " + settings.max_columns_to_read.toString(), + ErrorCodes::TOO_MANY_COLUMNS); + } + + size_t max_block_size = settings.max_block_size; + max_streams = settings.max_threads; + QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; + if (max_streams > 1) + { + max_streams *= settings.max_streams_to_max_threads_ratio; + } + + if (query_block.selection) + { + for (auto & condition : query_block.selection->selection().conditions()) + { + analyzer->makeExplicitSetForIndex(condition, storage); + } + } + SelectQueryInfo query_info; + // todo double check if it is ok to set it to nullptr + query_info.query = nullptr; + query_info.dag_query = std::make_unique(conditions, analyzer->getPreparedSets(), analyzer->getCurrentInputColumns()); + query_info.mvcc_query_info = std::make_unique(); + query_info.mvcc_query_info->resolve_locks = true; + query_info.mvcc_query_info->read_tso = settings.read_tso; + RegionQueryInfo info; + info.region_id = region_info.region_id; + info.version = region_info.region_version; + info.conf_version = region_info.region_conf_version; + info.range_in_table = current_region->getHandleRangeByTable(table_id); + query_info.mvcc_query_info->regions_query_info.push_back(info); + query_info.mvcc_query_info->concurrent = 0.0; + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + + if (pipeline.streams.empty()) + { + pipeline.streams.emplace_back(std::make_shared(storage->getSampleBlockForColumns(required_columns))); + } + + pipeline.transform([&](auto & stream) { stream->addTableLock(table_lock); }); + + /// Set the limits and quota for reading data, the speed and time of the query. + { + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode); + limits.max_execution_time = settings.max_execution_time; + limits.timeout_overflow_mode = settings.timeout_overflow_mode; + + /** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers, + * because the initiating server has a summary of the execution of the request on all servers. + * + * But limits on data size to read and maximum execution time are reasonable to check both on initiator and + * additionally on each remote server, because these limits are checked per block of data processed, + * and remote servers may process way more blocks of data than are received by initiator. + */ + limits.min_execution_speed = settings.min_execution_speed; + limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; + + QuotaForIntervals & quota = context.getQuota(); + + pipeline.transform([&](auto & stream) { + if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) + { + p_stream->setLimits(limits); + p_stream->setQuota(quota); + } + }); + } + + // only cast the timezone back in the final query block + // todo only add 2 timestamp column if it is the final query block + if (query_block.is_final_query_block && addTimeZoneCastAfterTS(is_ts_column, pipeline)) + { + // for arrow encode, the final select of timestamp column should be column with session timezone + if (keep_session_timezone_info && !query_block.aggregation) + { + for (auto i : query_block.output_offsets) + { + if (is_ts_column[i]) + { + final_project[i].first = analyzer->getCurrentInputColumns()[i].name; + } + } + } + } +} + +// add timezone cast for timestamp type, this is used to support session level timezone +bool InterpreterDAGQueryBlock::addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline) +{ + bool hasTSColumn = false; + for (auto b : is_ts_column) + hasTSColumn |= b; + if (!hasTSColumn) + return false; + + ExpressionActionsChain chain; + if (analyzer->appendTimeZoneCastsAfterTS(chain, is_ts_column, rqst)) + { + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); + return true; + } + else + return false; +} + +AnalysisResult InterpreterDAGQueryBlock::analyzeExpressions() +{ + AnalysisResult res; + ExpressionActionsChain chain; + if (!conditions.empty()) + { + analyzer->appendWhere(chain, conditions, res.filter_column_name); + res.has_where = true; + res.before_where = chain.getLastActions(); + chain.addStep(); + } + // There will be either Agg... + if (query_block.aggregation) + { + // todo need to get stream_agg ??? + analyzer->appendAggregation(chain, query_block.aggregation->aggregation(), res.aggregation_keys, res.aggregate_descriptions); + res.need_aggregate = true; + res.before_aggregation = chain.getLastActions(); + + chain.finalize(); + chain.clear(); + + // add cast if type is not match + analyzer->appendAggSelect( + chain, query_block.aggregation->aggregation(), rqst, keep_session_timezone_info || !query_block.is_final_query_block); + //todo use output_offset to reconstruct the final project columns + if (query_block.is_final_query_block) + { + for (auto & element : analyzer->getCurrentInputColumns()) + { + final_project.emplace_back(element.name, ""); + } + } + } + // Or TopN, not both. + if (query_block.limitOrTopN && query_block.limitOrTopN->tp() == tipb::ExecType::TypeTopN) + { + res.has_order_by = true; + analyzer->appendOrderBy(chain, query_block.limitOrTopN->topn(), res.order_column_names); + } + // Append final project results if needed. + if (query_block.is_final_query_block) + { + analyzer->appendFinalProject(chain, final_project); + res.before_order_and_select = chain.getLastActions(); + } + chain.finalize(); + chain.clear(); + //todo need call prependProjectInput?? + return res; +} + +void InterpreterDAGQueryBlock::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expr, String & filter_column) +{ + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expr, filter_column); }); +} + +void InterpreterDAGQueryBlock::executeAggregation( + Pipeline & pipeline, const ExpressionActionsPtr & expr, Names & key_names, AggregateDescriptions & aggregates) +{ + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expr); }); + + Block header = pipeline.firstStream()->getHeader(); + ColumnNumbers keys; + for (const auto & name : key_names) + { + keys.push_back(header.getPositionByName(name)); + } + for (auto & descr : aggregates) + { + if (descr.arguments.empty()) + { + for (const auto & name : descr.argument_names) + { + descr.arguments.push_back(header.getPositionByName(name)); + } + } + } + + const Settings & settings = context.getSettingsRef(); + + /** Two-level aggregation is useful in two cases: + * 1. Parallel aggregation is done, and the results should be merged in parallel. + * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. + */ + bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0; + + Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, + settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, + allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), + allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), + settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); + + /// If there are several sources, then we perform parallel aggregation + if (pipeline.streams.size() > 1) + { + pipeline.firstStream() = std::make_shared(pipeline.streams, nullptr, params, true, max_streams, + settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) + : static_cast(settings.max_threads)); + + pipeline.streams.resize(1); + } + else + { + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params, true); + } + // add cast +} + +void InterpreterDAGQueryBlock::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr) +{ + if (!expressionActionsPtr->getActions().empty()) + { + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expressionActionsPtr); }); + } +} + +void InterpreterDAGQueryBlock::getAndLockStorageWithSchemaVersion(TableID table_id, Int64 query_schema_version) +{ + /// Get current schema version in schema syncer for a chance to shortcut. + auto global_schema_version = context.getTMTContext().getSchemaSyncer()->getCurrentVersion(); + + /// Lambda for get storage, then align schema version under the read lock. + auto get_and_lock_storage = [&](bool schema_synced) -> std::tuple { + /// Get storage in case it's dropped then re-created. + // If schema synced, call getTable without try, leading to exception on table not existing. + auto storage_ = context.getTMTContext().getStorages().get(table_id); + if (!storage_) + { + if (schema_synced) + throw Exception("Table " + std::to_string(table_id) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + else + return std::make_tuple(nullptr, nullptr, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, false); + } + + if (storage_->engineType() != ::TiDB::StorageEngine::TMT && storage_->engineType() != ::TiDB::StorageEngine::DM) + { + throw Exception("Specifying schema_version for non-managed storage: " + storage_->getName() + + ", table: " + storage_->getTableName() + ", id: " + DB::toString(table_id) + " is not allowed", + ErrorCodes::LOGICAL_ERROR); + } + + /// Lock storage. + auto lock = storage_->lockStructure(false, __PRETTY_FUNCTION__); + + /// Check schema version, requiring TiDB/TiSpark and TiFlash both use exactly the same schema. + // We have three schema versions, two in TiFlash: + // 1. Storage: the version that this TiFlash table (storage) was last altered. + // 2. Global: the version that TiFlash global schema is at. + // And one from TiDB/TiSpark: + // 3. Query: the version that TiDB/TiSpark used for this query. + auto storage_schema_version = storage_->getTableInfo().schema_version; + // Not allow storage > query in any case, one example is time travel queries. + if (storage_schema_version > query_schema_version) + throw Exception("Table " + std::to_string(table_id) + " schema version " + std::to_string(storage_schema_version) + + " newer than query schema version " + std::to_string(query_schema_version), + ErrorCodes::SCHEMA_VERSION_ERROR); + // From now on we have storage <= query. + // If schema was synced, it implies that global >= query, as mentioned above we have storage <= query, we are OK to serve. + if (schema_synced) + return std::make_tuple(storage_, lock, storage_schema_version, true); + // From now on the schema was not synced. + // 1. storage == query, TiDB/TiSpark is using exactly the same schema that altered this table, we are just OK to serve. + // 2. global >= query, TiDB/TiSpark is using a schema older than TiFlash global, but as mentioned above we have storage <= query, + // meaning that the query schema is still newer than the time when this table was last altered, so we still OK to serve. + if (storage_schema_version == query_schema_version || global_schema_version >= query_schema_version) + return std::make_tuple(storage_, lock, storage_schema_version, true); + // From now on we have global < query. + // Return false for outer to sync and retry. + return std::make_tuple(nullptr, nullptr, storage_schema_version, false); + }; + + /// Try get storage and lock once. + ManageableStoragePtr storage_; + TableStructureReadLockPtr lock; + Int64 storage_schema_version; + auto log_schema_version = [&](const String & result) { + LOG_DEBUG(log, + __PRETTY_FUNCTION__ << " Table " << table_id << " schema " << result << " Schema version [storage, global, query]: " + << "[" << storage_schema_version << ", " << global_schema_version << ", " << query_schema_version << "]."); + }; + bool ok; + { + std::tie(storage_, lock, storage_schema_version, ok) = get_and_lock_storage(false); + if (ok) + { + log_schema_version("OK, no syncing required."); + storage = storage_; + table_lock = lock; + return; + } + } + + /// If first try failed, sync schema and try again. + { + log_schema_version("not OK, syncing schemas."); + auto start_time = Clock::now(); + context.getTMTContext().getSchemaSyncer()->syncSchemas(context); + auto schema_sync_cost = std::chrono::duration_cast(Clock::now() - start_time).count(); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << " Table " << table_id << " schema sync cost " << schema_sync_cost << "ms."); + + std::tie(storage_, lock, storage_schema_version, ok) = get_and_lock_storage(true); + if (ok) + { + log_schema_version("OK after syncing."); + storage = storage_; + table_lock = lock; + return; + } + + throw Exception("Shouldn't reach here", ErrorCodes::UNKNOWN_EXCEPTION); + } +} + +SortDescription InterpreterDAGQueryBlock::getSortDescription(Strings & order_column_names) +{ + // construct SortDescription + SortDescription order_descr; + const tipb::TopN & topn = query_block.limitOrTopN->topn(); + order_descr.reserve(topn.order_by_size()); + for (int i = 0; i < topn.order_by_size(); i++) + { + String name = order_column_names[i]; + int direction = topn.order_by(i).desc() ? -1 : 1; + // MySQL/TiDB treats NULL as "minimum". + int nulls_direction = -1; + // todo get this information from DAGRequest + // currently use the default value + std::shared_ptr collator; + + order_descr.emplace_back(name, direction, nulls_direction, collator); + } + return order_descr; +} + +void InterpreterDAGQueryBlock::executeUnion(Pipeline & pipeline) +{ + if (pipeline.hasMoreThanOneStream()) + { + pipeline.firstStream() = std::make_shared>(pipeline.streams, nullptr, max_streams); + pipeline.streams.resize(1); + } +} + +void InterpreterDAGQueryBlock::executeOrder(Pipeline & pipeline, Strings & order_column_names) +{ + SortDescription order_descr = getSortDescription(order_column_names); + const Settings & settings = context.getSettingsRef(); + Int64 limit = query_block.limitOrTopN->topn().limit(); + + pipeline.transform([&](auto & stream) { + auto sorting_stream = std::make_shared(stream, order_descr, limit); + + /// Limits on sorting + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sorting_stream->setLimits(limits); + + stream = sorting_stream; + }); + + /// If there are several streams, we merge them into one + executeUnion(pipeline); + + /// Merge the sorted blocks. + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), order_descr, settings.max_block_size, + limit, settings.max_bytes_before_external_sort, context.getTemporaryPath()); +} + +//void InterpreterDAGQueryBlock::recordProfileStreams(Pipeline & pipeline, Int32 index) +//{ +// for (auto & stream : pipeline.streams) +// { +// dag.getDAGContext().profile_streams_list[index].push_back(stream); +// } +//} + +void InterpreterDAGQueryBlock::executeImpl(Pipeline & pipeline) +{ + if (query_block.source->tp() == tipb::ExecType::TypeJoin) + { + // todo support join + throw Exception("Join is not supported yet", ErrorCodes::NOT_IMPLEMENTED); + } + else + { + executeTS(query_block.source->tbl_scan(), pipeline); + // todo enable profile stream info + //recordProfileStreams(pipeline, dag.getTSIndex()); + + auto res = analyzeExpressions(); + // execute selection + if (res.has_where) + { + executeWhere(pipeline, res.before_where, res.filter_column_name); + //if (dag.hasSelection()) + //recordProfileStreams(pipeline, dag.getSelectionIndex()); + } + if (res.need_aggregate) + { + // execute aggregation + executeAggregation(pipeline, res.before_aggregation, res.aggregation_keys, res.aggregate_descriptions); + //recordProfileStreams(pipeline, dag.getAggregationIndex()); + } + if (res.before_order_and_select) + { + executeExpression(pipeline, res.before_order_and_select); + } + + if (res.has_order_by) + { + // execute topN + executeOrder(pipeline, res.order_column_names); + //recordProfileStreams(pipeline, dag.getTopNIndex()); + } + + if (query_block.is_final_query_block) + { + // execute projection + executeFinalProject(pipeline); + } + + // execute limit + if (query_block.limitOrTopN != nullptr && query_block.limitOrTopN->tp() == tipb::TypeLimit) + { + executeLimit(pipeline); + //recordProfileStreams(pipeline, dag.getLimitIndex()); + } + } +} + +void InterpreterDAGQueryBlock::executeFinalProject(Pipeline & pipeline) +{ + auto columns = pipeline.firstStream()->getHeader(); + NamesAndTypesList input_column; + for (auto & column : columns.getColumnsWithTypeAndName()) + { + input_column.emplace_back(column.name, column.type); + } + ExpressionActionsPtr project = std::make_shared(input_column, context.getSettingsRef()); + project->add(ExpressionAction::project(final_project)); + // add final project + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, project); }); +} + +void InterpreterDAGQueryBlock::executeLimit(Pipeline & pipeline) +{ + size_t limit = 0; + if (query_block.limitOrTopN->tp() == tipb::TypeLimit) + limit = query_block.limitOrTopN->limit().limit(); + else + limit = query_block.limitOrTopN->topn().limit(); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, false); }); + if (pipeline.hasMoreThanOneStream()) + { + executeUnion(pipeline); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, false); }); + } +} + +BlockIO InterpreterDAGQueryBlock::execute() +{ + // todo should not return BlockIO as it will + // union all the streams + Pipeline pipeline; + executeImpl(pipeline); + executeUnion(pipeline); + + BlockIO res; + res.in = pipeline.firstStream(); + return res; +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h new file mode 100644 index 00000000000..dd846ae29af --- /dev/null +++ b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h @@ -0,0 +1,131 @@ +#pragma once + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class Region; +using RegionPtr = std::shared_ptr; + +struct RegionInfo +{ + RegionInfo(RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, + const std::vector> & key_ranges_) + : region_id(region_id_), region_version(region_version_), region_conf_version(region_conf_version_), key_ranges(key_ranges_) + {} + const RegionID region_id; + const UInt64 region_version; + const UInt64 region_conf_version; + const std::vector> & key_ranges; +}; + +struct Pipeline +{ + BlockInputStreams streams; + + BlockInputStreamPtr & firstStream() { return streams.at(0); } + + template + void transform(Transform && transform) + { + for (auto & stream : streams) + transform(stream); + } + + bool hasMoreThanOneStream() const { return streams.size() > 1; } +}; + +struct AnalysisResult +{ + bool has_where = false; + bool need_aggregate = false; + bool has_order_by = false; + + ExpressionActionsPtr before_where; + ExpressionActionsPtr before_aggregation; + ExpressionActionsPtr before_order_and_select; + ExpressionActionsPtr final_projection; + + String filter_column_name; + Strings order_column_names; + /// Columns from the SELECT list, before renaming them to aliases. + Names selected_columns; + + Names aggregation_keys; + AggregateDescriptions aggregate_descriptions; +}; +/** build ch plan from dag request: dag executors -> ch plan + */ +class InterpreterDAGQueryBlock +{ +public: + InterpreterDAGQueryBlock(Context & context_, const BlockInputStreams & input_streams_, const DAGQueryBlock & query_block_, + bool keep_session_timezone_info_, const RegionInfo & region_info, const tipb::DAGRequest & rqst); + + ~InterpreterDAGQueryBlock() = default; + + BlockIO execute(); + +private: + void executeImpl(Pipeline & pipeline); + void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); + void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); + void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); + void executeOrder(Pipeline & pipeline, Strings & order_column_names); + void executeUnion(Pipeline & pipeline); + void executeLimit(Pipeline & pipeline); + void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, Names & aggregation_keys, + AggregateDescriptions & aggregate_descriptions); + void executeFinalProject(Pipeline & pipeline); + void getAndLockStorageWithSchemaVersion(TableID table_id, Int64 schema_version); + SortDescription getSortDescription(Strings & order_column_names); + AnalysisResult analyzeExpressions(); + //void recordProfileStreams(Pipeline & pipeline, Int32 index); + bool addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline); + RegionException::RegionReadStatus getRegionReadStatus(const RegionPtr & current_region); + +private: + Context & context; + BlockInputStreams input_streams; + const DAGQueryBlock & query_block; + const bool keep_session_timezone_info; + const RegionInfo & region_info; + const tipb::DAGRequest & rqst; + + NamesWithAliases final_project; + + /// How many streams we ask for storage to produce, and in how many threads we will do further processing. + size_t max_streams = 1; + + /// Table from where to read data, if not subquery. + ManageableStoragePtr storage; + TableStructureReadLockPtr table_lock; + + std::unique_ptr analyzer; + + + bool filter_on_handle = false; + tipb::Expr handle_filter_expr; + Int32 handle_col_id = -1; + std::vector conditions; + + Poco::Logger * log; +}; +} // namespace DB From fd92e18603fa49c616521c8695503664648b9de5 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 2 Mar 2020 10:37:08 +0800 Subject: [PATCH 06/54] fix bug --- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 4 ++-- dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp | 6 ++++-- dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h | 3 ++- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index a038b4affaf..bedb8fe591e 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -859,13 +859,13 @@ BlockIO InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const Reg input_streams.push_back(child_stream.in); } InterpreterDAGQueryBlock query_block_interpreter(context, input_streams, query_block, - keep_session_timezone_info, region_info, dag.getDAGRequest()); + keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); return query_block_interpreter.execute(); } else { InterpreterDAGQueryBlock query_block_interpreter(context, {}, query_block, - keep_session_timezone_info, region_info, dag.getDAGRequest()); + keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); return query_block_interpreter.execute(); } } diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp index 240824f78f4..04186fa0c3d 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp @@ -43,13 +43,15 @@ extern const int COP_BAD_DAG_REQUEST; } // namespace ErrorCodes InterpreterDAGQueryBlock::InterpreterDAGQueryBlock(Context & context_, const BlockInputStreams & input_streams_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info_, const tipb::DAGRequest & rqst_) + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info_, + const tipb::DAGRequest & rqst_, ASTPtr dummy_query_) : context(context_), input_streams(input_streams_), query_block(query_block_), keep_session_timezone_info(keep_session_timezone_info_), region_info(region_info_), rqst(rqst_), + dummy_query(dummy_query_), log(&Logger::get("InterpreterDAGQueryBlock")) { if (query_block.selection != nullptr) @@ -422,7 +424,7 @@ void InterpreterDAGQueryBlock::executeTS(const tipb::TableScan & ts, Pipeline & } SelectQueryInfo query_info; // todo double check if it is ok to set it to nullptr - query_info.query = nullptr; + query_info.query = dummy_query; query_info.dag_query = std::make_unique(conditions, analyzer->getPreparedSets(), analyzer->getCurrentInputColumns()); query_info.mvcc_query_info = std::make_unique(); query_info.mvcc_query_info->resolve_locks = true; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h index dd846ae29af..7ee93299437 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h @@ -77,7 +77,7 @@ class InterpreterDAGQueryBlock { public: InterpreterDAGQueryBlock(Context & context_, const BlockInputStreams & input_streams_, const DAGQueryBlock & query_block_, - bool keep_session_timezone_info_, const RegionInfo & region_info, const tipb::DAGRequest & rqst); + bool keep_session_timezone_info_, const RegionInfo & region_info, const tipb::DAGRequest & rqst, ASTPtr dummp_query); ~InterpreterDAGQueryBlock() = default; @@ -108,6 +108,7 @@ class InterpreterDAGQueryBlock const bool keep_session_timezone_info; const RegionInfo & region_info; const tipb::DAGRequest & rqst; + ASTPtr dummy_query; NamesWithAliases final_project; From 4bcbc39a39f4aba89907725029b55b9b6144a2b8 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 2 Mar 2020 11:07:23 +0800 Subject: [PATCH 07/54] comment useless code --- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 10 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 110 +++++++++--------- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 24 ++-- dbms/src/Flash/Coprocessor/InterpreterDAG.h | 6 + 4 files changed, 79 insertions(+), 71 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 6608b57924d..fcf4cbf56c3 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -14,11 +14,11 @@ namespace ErrorCodes extern const int COP_BAD_DAG_REQUEST; } // namespace ErrorCodes -const String DAGQuerySource::TS_NAME("tablescan"); -const String DAGQuerySource::SEL_NAME("selection"); -const String DAGQuerySource::AGG_NAME("aggregation"); -const String DAGQuerySource::TOPN_NAME("topN"); -const String DAGQuerySource::LIMIT_NAME("limit"); +//const String DAGQuerySource::TS_NAME("tablescan"); +//const String DAGQuerySource::SEL_NAME("selection"); +//const String DAGQuerySource::AGG_NAME("aggregation"); +//const String DAGQuerySource::TOPN_NAME("topN"); +//const String DAGQuerySource::LIMIT_NAME("limit"); //static void assignOrThrowException(Int32 & index, Int32 value, const String & name) //{ diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index bd7f2192051..a76c116ebcc 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -24,11 +24,11 @@ using TiFlashMetricsPtr = std::shared_ptr; class DAGQuerySource : public IQuerySource { public: - static const String TS_NAME; - static const String SEL_NAME; - static const String AGG_NAME; - static const String TOPN_NAME; - static const String LIMIT_NAME; + //static const String TS_NAME; + //static const String SEL_NAME; + //static const String AGG_NAME; + //static const String TOPN_NAME; + //static const String LIMIT_NAME; DAGQuerySource(Context & context_, DAGContext & dag_context_, RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, const std::vector> & key_ranges_, const tipb::DAGRequest & dag_request_); @@ -37,49 +37,49 @@ class DAGQuerySource : public IQuerySource String str(size_t max_query_size) override; std::unique_ptr interpreter(Context & context, QueryProcessingStage::Enum stage) override; - DAGContext & getDAGContext() const { return dag_context; }; + //DAGContext & getDAGContext() const { return dag_context; }; RegionID getRegionID() const { return region_id; } UInt64 getRegionVersion() const { return region_version; } UInt64 getRegionConfVersion() const { return region_conf_version; } const std::vector> & getKeyRanges() const { return key_ranges; } - bool hasSelection() const { return sel_index != -1; }; - bool hasAggregation() const { return agg_index != -1; }; - bool hasTopN() const { return order_index != -1; }; - bool hasLimit() const { return order_index == -1 && limit_index != -1; }; - - Int32 getTSIndex() const { return ts_index; }; - Int32 getSelectionIndex() const { return sel_index; }; - Int32 getAggregationIndex() const { return agg_index; }; - Int32 getTopNIndex() const { return order_index; }; - Int32 getLimitIndex() const { return limit_index; }; - - const tipb::TableScan & getTS() const - { - assertValid(ts_index, TS_NAME); - return dag_request.executors(ts_index).tbl_scan(); - }; - const tipb::Selection & getSelection() const - { - assertValid(sel_index, SEL_NAME); - return dag_request.executors(sel_index).selection(); - }; - const tipb::Aggregation & getAggregation() const - { - assertValid(agg_index, AGG_NAME); - return dag_request.executors(agg_index).aggregation(); - }; - const tipb::TopN & getTopN() const - { - assertValid(order_index, TOPN_NAME); - return dag_request.executors(order_index).topn(); - }; - const tipb::Limit & getLimit() const - { - assertValid(limit_index, LIMIT_NAME); - return dag_request.executors(limit_index).limit(); - }; + //bool hasSelection() const { return sel_index != -1; }; + //bool hasAggregation() const { return agg_index != -1; }; + //bool hasTopN() const { return order_index != -1; }; + //bool hasLimit() const { return order_index == -1 && limit_index != -1; }; + + //Int32 getTSIndex() const { return ts_index; }; + //Int32 getSelectionIndex() const { return sel_index; }; + //Int32 getAggregationIndex() const { return agg_index; }; + //Int32 getTopNIndex() const { return order_index; }; + //Int32 getLimitIndex() const { return limit_index; }; + + //const tipb::TableScan & getTS() const + //{ + // assertValid(ts_index, TS_NAME); + // return dag_request.executors(ts_index).tbl_scan(); + //}; + //const tipb::Selection & getSelection() const + //{ + // assertValid(sel_index, SEL_NAME); + // return dag_request.executors(sel_index).selection(); + //}; + //const tipb::Aggregation & getAggregation() const + //{ + // assertValid(agg_index, AGG_NAME); + // return dag_request.executors(agg_index).aggregation(); + //}; + //const tipb::TopN & getTopN() const + //{ + // assertValid(order_index, TOPN_NAME); + // return dag_request.executors(order_index).topn(); + //}; + //const tipb::Limit & getLimit() const + //{ + // assertValid(limit_index, LIMIT_NAME); + // return dag_request.executors(limit_index).limit(); + //}; const tipb::DAGRequest & getDAGRequest() const { return dag_request; }; std::vector getResultFieldTypes() const { return result_field_types; }; @@ -87,17 +87,17 @@ class DAGQuerySource : public IQuerySource ASTPtr getAST() const { return ast; }; tipb::EncodeType getEncodeType() const { return encode_type; } - + std::shared_ptr getQueryBlock() const { return query_block_tree; } protected: - void assertValid(Int32 index, const String & name) const - { - if (index < 0 || index > dag_request.executors_size()) - { - throw Exception("Access invalid executor: " + name); - } - } + //void assertValid(Int32 index, const String & name) const + //{ + // if (index < 0 || index > dag_request.executors_size()) + // { + // throw Exception("Access invalid executor: " + name); + // } + //} //void analyzeResultFieldTypes(); void analyzeDAGEncodeType(); @@ -115,11 +115,11 @@ class DAGQuerySource : public IQuerySource TiFlashMetricsPtr metrics; - Int32 ts_index = -1; - Int32 sel_index = -1; - Int32 agg_index = -1; - Int32 order_index = -1; - Int32 limit_index = -1; + //Int32 ts_index = -1; + //Int32 sel_index = -1; + //Int32 agg_index = -1; + //Int32 order_index = -1; + //Int32 limit_index = -1; std::vector result_field_types; tipb::EncodeType encode_type; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index bedb8fe591e..91de228f6d9 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -49,13 +49,16 @@ InterpreterDAG::InterpreterDAG(Context & context_, const DAGQuerySource & dag_) dag.getEncodeType() == tipb::EncodeType::TypeChunk || dag.getEncodeType() == tipb::EncodeType::TypeCHBlock), log(&Logger::get("InterpreterDAG")) { + /* if (dag.hasSelection()) { for (auto & condition : dag.getSelection().conditions()) conditions.push_back(&condition); } + */ } +/* template void constructHandleColRefExpr(tipb::Expr & expr, Int64 col_index) { @@ -455,13 +458,12 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) limits.max_execution_time = settings.max_execution_time; limits.timeout_overflow_mode = settings.timeout_overflow_mode; - /** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers, - * because the initiating server has a summary of the execution of the request on all servers. - * - * But limits on data size to read and maximum execution time are reasonable to check both on initiator and - * additionally on each remote server, because these limits are checked per block of data processed, - * and remote servers may process way more blocks of data than are received by initiator. - */ + /// Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers, + /// because the initiating server has a summary of the execution of the request on all servers. + /// + /// But limits on data size to read and maximum execution time are reasonable to check both on initiator and + /// additionally on each remote server, because these limits are checked per block of data processed, + /// and remote servers may process way more blocks of data than are received by initiator. limits.min_execution_speed = settings.min_execution_speed; limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; @@ -584,10 +586,9 @@ void InterpreterDAG::executeAggregation( const Settings & settings = context.getSettingsRef(); - /** Two-level aggregation is useful in two cases: - * 1. Parallel aggregation is done, and the results should be merged in parallel. - * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. - */ + /// Two-level aggregation is useful in two cases: + /// 1. Parallel aggregation is done, and the results should be merged in parallel. + /// 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0; Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, @@ -847,6 +848,7 @@ void InterpreterDAG::executeLimit(Pipeline & pipeline) [&](auto & stream) { stream = std::make_shared(stream, dag.getLimit().limit(), 0, false); }); } } + */ BlockIO InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info) { diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index 8faf6abc0d5..ebd47b69a87 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -39,6 +39,7 @@ class InterpreterDAG : public IInterpreter private: BlockIO executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info); + /* void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); @@ -55,12 +56,14 @@ class InterpreterDAG : public IInterpreter void recordProfileStreams(Pipeline & pipeline, Int32 index); bool addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline); RegionException::RegionReadStatus getRegionReadStatus(const RegionPtr & current_region); + */ private: Context & context; const DAGQuerySource & dag; + /* NamesWithAliases final_project; /// How many streams we ask for storage to produce, and in how many threads we will do further processing. @@ -71,13 +74,16 @@ class InterpreterDAG : public IInterpreter TableStructureReadLockPtr table_lock; std::unique_ptr analyzer; + */ const bool keep_session_timezone_info; + /* bool filter_on_handle = false; tipb::Expr handle_filter_expr; Int32 handle_col_id = -1; std::vector conditions; + */ Poco::Logger * log; }; From 38eb05ca746b05d4e4276e7bd63a4588d3b4209d Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 4 Mar 2020 15:12:24 +0800 Subject: [PATCH 08/54] refine code --- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 113 +++++++++++------- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 3 +- ...Block.cpp => DAGQueryBlockInterpreter.cpp} | 51 ++++---- ...ueryBlock.h => DAGQueryBlockInterpreter.h} | 13 +- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 12 +- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 39 ++++-- dbms/src/Flash/Coprocessor/InterpreterDAG.h | 8 +- 7 files changed, 141 insertions(+), 98 deletions(-) rename dbms/src/Flash/Coprocessor/{InterpreterDAGQueryBlock.cpp => DAGQueryBlockInterpreter.cpp} (95%) rename dbms/src/Flash/Coprocessor/{InterpreterDAGQueryBlock.h => DAGQueryBlockInterpreter.h} (90%) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index ee85621822a..8b85a45cef3 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -72,15 +72,11 @@ DAGQueryBlock::DAGQueryBlock(const tipb::Executor * root) current = ¤t->selection().child(); break; case tipb::ExecType::TypeAggregation: + case tipb::ExecType::TypeStreamAgg: assignOrThrowException(&aggregation, current, AGG_NAME); collectOutPutFieldTypesFromAgg(output_field_types, current->aggregation()); current = ¤t->aggregation().child(); break; - case tipb::ExecType::TypeStreamAgg: - assignOrThrowException(&aggregation, current, AGG_NAME); - collectOutPutFieldTypesFromAgg(output_field_types, current->stream_agg()); - current = ¤t->stream_agg().child(); - break; case tipb::ExecType::TypeLimit: assignOrThrowException(&limitOrTopN, current, LIMIT_NAME); current = ¤t->limit().child(); @@ -96,40 +92,15 @@ DAGQueryBlock::DAGQueryBlock(const tipb::Executor * root) } } assignOrThrowException(&source, current, SOURCE_NAME); - if (source->tp() == tipb::ExecType::TypeJoin) - { - // todo need to figure out left and right side of the join - children.push_back(std::make_shared(&source->join().probe_exec())); - children.push_back(std::make_shared(&source->join().build_exec())); - if (output_field_types.empty()) - { - for (auto & field_type : children[0]->output_field_types) - output_field_types.push_back(field_type); - for (auto & field_type : children[1]->output_field_types) - output_field_types.push_back(field_type); - } - } - else - { - if (output_field_types.empty()) - { - for (auto & ci : source->tbl_scan().columns()) - { - tipb::FieldType field_type; - field_type.set_tp(ci.tp()); - field_type.set_flag(ci.flag()); - field_type.set_flen(ci.columnlen()); - field_type.set_decimal(ci.decimal()); - output_field_types.push_back(field_type); - } - } - } + fillOutputFieldTypes(); } -DAGQueryBlock::DAGQueryBlock(std::vector & executors) +DAGQueryBlock::DAGQueryBlock(std::vector & executors, int start_index, int end_index) { - for (size_t i = 0; i < executors.size(); i++) + for (int i = end_index; i >= start_index; i--) { + int build_end_index, build_start_index; + int probe_end_index, probe_start_index; switch (executors[i]->tp()) { case tipb::ExecType::TypeTableScan: @@ -139,9 +110,6 @@ DAGQueryBlock::DAGQueryBlock(std::vector & executors) assignOrThrowException(&selection, executors[i], SEL_NAME); break; case tipb::ExecType::TypeStreamAgg: - assignOrThrowException(&aggregation, executors[i], AGG_NAME); - collectOutPutFieldTypesFromAgg(output_field_types, executors[i]->stream_agg()); - break; case tipb::ExecType::TypeAggregation: assignOrThrowException(&aggregation, executors[i], AGG_NAME); collectOutPutFieldTypesFromAgg(output_field_types, executors[i]->aggregation()); @@ -152,20 +120,73 @@ DAGQueryBlock::DAGQueryBlock(std::vector & executors) case tipb::ExecType::TypeLimit: assignOrThrowException(&limitOrTopN, executors[i], LIMIT_NAME); break; + case tipb::ExecType::TypeJoin: + if (i <= start_index) + throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); + if (executors[i - 1] == &(executors[i]->join().build_exec())) + { + build_end_index = i - 1; + build_start_index = build_end_index; + while (build_start_index >= start_index && executors[build_start_index] != &(executors[i]->join().probe_exec())) + build_start_index--; + if (build_start_index < start_index) + throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); + probe_end_index = build_start_index; + build_start_index++; + probe_start_index = start_index; + } + else if (executors[i - 1] == &(executors[i]->join().probe_exec())) + { + probe_end_index = i - 1; + probe_start_index = probe_end_index; + while (probe_start_index >= start_index && executors[probe_start_index] != &(executors[i]->join().build_exec())) + probe_start_index--; + if (probe_start_index < start_index) + throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); + build_end_index = probe_start_index; + probe_start_index++; + build_start_index = start_index; + } + children.push_back(std::make_shared(executors, probe_start_index, probe_end_index)); + children.push_back(std::make_shared(executors, build_start_index, build_end_index)); + // to break the for loop + i = start_index - 1; + break; default: throw Exception("Unsupported executor in DAG request: " + executors[i]->DebugString(), ErrorCodes::NOT_IMPLEMENTED); } } - if (output_field_types.empty()) + fillOutputFieldTypes(); +} + +void DAGQueryBlock::fillOutputFieldTypes() +{ + if (source->tp() == tipb::ExecType::TypeJoin) { - for (auto & ci : source->tbl_scan().columns()) + // todo need to figure out left and right side of the join + children.push_back(std::make_shared(&source->join().probe_exec())); + children.push_back(std::make_shared(&source->join().build_exec())); + if (output_field_types.empty()) { - tipb::FieldType field_type; - field_type.set_tp(ci.tp()); - field_type.set_flag(ci.flag()); - field_type.set_flen(ci.columnlen()); - field_type.set_decimal(ci.decimal()); - output_field_types.push_back(field_type); + for (auto & field_type : children[0]->output_field_types) + output_field_types.push_back(field_type); + for (auto & field_type : children[1]->output_field_types) + output_field_types.push_back(field_type); + } + } + else + { + if (output_field_types.empty()) + { + for (auto & ci : source->tbl_scan().columns()) + { + tipb::FieldType field_type; + field_type.set_tp(ci.tp()); + field_type.set_flag(ci.flag()); + field_type.set_flen(ci.columnlen()); + field_type.set_decimal(ci.decimal()); + output_field_types.push_back(field_type); + } } } } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index 362a57d718b..e28af8ed26f 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -25,7 +25,7 @@ class DAGQueryBlock { public: explicit DAGQueryBlock(const tipb::Executor * root); - explicit DAGQueryBlock(std::vector & executors); + explicit DAGQueryBlock(std::vector & executors, int start_index, int end_index); const tipb::Executor * source = nullptr; const tipb::Executor * selection = nullptr; const tipb::Executor * aggregation = nullptr; @@ -36,6 +36,7 @@ class DAGQueryBlock // kinds of project std::vector output_offsets; bool is_final_query_block; + void fillOutputFieldTypes(); }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp similarity index 95% rename from dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp rename to dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 04186fa0c3d..7fd8719ed73 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -11,10 +11,10 @@ #include #include #include +#include #include #include #include -#include #include #include #include @@ -42,17 +42,17 @@ extern const int UNKNOWN_EXCEPTION; extern const int COP_BAD_DAG_REQUEST; } // namespace ErrorCodes -InterpreterDAGQueryBlock::InterpreterDAGQueryBlock(Context & context_, const BlockInputStreams & input_streams_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info_, - const tipb::DAGRequest & rqst_, ASTPtr dummy_query_) +DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info_, const tipb::DAGRequest & rqst_, + ASTPtr dummy_query_) : context(context_), - input_streams(input_streams_), + input_streams_vec(input_streams_vec_), query_block(query_block_), keep_session_timezone_info(keep_session_timezone_info_), region_info(region_info_), rqst(rqst_), dummy_query(dummy_query_), - log(&Logger::get("InterpreterDAGQueryBlock")) + log(&Logger::get("DAGQueryBlockInterpreter")) { if (query_block.selection != nullptr) { @@ -279,7 +279,7 @@ bool checkKeyRanges(const std::vector> return checkRangeAndGenExprIfNeeded(handle_ranges, region_handle_ranges, handle_col_id, handle_filter); } -RegionException::RegionReadStatus InterpreterDAGQueryBlock::getRegionReadStatus(const RegionPtr & current_region) +RegionException::RegionReadStatus DAGQueryBlockInterpreter::getRegionReadStatus(const RegionPtr & current_region) { if (!current_region) return RegionException::NOT_FOUND; @@ -291,7 +291,7 @@ RegionException::RegionReadStatus InterpreterDAGQueryBlock::getRegionReadStatus( } // the flow is the same as executeFetchcolumns -void InterpreterDAGQueryBlock::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) +void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) { if (!ts.has_table_id()) { @@ -493,7 +493,7 @@ void InterpreterDAGQueryBlock::executeTS(const tipb::TableScan & ts, Pipeline & } // add timezone cast for timestamp type, this is used to support session level timezone -bool InterpreterDAGQueryBlock::addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline) +bool DAGQueryBlockInterpreter::addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline) { bool hasTSColumn = false; for (auto b : is_ts_column) @@ -511,7 +511,7 @@ bool InterpreterDAGQueryBlock::addTimeZoneCastAfterTS(std::vector & is_ts_ return false; } -AnalysisResult InterpreterDAGQueryBlock::analyzeExpressions() +AnalysisResult DAGQueryBlockInterpreter::analyzeExpressions() { AnalysisResult res; ExpressionActionsChain chain; @@ -563,12 +563,12 @@ AnalysisResult InterpreterDAGQueryBlock::analyzeExpressions() return res; } -void InterpreterDAGQueryBlock::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expr, String & filter_column) +void DAGQueryBlockInterpreter::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expr, String & filter_column) { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expr, filter_column); }); } -void InterpreterDAGQueryBlock::executeAggregation( +void DAGQueryBlockInterpreter::executeAggregation( Pipeline & pipeline, const ExpressionActionsPtr & expr, Names & key_names, AggregateDescriptions & aggregates) { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expr); }); @@ -620,7 +620,7 @@ void InterpreterDAGQueryBlock::executeAggregation( // add cast } -void InterpreterDAGQueryBlock::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr) +void DAGQueryBlockInterpreter::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr) { if (!expressionActionsPtr->getActions().empty()) { @@ -628,7 +628,7 @@ void InterpreterDAGQueryBlock::executeExpression(Pipeline & pipeline, const Expr } } -void InterpreterDAGQueryBlock::getAndLockStorageWithSchemaVersion(TableID table_id, Int64 query_schema_version) +void DAGQueryBlockInterpreter::getAndLockStorageWithSchemaVersion(TableID table_id, Int64 query_schema_version) { /// Get current schema version in schema syncer for a chance to shortcut. auto global_schema_version = context.getTMTContext().getSchemaSyncer()->getCurrentVersion(); @@ -725,7 +725,7 @@ void InterpreterDAGQueryBlock::getAndLockStorageWithSchemaVersion(TableID table_ } } -SortDescription InterpreterDAGQueryBlock::getSortDescription(Strings & order_column_names) +SortDescription DAGQueryBlockInterpreter::getSortDescription(Strings & order_column_names) { // construct SortDescription SortDescription order_descr; @@ -746,7 +746,7 @@ SortDescription InterpreterDAGQueryBlock::getSortDescription(Strings & order_col return order_descr; } -void InterpreterDAGQueryBlock::executeUnion(Pipeline & pipeline) +void DAGQueryBlockInterpreter::executeUnion(Pipeline & pipeline) { if (pipeline.hasMoreThanOneStream()) { @@ -755,7 +755,7 @@ void InterpreterDAGQueryBlock::executeUnion(Pipeline & pipeline) } } -void InterpreterDAGQueryBlock::executeOrder(Pipeline & pipeline, Strings & order_column_names) +void DAGQueryBlockInterpreter::executeOrder(Pipeline & pipeline, Strings & order_column_names) { SortDescription order_descr = getSortDescription(order_column_names); const Settings & settings = context.getSettingsRef(); @@ -781,7 +781,7 @@ void InterpreterDAGQueryBlock::executeOrder(Pipeline & pipeline, Strings & order limit, settings.max_bytes_before_external_sort, context.getTemporaryPath()); } -//void InterpreterDAGQueryBlock::recordProfileStreams(Pipeline & pipeline, Int32 index) +//void DAGQueryBlockInterpreter::recordProfileStreams(Pipeline & pipeline, Int32 index) //{ // for (auto & stream : pipeline.streams) // { @@ -789,7 +789,7 @@ void InterpreterDAGQueryBlock::executeOrder(Pipeline & pipeline, Strings & order // } //} -void InterpreterDAGQueryBlock::executeImpl(Pipeline & pipeline) +void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { if (query_block.source->tp() == tipb::ExecType::TypeJoin) { @@ -843,7 +843,7 @@ void InterpreterDAGQueryBlock::executeImpl(Pipeline & pipeline) } } -void InterpreterDAGQueryBlock::executeFinalProject(Pipeline & pipeline) +void DAGQueryBlockInterpreter::executeFinalProject(Pipeline & pipeline) { auto columns = pipeline.firstStream()->getHeader(); NamesAndTypesList input_column; @@ -857,7 +857,7 @@ void InterpreterDAGQueryBlock::executeFinalProject(Pipeline & pipeline) pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, project); }); } -void InterpreterDAGQueryBlock::executeLimit(Pipeline & pipeline) +void DAGQueryBlockInterpreter::executeLimit(Pipeline & pipeline) { size_t limit = 0; if (query_block.limitOrTopN->tp() == tipb::TypeLimit) @@ -872,16 +872,11 @@ void InterpreterDAGQueryBlock::executeLimit(Pipeline & pipeline) } } -BlockIO InterpreterDAGQueryBlock::execute() +BlockInputStreams DAGQueryBlockInterpreter::execute() { - // todo should not return BlockIO as it will - // union all the streams Pipeline pipeline; executeImpl(pipeline); - executeUnion(pipeline); - BlockIO res; - res.in = pipeline.firstStream(); - return res; + return pipeline.streams; } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h similarity index 90% rename from dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h rename to dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 7ee93299437..306057a0e0b 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -73,15 +73,16 @@ struct AnalysisResult }; /** build ch plan from dag request: dag executors -> ch plan */ -class InterpreterDAGQueryBlock +class DAGQueryBlockInterpreter { public: - InterpreterDAGQueryBlock(Context & context_, const BlockInputStreams & input_streams_, const DAGQueryBlock & query_block_, - bool keep_session_timezone_info_, const RegionInfo & region_info, const tipb::DAGRequest & rqst, ASTPtr dummp_query); + DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info, const tipb::DAGRequest & rqst, + ASTPtr dummp_query); - ~InterpreterDAGQueryBlock() = default; + ~DAGQueryBlockInterpreter() = default; - BlockIO execute(); + BlockInputStreams execute(); private: void executeImpl(Pipeline & pipeline); @@ -103,7 +104,7 @@ class InterpreterDAGQueryBlock private: Context & context; - BlockInputStreams input_streams; + std::vector input_streams_vec; const DAGQueryBlock & query_block; const bool keep_session_timezone_info; const RegionInfo & region_info; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index fcf4cbf56c3..1350953c8ae 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -77,18 +77,20 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, Re if (dag_request.has_executor_tree()) { query_block_tree = std::make_shared(&dag_request.executor_tree()); - } else { + } + else + { std::vector executors; - for(const tipb::Executor & executor : dag_request.executors()) + for (const tipb::Executor & executor : dag_request.executors()) executors.push_back(&executor); - query_block_tree = std::make_shared(executors); + query_block_tree = std::make_shared(executors, 0, (int)executors.size() - 1); } query_block_tree->is_final_query_block = true; - for(Int32 i : dag_request.output_offsets()) + for (Int32 i : dag_request.output_offsets()) query_block_tree->output_offsets.push_back(i); if (query_block_tree->aggregation != nullptr) { - for(auto & field_type : query_block_tree->output_field_types) + for (auto & field_type : query_block_tree->output_field_types) result_field_types.push_back(field_type); } else diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 91de228f6d9..866e9e89f3d 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -49,6 +49,12 @@ InterpreterDAG::InterpreterDAG(Context & context_, const DAGQuerySource & dag_) dag.getEncodeType() == tipb::EncodeType::TypeChunk || dag.getEncodeType() == tipb::EncodeType::TypeCHBlock), log(&Logger::get("InterpreterDAG")) { + const Settings & settings = context.getSettingsRef(); + max_streams = settings.max_threads; + if (max_streams > 1) + { + max_streams *= settings.max_streams_to_max_threads_ratio; + } /* if (dag.hasSelection()) { @@ -850,24 +856,33 @@ void InterpreterDAG::executeLimit(Pipeline & pipeline) } */ -BlockIO InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info) +void InterpreterDAG::executeUnion(Pipeline & pipeline) +{ + if (pipeline.hasMoreThanOneStream()) + { + pipeline.firstStream() = std::make_shared>(pipeline.streams, nullptr, max_streams); + pipeline.streams.resize(1); + } +} + +BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info) { if (!query_block.children.empty()) { - BlockInputStreams input_streams; + std::vector input_streams_vec; for (auto & child : query_block.children) { - BlockIO child_stream = executeQueryBlock(*child, region_info); - input_streams.push_back(child_stream.in); + BlockInputStreams child_streams = executeQueryBlock(*child, region_info); + input_streams_vec.push_back(child_streams); } - InterpreterDAGQueryBlock query_block_interpreter(context, input_streams, query_block, - keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); + DAGQueryBlockInterpreter query_block_interpreter( + context, input_streams_vec, query_block, keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); return query_block_interpreter.execute(); } else { - InterpreterDAGQueryBlock query_block_interpreter(context, {}, query_block, - keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); + DAGQueryBlockInterpreter query_block_interpreter( + context, {}, query_block, keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); return query_block_interpreter.execute(); } } @@ -878,8 +893,14 @@ BlockIO InterpreterDAG::execute() /// tidb does not support multi-table dag request yet, so /// it is ok to use the same region_info for the whole dag request RegionInfo region_info(dag.getRegionID(), dag.getRegionVersion(), dag.getRegionConfVersion(), dag.getKeyRanges()); - BlockIO res = executeQueryBlock(*dag.getQueryBlock(), region_info); + BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), region_info); + Pipeline pipeline; + pipeline.streams = streams; + executeUnion(pipeline); + BlockIO res; + res.in = pipeline.firstStream(); + return res; /* LOG_DEBUG( log, __PRETTY_FUNCTION__ << " Convert DAG request to BlockIO, adding " << analyzer->getImplicitCastCount() << " implicit cast"); diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index ebd47b69a87..a4088ece56d 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -8,8 +8,8 @@ #include #include +#include #include -#include #include #include #include @@ -37,8 +37,8 @@ class InterpreterDAG : public IInterpreter BlockIO execute(); private: - - BlockIO executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info); + BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info); + void executeUnion(Pipeline & pipeline); /* void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); @@ -63,6 +63,8 @@ class InterpreterDAG : public IInterpreter const DAGQuerySource & dag; + /// How many streams we ask for storage to produce, and in how many threads we will do further processing. + size_t max_streams = 1; /* NamesWithAliases final_project; From 80c0821cd5ea33a411d2543cc76d88a2350dd85e Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 4 Mar 2020 15:53:45 +0800 Subject: [PATCH 09/54] fix bug --- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 21 ++++++++++++------- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 5 +++-- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 4 ++-- 3 files changed, 18 insertions(+), 12 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index 8b85a45cef3..061a90c618d 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -60,10 +60,11 @@ void collectOutPutFieldTypesFromAgg(std::vector & field_type, c } } -DAGQueryBlock::DAGQueryBlock(const tipb::Executor * root) +DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) +: id(id_) { const tipb::Executor * current = root; - while (isSourceNode(current)) + while (!isSourceNode(current)) { switch (current->tp()) { @@ -92,10 +93,17 @@ DAGQueryBlock::DAGQueryBlock(const tipb::Executor * root) } } assignOrThrowException(&source, current, SOURCE_NAME); + if (current->tp() == tipb::ExecType::TypeJoin) + { + // todo need to figure out left and right side of the join + children.push_back(std::make_shared(id * 2, &source->join().probe_exec())); + children.push_back(std::make_shared(id * 2 + 1, &source->join().build_exec())); + } fillOutputFieldTypes(); } -DAGQueryBlock::DAGQueryBlock(std::vector & executors, int start_index, int end_index) +DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & executors, int start_index, int end_index) +: id(id_) { for (int i = end_index; i >= start_index; i--) { @@ -147,8 +155,8 @@ DAGQueryBlock::DAGQueryBlock(std::vector & executors, in probe_start_index++; build_start_index = start_index; } - children.push_back(std::make_shared(executors, probe_start_index, probe_end_index)); - children.push_back(std::make_shared(executors, build_start_index, build_end_index)); + children.push_back(std::make_shared(id * 2, executors, probe_start_index, probe_end_index)); + children.push_back(std::make_shared(id * 2 + 1, executors, build_start_index, build_end_index)); // to break the for loop i = start_index - 1; break; @@ -163,9 +171,6 @@ void DAGQueryBlock::fillOutputFieldTypes() { if (source->tp() == tipb::ExecType::TypeJoin) { - // todo need to figure out left and right side of the join - children.push_back(std::make_shared(&source->join().probe_exec())); - children.push_back(std::make_shared(&source->join().build_exec())); if (output_field_types.empty()) { for (auto & field_type : children[0]->output_field_types) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index e28af8ed26f..6cc64091d90 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -24,12 +24,13 @@ using TiFlashMetricsPtr = std::shared_ptr; class DAGQueryBlock { public: - explicit DAGQueryBlock(const tipb::Executor * root); - explicit DAGQueryBlock(std::vector & executors, int start_index, int end_index); + DAGQueryBlock(UInt32 id, const tipb::Executor * root); + DAGQueryBlock(UInt32 id, std::vector & executors, int start_index, int end_index); const tipb::Executor * source = nullptr; const tipb::Executor * selection = nullptr; const tipb::Executor * aggregation = nullptr; const tipb::Executor * limitOrTopN = nullptr; + UInt32 id; // todo use unique_ptr instead std::vector> children; std::vector output_field_types; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 1350953c8ae..35df64a0961 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -76,14 +76,14 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, Re */ if (dag_request.has_executor_tree()) { - query_block_tree = std::make_shared(&dag_request.executor_tree()); + query_block_tree = std::make_shared(1, &dag_request.executor_tree()); } else { std::vector executors; for (const tipb::Executor & executor : dag_request.executors()) executors.push_back(&executor); - query_block_tree = std::make_shared(executors, 0, (int)executors.size() - 1); + query_block_tree = std::make_shared(1, executors, 0, (int)executors.size() - 1); } query_block_tree->is_final_query_block = true; for (Int32 i : dag_request.output_offsets()) From 16794677a0e5049c0d65508ed5e6e2ea65597665 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 4 Mar 2020 19:24:10 +0800 Subject: [PATCH 10/54] refine code --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 7 +- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 3 +- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 21 +++-- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 3 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 87 ++++++++++--------- .../Coprocessor/DAGQueryBlockInterpreter.h | 1 + dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 24 ++--- 7 files changed, 80 insertions(+), 66 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 4952db0c3e0..93d3b695e87 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -270,7 +270,8 @@ String DAGExpressionAnalyzer::appendTimeZoneCast( // column to the columns with session-level timezone info, the original ts columns with UTC timezone // are still kept, and the InterpreterDAG will choose the correct column based on encode type bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( - ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst) + ExpressionActionsChain & chain, std::vector is_ts_column, + const tipb::DAGRequest & rqst, bool keep_UTC_column) { if (!hasMeaningfulTZInfo(rqst)) return false; @@ -290,8 +291,8 @@ bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( if (tz_col.length() == 0) tz_col = getActions(tz_expr, actions); String casted_name = appendTimeZoneCast(tz_col, source_columns[i].name, func_name, actions); - // todo only append the original colum if keep_session_timezone is false - source_columns.emplace_back(source_columns[i].name, source_columns[i].type); + if (keep_UTC_column) + source_columns.emplace_back(source_columns[i].name, source_columns[i].type); source_columns[i].name = casted_name; ret = true; } diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index fb38e696ce4..1d51cb66005 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -66,7 +66,8 @@ class DAGExpressionAnalyzer : private boost::noncopyable void makeExplicitSetForIndex(const tipb::Expr & expr, const ManageableStoragePtr & storage); String applyFunction(const String & func_name, const Names & arg_names, ExpressionActionsPtr & actions); Int32 getImplicitCastCount() { return implicit_cast_count; }; - bool appendTimeZoneCastsAfterTS(ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst); + bool appendTimeZoneCastsAfterTS(ExpressionActionsChain & chain, std::vector is_ts_column, + const tipb::DAGRequest & rqst, bool keep_UTC_column); String appendTimeZoneCast(const String & tz_col, const String & ts_col, const String & func_name, ExpressionActionsPtr & actions); DAGPreparedSets getPreparedSets() { return prepared_sets; } String convertToUInt8ForFilter(ExpressionActionsChain & chain, const String & column_name); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index 061a90c618d..db9d3a7289a 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -61,7 +61,7 @@ void collectOutPutFieldTypesFromAgg(std::vector & field_type, c } DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) -: id(id_) +: id(id_), qb_column_prefix("___QB_" + std::to_string(id_)) { const tipb::Executor * current = root; while (!isSourceNode(current)) @@ -95,20 +95,21 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) assignOrThrowException(&source, current, SOURCE_NAME); if (current->tp() == tipb::ExecType::TypeJoin) { - // todo need to figure out left and right side of the join - children.push_back(std::make_shared(id * 2, &source->join().probe_exec())); - children.push_back(std::make_shared(id * 2 + 1, &source->join().build_exec())); + if (source->join().children_size() != 2) + throw Exception("Join executor children size not equal to 2field type", ErrorCodes::COP_BAD_DAG_REQUEST); + children.push_back(std::make_shared(id * 2, &source->join().children(0))); + children.push_back(std::make_shared(id * 2 + 1, &source->join().children(1))); } fillOutputFieldTypes(); } DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & executors, int start_index, int end_index) -: id(id_) +: id(id_), qb_column_prefix("___QB_" + std::to_string(id_)) { for (int i = end_index; i >= start_index; i--) { - int build_end_index, build_start_index; - int probe_end_index, probe_start_index; + //int build_end_index, build_start_index; + //int probe_end_index, probe_start_index; switch (executors[i]->tp()) { case tipb::ExecType::TypeTableScan: @@ -128,6 +129,7 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & e case tipb::ExecType::TypeLimit: assignOrThrowException(&limitOrTopN, executors[i], LIMIT_NAME); break; + /* case tipb::ExecType::TypeJoin: if (i <= start_index) throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); @@ -155,11 +157,16 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & e probe_start_index++; build_start_index = start_index; } + else + { + throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); + } children.push_back(std::make_shared(id * 2, executors, probe_start_index, probe_end_index)); children.push_back(std::make_shared(id * 2 + 1, executors, build_start_index, build_end_index)); // to break the for loop i = start_index - 1; break; + */ default: throw Exception("Unsupported executor in DAG request: " + executors[i]->DebugString(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index 6cc64091d90..ab3d7ce1002 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -31,13 +31,14 @@ class DAGQueryBlock const tipb::Executor * aggregation = nullptr; const tipb::Executor * limitOrTopN = nullptr; UInt32 id; + String qb_column_prefix; // todo use unique_ptr instead std::vector> children; std::vector output_field_types; // kinds of project std::vector output_offsets; - bool is_final_query_block; void fillOutputFieldTypes(); + bool isRootQueryBlock() const { return id == 1; }; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 7fd8719ed73..07dbfde5697 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -383,19 +383,27 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & analyzer = std::make_unique(std::move(source_columns), context); - if (query_block.aggregation == nullptr && query_block.is_final_query_block) + if (query_block.aggregation == nullptr) { - // only add final project if the query block is the final query block int extra_col_size = (filter_on_handle && !has_handle_column) ? 1 : 0; - for (auto i : query_block.output_offsets) + if (query_block.isRootQueryBlock()) { - if ((size_t)i >= required_columns.size() - extra_col_size) + for (auto i : query_block.output_offsets) { - // array index out of bound - throw Exception("Output offset index is out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); + if ((size_t) i >= required_columns.size() - extra_col_size) + { + // array index out of bound + throw Exception("Output offset index is out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); + } + // do not have alias + final_project.emplace_back(required_columns[i], ""); } - // do not have alias - final_project.emplace_back(required_columns[i], ""); + } + else + { + for(size_t i = 0; i < required_columns.size() - extra_col_size; i++) + /// for child query block, add alias start with qb_column_prefix to avoid column name conflict + final_project.emplace_back(required_columns[i], query_block.qb_column_prefix + required_columns[i]); } } // todo handle alias column @@ -423,7 +431,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & } } SelectQueryInfo query_info; - // todo double check if it is ok to set it to nullptr + /// to avoid null point exception query_info.query = dummy_query; query_info.dag_query = std::make_unique(conditions, analyzer->getPreparedSets(), analyzer->getCurrentInputColumns()); query_info.mvcc_query_info = std::make_unique(); @@ -474,22 +482,13 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & }); } - // only cast the timezone back in the final query block - // todo only add 2 timestamp column if it is the final query block - if (query_block.is_final_query_block && addTimeZoneCastAfterTS(is_ts_column, pipeline)) - { - // for arrow encode, the final select of timestamp column should be column with session timezone - if (keep_session_timezone_info && !query_block.aggregation) - { - for (auto i : query_block.output_offsets) - { - if (is_ts_column[i]) - { - final_project[i].first = analyzer->getCurrentInputColumns()[i].name; - } - } - } - } + addTimeZoneCastAfterTS(is_ts_column, pipeline); +} + +void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & , Pipeline & ) +{ + // todo support join + throw Exception("Join is not supported yet", ErrorCodes::NOT_IMPLEMENTED); } // add timezone cast for timestamp type, this is used to support session level timezone @@ -502,7 +501,12 @@ bool DAGQueryBlockInterpreter::addTimeZoneCastAfterTS(std::vector & is_ts_ return false; ExpressionActionsChain chain; - if (analyzer->appendTimeZoneCastsAfterTS(chain, is_ts_column, rqst)) + /// only keep UTC column if + /// 1. the query block is the root query block + /// 2. keep_session_timezone_info is false + /// 3. current query block does not have aggregation + if (analyzer->appendTimeZoneCastsAfterTS(chain, is_ts_column, rqst, + query_block.isRootQueryBlock() && !keep_session_timezone_info && query_block.aggregation == nullptr)) { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); return true; @@ -525,7 +529,6 @@ AnalysisResult DAGQueryBlockInterpreter::analyzeExpressions() // There will be either Agg... if (query_block.aggregation) { - // todo need to get stream_agg ??? analyzer->appendAggregation(chain, query_block.aggregation->aggregation(), res.aggregation_keys, res.aggregate_descriptions); res.need_aggregate = true; res.before_aggregation = chain.getLastActions(); @@ -535,15 +538,22 @@ AnalysisResult DAGQueryBlockInterpreter::analyzeExpressions() // add cast if type is not match analyzer->appendAggSelect( - chain, query_block.aggregation->aggregation(), rqst, keep_session_timezone_info || !query_block.is_final_query_block); - //todo use output_offset to reconstruct the final project columns - if (query_block.is_final_query_block) + chain, query_block.aggregation->aggregation(), rqst, keep_session_timezone_info || !query_block.isRootQueryBlock()); + if (query_block.isRootQueryBlock()) { + // todo for root query block, use output offsets to reconstruct the final project for (auto & element : analyzer->getCurrentInputColumns()) { final_project.emplace_back(element.name, ""); } } + else + { + for (auto & element : analyzer->getCurrentInputColumns()) + { + final_project.emplace_back(element.name, query_block.qb_column_prefix + element.name); + } + } } // Or TopN, not both. if (query_block.limitOrTopN && query_block.limitOrTopN->tp() == tipb::ExecType::TypeTopN) @@ -552,11 +562,8 @@ AnalysisResult DAGQueryBlockInterpreter::analyzeExpressions() analyzer->appendOrderBy(chain, query_block.limitOrTopN->topn(), res.order_column_names); } // Append final project results if needed. - if (query_block.is_final_query_block) - { - analyzer->appendFinalProject(chain, final_project); - res.before_order_and_select = chain.getLastActions(); - } + analyzer->appendFinalProject(chain, final_project); + res.before_order_and_select = chain.getLastActions(); chain.finalize(); chain.clear(); //todo need call prependProjectInput?? @@ -793,8 +800,7 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { if (query_block.source->tp() == tipb::ExecType::TypeJoin) { - // todo support join - throw Exception("Join is not supported yet", ErrorCodes::NOT_IMPLEMENTED); + executeJoin(query_block.source->join(), pipeline); } else { @@ -828,11 +834,8 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) //recordProfileStreams(pipeline, dag.getTopNIndex()); } - if (query_block.is_final_query_block) - { - // execute projection - executeFinalProject(pipeline); - } + // execute projection + executeFinalProject(pipeline); // execute limit if (query_block.limitOrTopN != nullptr && query_block.limitOrTopN->tp() == tipb::TypeLimit) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 306057a0e0b..0363f85a386 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -87,6 +87,7 @@ class DAGQueryBlockInterpreter private: void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); + void executeJoin(const tipb::Join & join, Pipeline & pipeline); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); void executeOrder(Pipeline & pipeline, Strings & order_column_names); diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 35df64a0961..77fbd4d1664 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -74,18 +74,18 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, Re } } */ - if (dag_request.has_executor_tree()) - { - query_block_tree = std::make_shared(1, &dag_request.executor_tree()); - } - else - { - std::vector executors; - for (const tipb::Executor & executor : dag_request.executors()) - executors.push_back(&executor); - query_block_tree = std::make_shared(1, executors, 0, (int)executors.size() - 1); - } - query_block_tree->is_final_query_block = true; + //if (dag_request.has_executor_tree()) + //{ + // query_block_tree = std::make_shared(1, &dag_request.executor_tree()); + //} + //else + //{ + // std::vector executors; + // for (const tipb::Executor & executor : dag_request.executors()) + // executors.push_back(&executor); + // query_block_tree = std::make_shared(1, executors, 0, (int)executors.size() - 1); + //} + query_block_tree = std::make_shared(1, &dag_request.executors(0)); for (Int32 i : dag_request.output_offsets()) query_block_tree->output_offsets.push_back(i); if (query_block_tree->aggregation != nullptr) From 3137f0a70dfc32828d98e5f25c623550d1f50685 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 5 Mar 2020 08:54:32 +0800 Subject: [PATCH 11/54] fix bug --- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 4 ++-- dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 9 ++++++++- dbms/src/Flash/Coprocessor/DAGStringConverter.cpp | 2 -- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index db9d3a7289a..8ac07957de4 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -61,7 +61,7 @@ void collectOutPutFieldTypesFromAgg(std::vector & field_type, c } DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) -: id(id_), qb_column_prefix("___QB_" + std::to_string(id_)) +: id(id_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") { const tipb::Executor * current = root; while (!isSourceNode(current)) @@ -104,7 +104,7 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) } DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & executors, int start_index, int end_index) -: id(id_), qb_column_prefix("___QB_" + std::to_string(id_)) +: id(id_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") { for (int i = end_index; i >= start_index; i--) { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 07dbfde5697..6600fc0635a 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -444,7 +444,14 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); query_info.mvcc_query_info->concurrent = 0.0; - pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + //if (ts.engine() == 0) { + // pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, + // max_streams); + //} else { + //pipeline.streams = storage->remote_read(dag.getKeyRanges(), query_info, ts, context); + //} + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, + max_streams); if (pipeline.streams.empty()) { diff --git a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp index 7cb3ad33394..48c91964120 100644 --- a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp @@ -153,8 +153,6 @@ void DAGStringConverter::buildString(const tipb::Executor & executor, std::strin return buildTopNString(executor.topn(), ss); case tipb::ExecType::TypeLimit: return buildLimitString(executor.limit(), ss); - case tipb::ExecType::TypeJoin: - throw Exception("join has not implemented"); } } From b346aeb2e0ad72cb8f641423452321554bd2a4cc Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 5 Mar 2020 12:42:09 +0800 Subject: [PATCH 12/54] save work --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 27 +++++++ .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 1 + .../Coprocessor/DAGQueryBlockInterpreter.cpp | 80 ++++++++++++++++++- .../Coprocessor/DAGQueryBlockInterpreter.h | 4 +- 4 files changed, 108 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 93d3b695e87..a468c23da98 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -300,6 +300,33 @@ bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( return ret; } +/// return true if same actions is needed +bool DAGExpressionAnalyzer::appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left) +{ + bool ret = false; + initChain(chain, getCurrentInputColumns()); + ExpressionActionsPtr actions = chain.getLastActions(); + if ((tiflash_left && join.inner_idx() == 1) || (!tiflash_left && join.inner_idx() == 0)) + { + for(const auto & key : join.left_join_keys()) + { + key_names.push_back(getActions(key, actions)); + if (key.tp() != tipb::ExprType::ColumnRef) + ret = true; + } + } + else + { + for(const auto & key : join.right_join_keys()) + { + key_names.push_back(getActions(key, actions)); + if (key.tp() != tipb::ExprType::ColumnRef) + ret = true; + } + } + return ret; +} + void DAGExpressionAnalyzer::appendAggSelect( ExpressionActionsChain & chain, const tipb::Aggregation & aggregation, const tipb::DAGRequest & rqst, bool keep_session_timezone_info) { diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 1d51cb66005..151227e26f8 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -68,6 +68,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable Int32 getImplicitCastCount() { return implicit_cast_count; }; bool appendTimeZoneCastsAfterTS(ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst, bool keep_UTC_column); + bool appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left); String appendTimeZoneCast(const String & tz_col, const String & ts_col, const String & func_name, ExpressionActionsPtr & actions); DAGPreparedSets getPreparedSets() { return prepared_sets; } String convertToUInt8ForFilter(ExpressionActionsChain & chain, const String & column_name); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 6600fc0635a..d8c295fc24e 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,8 @@ #include #include #include +#include +#include namespace DB { @@ -492,9 +495,79 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & addTimeZoneCastAfterTS(is_ts_column, pipeline); } -void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & , Pipeline & ) +void DAGQueryBlockInterpreter::prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left) { - // todo support join + std::vector source_columns; + for( auto const & p : pipeline.firstStream()->getHeader().getNamesAndTypesList()) + source_columns.emplace_back(p.name, p.type); + DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); + ExpressionActionsChain chain; + if (dag_analyzer.appendJoinKey(chain, join, key_names, tiflash_left)) + { + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); + } +} +void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & , SubqueryForSet & right_query) +{ + // build + static const std::unordered_map join_type_map{ + {tipb::JoinType::TypeInnerJoin, ASTTableJoin::Kind::Inner}, + {tipb::JoinType::TypeLeftOuterJoin, ASTTableJoin::Kind::Left}, + {tipb::JoinType::TypeRightOuterJoin, ASTTableJoin::Kind::Right} + }; + if (input_streams_vec.size() != 2) + { + throw Exception("Join query block must have 2 input streams", ErrorCodes::LOGICAL_ERROR); + } + + auto join_type_it = join_type_map.find(join.join_type()); + if (join_type_it == join_type_map.end()) + throw Exception("Unknown join type in dag request", ErrorCodes::COP_BAD_DAG_REQUEST); + ASTTableJoin::Kind kind = join_type_it->second; + BlockInputStreams left_streams; + BlockInputStreams right_streams; + Names left_key_names; + Names right_key_names; + if (join.inner_idx() == 0) + { + // in DAG request, inner part is the build side, however for tiflash implementation, + // the build side must be the right side, so need to update the join type if needed + if (kind == ASTTableJoin::Kind::Left) + kind = ASTTableJoin::Kind::Right; + else if (kind == ASTTableJoin::Kind::Right) + kind = ASTTableJoin::Kind::Left; + left_streams = input_streams_vec[1]; + right_streams = input_streams_vec[0]; + } + else + { + left_streams = input_streams_vec[0]; + right_streams = input_streams_vec[1]; + } + + /// add necessary transformation if the join key is an expression + Pipeline left_pipeline; + left_pipeline.streams = left_streams; + prepareJoinKeys(join, left_pipeline, left_key_names, true); + Pipeline right_pipeline; + right_pipeline.streams = right_streams; + prepareJoinKeys(join, right_pipeline, right_key_names, false); + + left_streams = left_pipeline.streams; + right_streams = right_pipeline.streams; + + const Settings & settings = context.getSettingsRef(); + JoinPtr joinPtr = std::make_shared(left_key_names, right_key_names, true, + SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), + kind, ASTTableJoin::Strictness::All); + executeUnion(right_pipeline); + // todo clickhouse use LazyBlockInputStream to initialize the source, need to double check why and + // how to use LazyBlockInputStream here + right_query.source = right_pipeline.firstStream(); + right_query.join = joinPtr; + right_query.join->setSampleBlock(right_query.source->getHeader()); + + throw Exception("Join is not supported yet", ErrorCodes::NOT_IMPLEMENTED); } @@ -807,7 +880,8 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { if (query_block.source->tp() == tipb::ExecType::TypeJoin) { - executeJoin(query_block.source->join(), pipeline); + SubqueryForSet right_query; + executeJoin(query_block.source->join(), pipeline, right_query); } else { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 0363f85a386..31eac9216a0 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -87,7 +88,8 @@ class DAGQueryBlockInterpreter private: void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); - void executeJoin(const tipb::Join & join, Pipeline & pipeline); + void executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query); + void prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); void executeOrder(Pipeline & pipeline, Strings & order_column_names); From 602412bfa0779abdcadf488fc135ead14b12cecc Mon Sep 17 00:00:00 2001 From: Fei Han Date: Fri, 6 Mar 2020 18:43:18 +0800 Subject: [PATCH 13/54] tiny fix --- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 6598896fd24..4722ac1caed 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -432,7 +432,7 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); query_info.mvcc_query_info->concurrent = 0.0; - if (ts.engine() == 0) { + if (ts.next_read_engine() == tipb::EngineType::Local) { pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); } else { From c2f80ab002a077abd88d79cd97733d033cec1cb6 Mon Sep 17 00:00:00 2001 From: xufei Date: Sat, 7 Mar 2020 11:44:33 +0800 Subject: [PATCH 14/54] save work --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 33 ++++++++++- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 2 + .../Coprocessor/DAGQueryBlockInterpreter.cpp | 57 +++++++++++++++++-- .../Coprocessor/DAGQueryBlockInterpreter.h | 1 + 4 files changed, 86 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index a468c23da98..2d7c2954c11 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -300,28 +300,55 @@ bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( return ret; } -/// return true if same actions is needed +void DAGExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, SubqueryForSet & join_query, const NamesAndTypesList & columns_added_by_join) +{ + initChain(chain, getCurrentInputColumns()); + ExpressionActionsPtr actions = chain.getLastActions(); + actions->add(ExpressionAction::ordinaryJoin(join_query.join, columns_added_by_join)); +} +/// return true if some actions is needed bool DAGExpressionAnalyzer::appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left) { bool ret = false; initChain(chain, getCurrentInputColumns()); ExpressionActionsPtr actions = chain.getLastActions(); + NamesWithAliases right_key_dummy_production; if ((tiflash_left && join.inner_idx() == 1) || (!tiflash_left && join.inner_idx() == 0)) { for(const auto & key : join.left_join_keys()) { - key_names.push_back(getActions(key, actions)); if (key.tp() != tipb::ExprType::ColumnRef) ret = true; + if (!tiflash_left && key.tp() == tipb::ExprType::ColumnRef) + { + // for right side table, add a project + String org_key_name = getActions(key, actions); + String updated_key_name = "_r_k_" + org_key_name; + actions->add(ExpressionAction::copyColumn(org_key_name, updated_key_name)); + key_names.push_back(updated_key_name); + ret = true; + } + else + key_names.push_back(getActions(key, actions)); } } else { for(const auto & key : join.right_join_keys()) { - key_names.push_back(getActions(key, actions)); if (key.tp() != tipb::ExprType::ColumnRef) ret = true; + if (!tiflash_left && key.tp() == tipb::ExprType::ColumnRef) + { + // for right side table, add a project + String org_key_name = getActions(key, actions); + String updated_key_name = "_r_k_" + org_key_name; + actions->add(ExpressionAction::copyColumn(org_key_name, updated_key_name)); + key_names.push_back(updated_key_name); + ret = true; + } + else + key_names.push_back(getActions(key, actions)); } } return ret; diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 151227e26f8..2502fb4d3eb 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -58,6 +59,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable chain.steps.emplace_back(std::make_shared(column_list, settings)); } } + void appendJoin(ExpressionActionsChain & chain, SubqueryForSet & join_query, const NamesAndTypesList & columns_added_by_join); void appendFinalProject(ExpressionActionsChain & chain, const NamesWithAliases & final_project); String getActions(const tipb::Expr & expr, ExpressionActionsPtr & actions); String getActionsForInOperator(const tipb::Expr & expr, ExpressionActionsPtr & actions); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index d8c295fc24e..2f11d61c309 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -32,6 +32,7 @@ #include #include #include +#include namespace DB { @@ -507,7 +508,7 @@ void DAGQueryBlockInterpreter::prepareJoinKeys(const tipb::Join & join, Pipeline pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); } } -void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & , SubqueryForSet & right_query) +void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query) { // build static const std::unordered_map join_type_map{ @@ -524,6 +525,12 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & , if (join_type_it == join_type_map.end()) throw Exception("Unknown join type in dag request", ErrorCodes::COP_BAD_DAG_REQUEST); ASTTableJoin::Kind kind = join_type_it->second; + if (kind != ASTTableJoin::Kind::Inner) + { + // todo support left and right join + throw Exception("Only Inner join is supported", ErrorCodes::NOT_IMPLEMENTED); + } + BlockInputStreams left_streams; BlockInputStreams right_streams; Names left_key_names; @@ -544,6 +551,22 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & , left_streams = input_streams_vec[0]; right_streams = input_streams_vec[1]; } + std::vector join_output_columns; + for( auto const & p : left_streams[0]->getHeader().getNamesAndTypesList()) + { + join_output_columns.emplace_back(p.name, p.type); + if (!query_block.aggregation) + final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); + } + /// all the columns from right table should be added after join, even for the join key + NamesAndTypesList columns_added_by_join; + for( auto const & p : right_streams[0]->getHeader().getNamesAndTypesList()) + { + columns_added_by_join.emplace_back(p.name, p.type); + join_output_columns.emplace_back(p.name, p.type); + if (!query_block.aggregation) + final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); + } /// add necessary transformation if the join key is an expression Pipeline left_pipeline; @@ -567,8 +590,18 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & , right_query.join = joinPtr; right_query.join->setSampleBlock(right_query.source->getHeader()); + std::vector source_columns; + for(const auto & p : left_streams[0]->getHeader().getNamesAndTypesList()) + source_columns.emplace_back(p.name, p.type); + DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); + ExpressionActionsChain chain; + dag_analyzer.appendJoin(chain, right_query, columns_added_by_join); + pipeline.streams = left_streams; + for (auto & stream : pipeline.streams) + stream = std::make_shared(stream, chain.getLastActions()); - throw Exception("Join is not supported yet", ErrorCodes::NOT_IMPLEMENTED); + // todo should add a project here??? + analyzer = std::make_unique(std::move(join_output_columns), context); } // add timezone cast for timestamp type, this is used to support session level timezone @@ -876,16 +909,28 @@ void DAGQueryBlockInterpreter::executeOrder(Pipeline & pipeline, Strings & order // } //} +void DAGQueryBlockInterpreter::executeSubqueryInJoin(Pipeline & pipeline, SubqueryForSet & subquery) +{ + const Settings & settings = context.getSettingsRef(); + executeUnion(pipeline); + SubqueriesForSets subquries; + subquries[query_block.qb_column_prefix + "join"] = subquery; + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), subquries, + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); +} + void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { + SubqueryForSet right_query; if (query_block.source->tp() == tipb::ExecType::TypeJoin) { - SubqueryForSet right_query; executeJoin(query_block.source->join(), pipeline, right_query); } else { executeTS(query_block.source->tbl_scan(), pipeline); + } // todo enable profile stream info //recordProfileStreams(pipeline, dag.getTSIndex()); @@ -924,7 +969,11 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) executeLimit(pipeline); //recordProfileStreams(pipeline, dag.getLimitIndex()); } - } + if (query_block.source->tp() == tipb::ExecType::TypeJoin) + { + // add the + executeSubqueryInJoin(pipeline, right_query); + } } void DAGQueryBlockInterpreter::executeFinalProject(Pipeline & pipeline) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 31eac9216a0..a9736f58701 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -90,6 +90,7 @@ class DAGQueryBlockInterpreter void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query); void prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left); + void executeSubqueryInJoin(Pipeline & pipeline, SubqueryForSet & subquery); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); void executeOrder(Pipeline & pipeline, Strings & order_column_names); From 944515c52fdf634ecf41287e03c4d85845e0bcd4 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 9 Mar 2020 10:29:58 +0800 Subject: [PATCH 15/54] support remote read --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 2f11d61c309..a27c337825b 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -448,14 +448,23 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); query_info.mvcc_query_info->concurrent = 0.0; - //if (ts.engine() == 0) { - // pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, - // max_streams); - //} else { - //pipeline.streams = storage->remote_read(dag.getKeyRanges(), query_info, ts, context); - //} - pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, - max_streams); + if (ts.next_read_engine() == tipb::EngineType::Local) { + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, + max_streams); + } else { + std::vector> key_ranges; + for (auto & range : ts.ranges()) + { + std::string start_key(range.low()); + DecodedTiKVKey start(std::move(start_key)); + std::string end_key(range.high()); + DecodedTiKVKey end(std::move(end_key)); + key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); + } + pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); + } + //pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, + // max_streams); if (pipeline.streams.empty()) { From e8395ef91af7908138deb63169f59be447db1bd8 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 9 Mar 2020 10:30:17 +0800 Subject: [PATCH 16/54] update tipb --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index aa600db6e32..5f3b53798f0 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit aa600db6e32e9612d031bbbce2b51a744b132ba5 +Subproject commit 5f3b53798f0099d9016c9cb0dca1a04fdeac8116 From 2299220f633be87778a14d278901196ff96f8039 Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 10 Mar 2020 12:21:01 +0800 Subject: [PATCH 17/54] refine code --- dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index a27c337825b..08778372121 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -55,7 +55,7 @@ DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std keep_session_timezone_info(keep_session_timezone_info_), region_info(region_info_), rqst(rqst_), - dummy_query(dummy_query_), + dummy_query(std::move(dummy_query_)), log(&Logger::get("DAGQueryBlockInterpreter")) { if (query_block.selection != nullptr) @@ -594,7 +594,7 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p kind, ASTTableJoin::Strictness::All); executeUnion(right_pipeline); // todo clickhouse use LazyBlockInputStream to initialize the source, need to double check why and - // how to use LazyBlockInputStream here + // if we need to use LazyBlockInputStream here right_query.source = right_pipeline.firstStream(); right_query.join = joinPtr; right_query.join->setSampleBlock(right_query.source->getHeader()); From 3bf762f78e57e23cdd99645c9d7f1be9ff3bef7e Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 10 Mar 2020 18:02:40 +0800 Subject: [PATCH 18/54] fix bug --- .../Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 08778372121..cb80ff2e7a4 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -463,8 +463,6 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & } pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); } - //pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, - // max_streams); if (pipeline.streams.empty()) { @@ -564,8 +562,6 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p for( auto const & p : left_streams[0]->getHeader().getNamesAndTypesList()) { join_output_columns.emplace_back(p.name, p.type); - if (!query_block.aggregation) - final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); } /// all the columns from right table should be added after join, even for the join key NamesAndTypesList columns_added_by_join; @@ -573,7 +569,13 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p { columns_added_by_join.emplace_back(p.name, p.type); join_output_columns.emplace_back(p.name, p.type); - if (!query_block.aggregation) + } + + if (!query_block.aggregation) + { + for(auto const & p : input_streams_vec[0][0]->getHeader().getNamesAndTypesList()) + final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); + for(auto const & p : input_streams_vec[1][0]->getHeader().getNamesAndTypesList()) final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); } From 50644ecf9992f4339bab8d85804ac437bc001bd5 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 10 Mar 2020 21:58:27 +0800 Subject: [PATCH 19/54] update client-c --- contrib/client-c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/client-c b/contrib/client-c index 891ea87a882..effedfb8a8f 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 891ea87a882fc2fa8a6c1762389200dbe24040a4 +Subproject commit effedfb8a8f2a2dd8a1ba262e31e4a4742004b9c From b02240033d5dc5e32680d70e3842e9b834cc5bab Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 11 Mar 2020 17:16:48 +0800 Subject: [PATCH 20/54] refine cop read --- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 80 ++++---- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 7 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 187 +++++++++++------- .../Coprocessor/DAGQueryBlockInterpreter.h | 3 +- dbms/src/Storages/IManageableStorage.cpp | 119 +++++++++++ dbms/src/Storages/IManageableStorage.h | 40 +--- 6 files changed, 283 insertions(+), 153 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index 8ac07957de4..b2f17ce4f88 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -60,8 +60,8 @@ void collectOutPutFieldTypesFromAgg(std::vector & field_type, c } } -DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) -: id(id_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") +DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root_) + : id(id_), root(root_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") { const tipb::Executor * current = root; while (!isSourceNode(current)) @@ -103,6 +103,7 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root) fillOutputFieldTypes(); } +/* DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & executors, int start_index, int end_index) : id(id_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") { @@ -129,50 +130,49 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & e case tipb::ExecType::TypeLimit: assignOrThrowException(&limitOrTopN, executors[i], LIMIT_NAME); break; - /* - case tipb::ExecType::TypeJoin: - if (i <= start_index) - throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); - if (executors[i - 1] == &(executors[i]->join().build_exec())) - { - build_end_index = i - 1; - build_start_index = build_end_index; - while (build_start_index >= start_index && executors[build_start_index] != &(executors[i]->join().probe_exec())) - build_start_index--; - if (build_start_index < start_index) - throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); - probe_end_index = build_start_index; - build_start_index++; - probe_start_index = start_index; - } - else if (executors[i - 1] == &(executors[i]->join().probe_exec())) - { - probe_end_index = i - 1; - probe_start_index = probe_end_index; - while (probe_start_index >= start_index && executors[probe_start_index] != &(executors[i]->join().build_exec())) - probe_start_index--; - if (probe_start_index < start_index) - throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); - build_end_index = probe_start_index; - probe_start_index++; - build_start_index = start_index; - } - else - { - throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); - } - children.push_back(std::make_shared(id * 2, executors, probe_start_index, probe_end_index)); - children.push_back(std::make_shared(id * 2 + 1, executors, build_start_index, build_end_index)); - // to break the for loop - i = start_index - 1; - break; - */ +// case tipb::ExecType::TypeJoin: +// if (i <= start_index) +// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); +// if (executors[i - 1] == &(executors[i]->join().build_exec())) +// { +// build_end_index = i - 1; +// build_start_index = build_end_index; +// while (build_start_index >= start_index && executors[build_start_index] != &(executors[i]->join().probe_exec())) +// build_start_index--; +// if (build_start_index < start_index) +// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); +// probe_end_index = build_start_index; +// build_start_index++; +// probe_start_index = start_index; +// } +// else if (executors[i - 1] == &(executors[i]->join().probe_exec())) +// { +// probe_end_index = i - 1; +// probe_start_index = probe_end_index; +// while (probe_start_index >= start_index && executors[probe_start_index] != &(executors[i]->join().build_exec())) +// probe_start_index--; +// if (probe_start_index < start_index) +// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); +// build_end_index = probe_start_index; +// probe_start_index++; +// build_start_index = start_index; +// } +// else +// { +// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); +// } +// children.push_back(std::make_shared(id * 2, executors, probe_start_index, probe_end_index)); +// children.push_back(std::make_shared(id * 2 + 1, executors, build_start_index, build_end_index)); +// // to break the for loop +// i = start_index - 1; +// break; default: throw Exception("Unsupported executor in DAG request: " + executors[i]->DebugString(), ErrorCodes::NOT_IMPLEMENTED); } } fillOutputFieldTypes(); } +*/ void DAGQueryBlock::fillOutputFieldTypes() { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index ab3d7ce1002..2e1afbd61d9 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -25,12 +25,13 @@ class DAGQueryBlock { public: DAGQueryBlock(UInt32 id, const tipb::Executor * root); - DAGQueryBlock(UInt32 id, std::vector & executors, int start_index, int end_index); + //DAGQueryBlock(UInt32 id, std::vector & executors, int start_index, int end_index); const tipb::Executor * source = nullptr; const tipb::Executor * selection = nullptr; const tipb::Executor * aggregation = nullptr; const tipb::Executor * limitOrTopN = nullptr; UInt32 id; + const tipb::Executor * root; String qb_column_prefix; // todo use unique_ptr instead std::vector> children; @@ -39,6 +40,10 @@ class DAGQueryBlock std::vector output_offsets; void fillOutputFieldTypes(); bool isRootQueryBlock() const { return id == 1; }; + bool isRemoteQuery() const + { + return source->tp() == tipb::ExecType::TypeTableScan && source->tbl_scan().next_read_engine() != tipb::EngineType::Local; + } }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index cb80ff2e7a4..9d16866e661 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -16,8 +17,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -30,9 +33,6 @@ #include #include #include -#include -#include -#include namespace DB { @@ -394,7 +394,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & { for (auto i : query_block.output_offsets) { - if ((size_t) i >= required_columns.size() - extra_col_size) + if ((size_t)i >= required_columns.size() - extra_col_size) { // array index out of bound throw Exception("Output offset index is out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); @@ -405,7 +405,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & } else { - for(size_t i = 0; i < required_columns.size() - extra_col_size; i++) + for (size_t i = 0; i < required_columns.size() - extra_col_size; i++) /// for child query block, add alias start with qb_column_prefix to avoid column name conflict final_project.emplace_back(required_columns[i], query_block.qb_column_prefix + required_columns[i]); } @@ -448,20 +448,23 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); query_info.mvcc_query_info->concurrent = 0.0; - if (ts.next_read_engine() == tipb::EngineType::Local) { - pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, - max_streams); - } else { - std::vector> key_ranges; - for (auto & range : ts.ranges()) - { - std::string start_key(range.low()); - DecodedTiKVKey start(std::move(start_key)); - std::string end_key(range.high()); - DecodedTiKVKey end(std::move(end_key)); - key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); - } - pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); + if (ts.next_read_engine() == tipb::EngineType::Local) + { + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + } + else + { + throw Exception("Should not reach here", ErrorCodes::LOGICAL_ERROR); + //std::vector> key_ranges; + //for (auto & range : ts.ranges()) + //{ + // std::string start_key(range.low()); + // DecodedTiKVKey start(std::move(start_key)); + // std::string end_key(range.high()); + // DecodedTiKVKey end(std::move(end_key)); + // key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); + //} + //pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); } if (pipeline.streams.empty()) @@ -506,7 +509,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & void DAGQueryBlockInterpreter::prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left) { std::vector source_columns; - for( auto const & p : pipeline.firstStream()->getHeader().getNamesAndTypesList()) + for (auto const & p : pipeline.firstStream()->getHeader().getNamesAndTypesList()) source_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); ExpressionActionsChain chain; @@ -519,10 +522,8 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p { // build static const std::unordered_map join_type_map{ - {tipb::JoinType::TypeInnerJoin, ASTTableJoin::Kind::Inner}, - {tipb::JoinType::TypeLeftOuterJoin, ASTTableJoin::Kind::Left}, - {tipb::JoinType::TypeRightOuterJoin, ASTTableJoin::Kind::Right} - }; + {tipb::JoinType::TypeInnerJoin, ASTTableJoin::Kind::Inner}, {tipb::JoinType::TypeLeftOuterJoin, ASTTableJoin::Kind::Left}, + {tipb::JoinType::TypeRightOuterJoin, ASTTableJoin::Kind::Right}}; if (input_streams_vec.size() != 2) { throw Exception("Join query block must have 2 input streams", ErrorCodes::LOGICAL_ERROR); @@ -559,13 +560,13 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p right_streams = input_streams_vec[1]; } std::vector join_output_columns; - for( auto const & p : left_streams[0]->getHeader().getNamesAndTypesList()) + for (auto const & p : left_streams[0]->getHeader().getNamesAndTypesList()) { join_output_columns.emplace_back(p.name, p.type); } /// all the columns from right table should be added after join, even for the join key NamesAndTypesList columns_added_by_join; - for( auto const & p : right_streams[0]->getHeader().getNamesAndTypesList()) + for (auto const & p : right_streams[0]->getHeader().getNamesAndTypesList()) { columns_added_by_join.emplace_back(p.name, p.type); join_output_columns.emplace_back(p.name, p.type); @@ -573,9 +574,9 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p if (!query_block.aggregation) { - for(auto const & p : input_streams_vec[0][0]->getHeader().getNamesAndTypesList()) + for (auto const & p : input_streams_vec[0][0]->getHeader().getNamesAndTypesList()) final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); - for(auto const & p : input_streams_vec[1][0]->getHeader().getNamesAndTypesList()) + for (auto const & p : input_streams_vec[1][0]->getHeader().getNamesAndTypesList()) final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); } @@ -592,8 +593,8 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p const Settings & settings = context.getSettingsRef(); JoinPtr joinPtr = std::make_shared(left_key_names, right_key_names, true, - SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), - kind, ASTTableJoin::Strictness::All); + SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), kind, + ASTTableJoin::Strictness::All); executeUnion(right_pipeline); // todo clickhouse use LazyBlockInputStream to initialize the source, need to double check why and // if we need to use LazyBlockInputStream here @@ -602,7 +603,7 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p right_query.join->setSampleBlock(right_query.source->getHeader()); std::vector source_columns; - for(const auto & p : left_streams[0]->getHeader().getNamesAndTypesList()) + for (const auto & p : left_streams[0]->getHeader().getNamesAndTypesList()) source_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); ExpressionActionsChain chain; @@ -629,8 +630,8 @@ bool DAGQueryBlockInterpreter::addTimeZoneCastAfterTS(std::vector & is_ts_ /// 1. the query block is the root query block /// 2. keep_session_timezone_info is false /// 3. current query block does not have aggregation - if (analyzer->appendTimeZoneCastsAfterTS(chain, is_ts_column, rqst, - query_block.isRootQueryBlock() && !keep_session_timezone_info && query_block.aggregation == nullptr)) + if (analyzer->appendTimeZoneCastsAfterTS( + chain, is_ts_column, rqst, query_block.isRootQueryBlock() && !keep_session_timezone_info && query_block.aggregation == nullptr)) { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); return true; @@ -926,13 +927,49 @@ void DAGQueryBlockInterpreter::executeSubqueryInJoin(Pipeline & pipeline, Subque executeUnion(pipeline); SubqueriesForSets subquries; subquries[query_block.qb_column_prefix + "join"] = subquery; - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), subquries, - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), subquries, + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); +} + +void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) +{ + const auto & ts = query_block.source->tbl_scan(); + TableID table_id = ts.table_id(); + + const Settings & settings = context.getSettingsRef(); + + if (settings.schema_version == DEFAULT_UNSPECIFIED_SCHEMA_VERSION) + { + storage = context.getTMTContext().getStorages().get(table_id); + if (storage == nullptr) + { + throw Exception("Table " + std::to_string(table_id) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + } + table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + } + else + { + getAndLockStorageWithSchemaVersion(table_id, settings.schema_version); + } + std::vector> key_ranges; + for (auto & range : ts.ranges()) + { + std::string start_key(range.low()); + DecodedTiKVKey start(std::move(start_key)); + std::string end_key(range.high()); + DecodedTiKVKey end(std::move(end_key)); + key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); + } + pipeline.streams = storage->remote_read(key_ranges, context.getSettingsRef().read_tso, query_block, context); } void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { + if (query_block.isRemoteQuery()) + { + executeRemoteQuery(pipeline); + return; + } SubqueryForSet right_query; if (query_block.source->tp() == tipb::ExecType::TypeJoin) { @@ -942,49 +979,49 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { executeTS(query_block.source->tbl_scan(), pipeline); } - // todo enable profile stream info - //recordProfileStreams(pipeline, dag.getTSIndex()); + // todo enable profile stream info + //recordProfileStreams(pipeline, dag.getTSIndex()); - auto res = analyzeExpressions(); - // execute selection - if (res.has_where) - { - executeWhere(pipeline, res.before_where, res.filter_column_name); - //if (dag.hasSelection()) - //recordProfileStreams(pipeline, dag.getSelectionIndex()); - } - if (res.need_aggregate) - { - // execute aggregation - executeAggregation(pipeline, res.before_aggregation, res.aggregation_keys, res.aggregate_descriptions); - //recordProfileStreams(pipeline, dag.getAggregationIndex()); - } - if (res.before_order_and_select) - { - executeExpression(pipeline, res.before_order_and_select); - } + auto res = analyzeExpressions(); + // execute selection + if (res.has_where) + { + executeWhere(pipeline, res.before_where, res.filter_column_name); + //if (dag.hasSelection()) + //recordProfileStreams(pipeline, dag.getSelectionIndex()); + } + if (res.need_aggregate) + { + // execute aggregation + executeAggregation(pipeline, res.before_aggregation, res.aggregation_keys, res.aggregate_descriptions); + //recordProfileStreams(pipeline, dag.getAggregationIndex()); + } + if (res.before_order_and_select) + { + executeExpression(pipeline, res.before_order_and_select); + } - if (res.has_order_by) - { - // execute topN - executeOrder(pipeline, res.order_column_names); - //recordProfileStreams(pipeline, dag.getTopNIndex()); - } + if (res.has_order_by) + { + // execute topN + executeOrder(pipeline, res.order_column_names); + //recordProfileStreams(pipeline, dag.getTopNIndex()); + } - // execute projection - executeFinalProject(pipeline); + // execute projection + executeFinalProject(pipeline); - // execute limit - if (query_block.limitOrTopN != nullptr && query_block.limitOrTopN->tp() == tipb::TypeLimit) - { - executeLimit(pipeline); - //recordProfileStreams(pipeline, dag.getLimitIndex()); - } - if (query_block.source->tp() == tipb::ExecType::TypeJoin) - { - // add the - executeSubqueryInJoin(pipeline, right_query); - } + // execute limit + if (query_block.limitOrTopN != nullptr && query_block.limitOrTopN->tp() == tipb::TypeLimit) + { + executeLimit(pipeline); + //recordProfileStreams(pipeline, dag.getLimitIndex()); + } + if (query_block.source->tp() == tipb::ExecType::TypeJoin) + { + // add the + executeSubqueryInJoin(pipeline, right_query); + } } void DAGQueryBlockInterpreter::executeFinalProject(Pipeline & pipeline) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index a9736f58701..ce97ffb363a 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -12,11 +12,11 @@ #include #include #include +#include #include #include #include #include -#include namespace DB { @@ -86,6 +86,7 @@ class DAGQueryBlockInterpreter BlockInputStreams execute(); private: + void executeRemoteQuery(Pipeline & pipeline); void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query); diff --git a/dbms/src/Storages/IManageableStorage.cpp b/dbms/src/Storages/IManageableStorage.cpp index e69de29bb2d..0e8d0e1fc4e 100644 --- a/dbms/src/Storages/IManageableStorage.cpp +++ b/dbms/src/Storages/IManageableStorage.cpp @@ -0,0 +1,119 @@ +#include + +namespace DB +{ + +void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb::Executor * root) +{ + const tipb::Executor * current = root; + auto * exec = dag_req.add_executors(); + while (current->tp() != tipb::ExecType::TypeTableScan) + { + if (current->tp() == tipb::ExecType::TypeSelection) + { + exec->set_tp(tipb::ExecType::TypeSelection); + auto * sel = exec->mutable_selection(); + for (auto const & condition : current->selection().conditions()) + { + auto * tmp = sel->add_conditions(); + tmp->CopyFrom(condition); + } + exec = sel->mutable_child(); + current = ¤t->selection().child(); + } + else if (current->tp() == tipb::ExecType::TypeAggregation || current->tp() == tipb::ExecType::TypeStreamAgg) + { + exec->set_tp(current->tp()); + auto * agg = exec->mutable_aggregation(); + for (auto const & expr : current->aggregation().agg_func()) + { + auto * tmp = agg->add_agg_func(); + tmp->CopyFrom(expr); + } + for (auto const & expr : current->aggregation().group_by()) + { + auto * tmp = agg->add_group_by(); + tmp->CopyFrom(expr); + } + agg->set_streamed(current->aggregation().streamed()); + exec = agg->mutable_child(); + current = ¤t->aggregation().child(); + } + else if (current->tp() == tipb::ExecType::TypeLimit) + { + exec->set_tp(current->tp()); + auto * limit = exec->mutable_limit(); + limit->set_limit(current->limit().limit()); + exec = limit->mutable_child(); + current = ¤t->limit().child(); + } + else if (current->tp() == tipb::ExecType::TypeTopN) + { + exec->set_tp(current->tp()); + auto * topn = exec->mutable_topn(); + topn->set_limit(current->topn().limit()); + for (auto const & expr : current->topn().order_by()) + { + auto * tmp = topn->add_order_by(); + tmp->CopyFrom(expr); + } + exec = topn->mutable_child(); + current = ¤t->topn().child(); + } + else + { + throw Exception("Not supported yet"); + } + } + + if (current->tp() != tipb::ExecType::TypeTableScan) + throw Exception("Only support copy from table scan sourced query block"); + exec->set_tp(tipb::ExecType::TypeTableScan); + auto * new_ts = new tipb::TableScan(current->tbl_scan()); + new_ts->set_next_read_engine(tipb::EngineType::Local); + exec->set_allocated_tbl_scan(new_ts); + + dag_req.set_encode_type(tipb::EncodeType::TypeChunk); +} + +BlockInputStreams IManageableStorage::remote_read(const std::vector> & key_ranges, + UInt64 read_tso, const DAGQueryBlock & remote_query_block, Context & context) +{ + std::vector cop_key_ranges; + for (const auto & key_range : key_ranges) + { + cop_key_ranges.push_back( + pingcap::coprocessor::KeyRange{static_cast(key_range.first), static_cast(key_range.second)}); + } + + ::tipb::DAGRequest dag_req; + + copyExecutorTreeWithLocalTableScan(dag_req, remote_query_block.root); + + DAGSchema schema; + ColumnsWithTypeAndName columns; + for (int i = 0; i < (int)remote_query_block.output_field_types.size(); i++) + { + dag_req.add_output_offsets(i); + ColumnInfo info = fieldTypeToColumnInfo(remote_query_block.output_field_types[i]); + String col_name = "col_" + std::to_string(i); + schema.push_back(std::make_pair(col_name, info)); + auto tp = getDataTypeByColumnInfo(info); + ColumnWithTypeAndName col(tp, col_name); + columns.emplace_back(col); + } + Block sample_block = Block(columns); + + pingcap::coprocessor::Request req; + + dag_req.SerializeToString(&req.data); + req.tp = pingcap::coprocessor::ReqType::DAG; + req.start_ts = read_tso; + req.ranges = cop_key_ranges; + + pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); + BlockInputStreamPtr input = std::make_shared( + sample_block, [cluster, req, schema]() { return std::make_shared(cluster, req, schema); }); + return {input}; +}; +} // namespace DB diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index 4ee56ce54c8..5d11c1d6c43 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -2,7 +2,9 @@ #include #include +#include #include +#include #include #include #include @@ -55,42 +57,8 @@ class IManageableStorage : public IStorage virtual const TiDB::TableInfo & getTableInfo() const = 0; - virtual BlockInputStreams remote_read(const std::vector> & key_ranges, - const SelectQueryInfo & query_info, const tipb::TableScan & ts, Context & context) - { - std::vector cop_key_ranges; - for (const auto & key_range : key_ranges) - { - cop_key_ranges.push_back( - pingcap::coprocessor::KeyRange{static_cast(key_range.first), static_cast(key_range.second)}); - } - - ::tipb::DAGRequest dag_req; - - auto * exec = dag_req.add_executors(); - exec->set_tp(tipb::ExecType::TypeTableScan); - exec->set_allocated_tbl_scan(new tipb::TableScan(ts)); - dag_req.set_encode_type(tipb::EncodeType::TypeChunk); - - DAGSchema schema; - for (int i = 0; i < ts.columns_size(); i++) - { - dag_req.add_output_offsets(i); - auto id = ts.columns(i).column_id(); - const ColumnInfo & info = getTableInfo().getColumnInfoByID(id); - schema.push_back(std::make_pair(info.name, info)); - } - - pingcap::coprocessor::Request req; - - dag_req.SerializeToString(&req.data); - req.tp = pingcap::coprocessor::ReqType::DAG; - req.start_ts = query_info.mvcc_query_info->read_tso; - req.ranges = cop_key_ranges; - - BlockInputStreamPtr input = std::make_shared(context.getTMTContext().getKVCluster(), req, schema); - return {input}; - }; + BlockInputStreams remote_read(const std::vector> & key_ranges, UInt64 read_tso, + const DAGQueryBlock & remote_query_block, Context & context); // Apply AlterCommands synced from TiDB should use `alterFromTiDB` instead of `alter(...)` virtual void alterFromTiDB( From 116a5085fe71bd41b4f208ac14b0769453b6e2d6 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 11 Mar 2020 20:21:52 +0800 Subject: [PATCH 21/54] update client-c to support cop reading from tiflash --- contrib/client-c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/client-c b/contrib/client-c index effedfb8a8f..9947e7cd873 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit effedfb8a8f2a2dd8a1ba262e31e4a4742004b9c +Subproject commit 9947e7cd873b49c3c044b1eede95088d0e528532 From d101639c650477df8d7dca729e892f21959358f9 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 12 Mar 2020 11:33:52 +0800 Subject: [PATCH 22/54] refine code --- contrib/client-c | 2 +- dbms/src/DataStreams/CoprocessorBlockInputStream.h | 5 +++-- dbms/src/Storages/IManageableStorage.cpp | 3 ++- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/contrib/client-c b/contrib/client-c index 9947e7cd873..a31669150e5 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 9947e7cd873b49c3c044b1eede95088d0e528532 +Subproject commit a31669150e5dd12b269e8f389487d9305f053ddb diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h index 1606c649ed5..dd767ebfa9f 100644 --- a/dbms/src/DataStreams/CoprocessorBlockInputStream.h +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -11,9 +11,10 @@ namespace DB class CoprocessorBlockInputStream : public IProfilingBlockInputStream { public: - CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_) + CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_, + pingcap::kv::StoreType store_type) : req(req_), - resp_iter(pingcap::coprocessor::Client::send(cluster_, &req)), + resp_iter(pingcap::coprocessor::Client::send(cluster_, &req, store_type)), schema(schema_), log(&Logger::get("pingcap/coprocessor")) { diff --git a/dbms/src/Storages/IManageableStorage.cpp b/dbms/src/Storages/IManageableStorage.cpp index 0e8d0e1fc4e..8fc643c4bfa 100644 --- a/dbms/src/Storages/IManageableStorage.cpp +++ b/dbms/src/Storages/IManageableStorage.cpp @@ -112,8 +112,9 @@ BlockInputStreams IManageableStorage::remote_read(const std::vector( - sample_block, [cluster, req, schema]() { return std::make_shared(cluster, req, schema); }); + sample_block, [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); return {input}; }; } // namespace DB From 9047e4e33d2074f81aa5833494a7aacbb46dce84 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 10 Mar 2020 14:50:28 +0800 Subject: [PATCH 23/54] support batch cop --- dbms/src/Debug/dbgFuncCoprocessor.cpp | 4 +- dbms/src/Flash/BatchCoprocessorHandler.cpp | 147 ++++++++++++++++++ dbms/src/Flash/BatchCoprocessorHandler.h | 52 +++++++ dbms/src/Flash/Coprocessor/DAGDriver.cpp | 78 +++++++++- dbms/src/Flash/Coprocessor/DAGDriver.h | 24 ++- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 88 +++++++---- .../Coprocessor/DAGQueryBlockInterpreter.h | 4 +- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 8 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 11 +- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 88 +++++++---- dbms/src/Flash/Coprocessor/InterpreterDAG.h | 2 +- .../StreamingDAGBlockOutputStream.cpp | 106 +++++++++++++ .../StreamingDAGBlockOutputStream.h | 42 +++++ dbms/src/Flash/CoprocessorHandler.cpp | 5 +- dbms/src/Flash/CoprocessorHandler.h | 2 + dbms/src/Flash/FlashService.cpp | 19 +++ dbms/src/Flash/FlashService.h | 4 + dbms/src/Storages/Transaction/TMTContext.h | 2 + 18 files changed, 599 insertions(+), 87 deletions(-) create mode 100644 dbms/src/Flash/BatchCoprocessorHandler.cpp create mode 100644 dbms/src/Flash/BatchCoprocessorHandler.h create mode 100644 dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp create mode 100644 dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index 96d953c7f0c..4b457d667b5 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -643,7 +643,9 @@ tipb::SelectResponse executeDAGRequest(Context & context, const tipb::DAGRequest static Logger * log = &Logger::get("MockDAG"); LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); tipb::SelectResponse dag_response; - DAGDriver driver(context, dag_request, region_id, region_version, region_conf_version, start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, + std::vector regions; + regions.emplace_back(region_id, region_version, region_conf_version); + DAGDriver driver(context, dag_request, regions, start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, std::move(key_ranges), dag_response, true); driver.execute(); LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); diff --git a/dbms/src/Flash/BatchCoprocessorHandler.cpp b/dbms/src/Flash/BatchCoprocessorHandler.cpp new file mode 100644 index 00000000000..c61de88c205 --- /dev/null +++ b/dbms/src/Flash/BatchCoprocessorHandler.cpp @@ -0,0 +1,147 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + + namespace ErrorCodes + { + extern const int NOT_IMPLEMENTED; + } + + BatchCoprocessorHandler::BatchCoprocessorHandler( + CoprocessorContext & cop_context_, const coprocessor::BatchRequest * cop_request_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) + : cop_context(cop_context_), cop_request(cop_request_), writer(writer_), log(&Logger::get("BatchCoprocessorHandler")) + {} + + grpc::Status BatchCoprocessorHandler::execute() + try + { + switch (cop_request->tp()) + { + case COP_REQ_TYPE_DAG: + { + std::vector> key_ranges; + tipb::DAGRequest dag_request; + dag_request.ParseFromString(cop_request->data()); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); + std::vector regions; + for (auto & r : cop_request->regions()) { + regions.emplace_back(r.region_id(), r.region_epoch().version(), r.region_epoch().conf_ver()); + } + tipb::SelectResponse dag_response; // unused + DAGDriver driver(cop_context.db_context, dag_request, regions, cop_request->start_ts() > 0 ? + cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), std::move(key_ranges), + dag_response); + // batch execution; + driver.batchExecute(writer); + if (dag_response.has_error()) + { + err_response.set_other_error(dag_response.error().msg()); + writer->Write(err_response); + } + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); + break; + } + case COP_REQ_TYPE_ANALYZE: + case COP_REQ_TYPE_CHECKSUM: + default: + throw Exception( + "Coprocessor request type " + std::to_string(cop_request->tp()) + " is not implemented", ErrorCodes::NOT_IMPLEMENTED); + } + return grpc::Status::OK; + } + catch (const LockException & e) + { + LOG_ERROR(log, + __PRETTY_FUNCTION__ << ": LockException: region " << cop_request->context().region_id() << "\n" + << e.getStackTrace().toString()); + + //for (int i = 0; i < cop_request->regions_size(); i++) { + // auto * status = cop_response->add_region_status(); + // if (cop_request->regions(i).region_id() == e.region_id) + // { + // auto * lock_info = status->mutable_locked(); + // lock_info->set_key(e.lock_infos[0]->key); + // lock_info->set_primary_lock(e.lock_infos[0]->primary_lock); + // lock_info->set_lock_ttl(e.lock_infos[0]->lock_ttl); + // lock_info->set_lock_version(e.lock_infos[0]->lock_version); + // } + // else + // { + // status->set_success(true); + // } + //} + // return ok so TiDB has the chance to see the LockException + return grpc::Status::OK; + } + catch (const RegionException & e) + { + LOG_ERROR(log, + __PRETTY_FUNCTION__ << ": RegionException: region " << cop_request->context().region_id() << "\n" + << e.getStackTrace().toString()); + //errorpb::Error * region_err; + switch (e.status) + { + case RegionException::RegionReadStatus::NOT_FOUND: + case RegionException::RegionReadStatus::PENDING_REMOVE: + for (auto region_id : e.region_ids) { + auto *status = err_response.add_region_status(); + status->mutable_region_error()->mutable_region_not_found()->set_region_id(region_id); + status->set_region_id(region_id); + } + break; + case RegionException::RegionReadStatus::VERSION_ERROR: + + for (auto region_id : e.region_ids) { + auto * status = err_response.add_region_status(); + status->mutable_region_error()->mutable_epoch_not_match(); + status->set_region_id(region_id); + } + break; + default: + // should not happen + break; + } + // TODO: This should not be the last response, we should return data and exception at the same time. + writer->Write(err_response); + // return ok so TiDB has the chance to see the LockException + return grpc::Status::OK; + } + catch (const Exception & e) + { + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": DB Exception: " << e.message() << "\n" << e.getStackTrace().toString()); + return recordError(e.code() == ErrorCodes::NOT_IMPLEMENTED ? grpc::StatusCode::UNIMPLEMENTED : grpc::StatusCode::INTERNAL, e.message()); + } + catch (const pingcap::Exception & e) + { + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": KV Client Exception: " << e.message()); + return recordError(e.code() == ErrorCodes::NOT_IMPLEMENTED ? grpc::StatusCode::UNIMPLEMENTED : grpc::StatusCode::INTERNAL, e.message()); + } + catch (const std::exception & e) + { + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": std exception: " << e.what()); + return recordError(grpc::StatusCode::INTERNAL, e.what()); + } + catch (...) + { + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": other exception"); + return recordError(grpc::StatusCode::INTERNAL, "other exception"); + } + + grpc::Status BatchCoprocessorHandler::recordError(grpc::StatusCode err_code, const String & err_msg) + { + err_response.set_other_error(err_msg); + + return grpc::Status(err_code, err_msg); + } + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/BatchCoprocessorHandler.h b/dbms/src/Flash/BatchCoprocessorHandler.h new file mode 100644 index 00000000000..ba2232bb388 --- /dev/null +++ b/dbms/src/Flash/BatchCoprocessorHandler.h @@ -0,0 +1,52 @@ +#pragma once + +#include + +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop +#include +#include "CoprocessorHandler.h" + +namespace DB +{ + +/// Coprocessor request handler, deals with: +/// 1. DAG request: WIP; +/// 2. Analyze request: NOT IMPLEMENTED; +/// 3. Checksum request: NOT IMPLEMENTED; +class BatchCoprocessorHandler +{ +public: + BatchCoprocessorHandler(CoprocessorContext & cop_context_, const coprocessor::BatchRequest * cop_request_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_); + + ~BatchCoprocessorHandler() = default; + + grpc::Status execute(); + +protected: + grpc::Status recordError(grpc::StatusCode err_code, const String & err_msg); + +protected: + enum + { + COP_REQ_TYPE_DAG = 103, + COP_REQ_TYPE_ANALYZE = 104, + COP_REQ_TYPE_CHECKSUM = 105, + }; + + CoprocessorContext & cop_context; + const coprocessor::BatchRequest * cop_request; + ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; + + ::coprocessor::BatchResponse err_response; + + Logger * log; +}; + +using BatchCopHandlerPtr = std::shared_ptr; + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index ee5ea3ccb41..a3f9773576b 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -21,14 +22,12 @@ extern const int LOGICAL_ERROR; extern const int UNKNOWN_EXCEPTION; } // namespace ErrorCodes -DAGDriver::DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, RegionID region_id_, UInt64 region_version_, - UInt64 region_conf_version_, UInt64 start_ts, UInt64 schema_ver, std::vector> && key_ranges_, +DAGDriver::DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, const std::vector & regions_, + UInt64 start_ts, UInt64 schema_ver, std::vector> && key_ranges_, tipb::SelectResponse & dag_response_, bool internal_) : context(context_), dag_request(dag_request_), - region_id(region_id_), - region_version(region_version_), - region_conf_version(region_conf_version_), + regions(regions_), key_ranges(std::move(key_ranges_)), dag_response(dag_response_), internal(internal_), @@ -44,7 +43,7 @@ void DAGDriver::execute() try { DAGContext dag_context(dag_request.executors_size()); - DAGQuerySource dag(context, dag_context, region_id, region_version, region_conf_version, key_ranges, dag_request); + DAGQuerySource dag(context, dag_context, regions, key_ranges, dag_request); BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); if (!streams.in || streams.out) @@ -107,6 +106,73 @@ catch (...) recordError(ErrorCodes::UNKNOWN_EXCEPTION, "other exception"); } +void DAGDriver::batchExecute(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer) +try +{ + DAGContext dag_context(dag_request.executors_size()); + DAGQuerySource dag(context, dag_context, regions, key_ranges, dag_request); + + BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); + if (!streams.in || streams.out) + // Only query is allowed, so streams.in must not be null and streams.out must be null + throw Exception("DAG is not query.", ErrorCodes::LOGICAL_ERROR); + + BlockOutputStreamPtr dag_output_stream = std::make_shared( + writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), streams.in->getHeader()); + copyData(*streams.in, *dag_output_stream); + + if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) + return; + // add ExecutorExecutionSummary info + for (auto & p_streams : dag_context.profile_streams_list) + { + auto * executeSummary = dag_response.add_execution_summaries(); + UInt64 time_processed_ns = 0; + UInt64 num_produced_rows = 0; + UInt64 num_iterations = 0; + for (auto & streamPtr : p_streams) + { + if (auto * p_stream = dynamic_cast(streamPtr.get())) + { + time_processed_ns = std::max(time_processed_ns, p_stream->getProfileInfo().execution_time); + num_produced_rows += p_stream->getProfileInfo().rows; + num_iterations += p_stream->getProfileInfo().blocks; + } + } + executeSummary->set_time_processed_ns(time_processed_ns); + executeSummary->set_num_produced_rows(num_produced_rows); + executeSummary->set_num_iterations(num_iterations); + } +} +catch (const RegionException & e) +{ + throw; +} +catch (const LockException & e) +{ + throw; +} +catch (const Exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": DB Exception: " << e.message() << "\n" << e.getStackTrace().toString()); + recordError(e.code(), e.message()); +} +catch (const pingcap::Exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": KV Client Exception: " << e.message()); + recordError(e.code(), e.message()); +} +catch (const std::exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": std exception: " << e.what()); + recordError(ErrorCodes::UNKNOWN_EXCEPTION, e.what()); +} +catch (...) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": other exception"); + recordError(ErrorCodes::UNKNOWN_EXCEPTION, "other exception"); +} + void DAGDriver::recordError(Int32 err_code, const String & err_msg) { dag_response.Clear(); diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.h b/dbms/src/Flash/Coprocessor/DAGDriver.h index cd74b5d6ad8..aa99aa2be34 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.h +++ b/dbms/src/Flash/Coprocessor/DAGDriver.h @@ -4,24 +4,37 @@ #include #include #include +#include +#include +#include namespace DB { class Context; +class RegionInfo { +public: + RegionID region_id; + UInt64 region_version; + UInt64 region_conf_version; + RegionInfo(RegionID id, UInt64 ver, UInt64 conf_ver) : region_id(id), region_version(ver), region_conf_version(conf_ver) {} +}; + /// An abstraction of driver running DAG request. /// Now is a naive native executor. Might get evolved to drive MPP-like computation. class DAGDriver { public: - DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, RegionID region_id_, UInt64 region_version_, - UInt64 region_conf_version_, UInt64 start_ts, UInt64 schema_ver, + DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, const std::vector & regions_, + UInt64 start_ts, UInt64 schema_ver, std::vector> && key_ranges_, tipb::SelectResponse & dag_response_, bool internal_ = false); void execute(); + void batchExecute(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer); + private: void recordError(Int32 err_code, const String & err_msg); @@ -30,9 +43,10 @@ class DAGDriver const tipb::DAGRequest & dag_request; - RegionID region_id; - UInt64 region_version; - UInt64 region_conf_version; + const std::vector & regions; + //RegionID region_id; + //UInt64 region_version; + //UInt64 region_conf_version; std::vector> key_ranges; tipb::SelectResponse & dag_response; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 9d16866e661..a807712d356 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -47,13 +47,13 @@ extern const int COP_BAD_DAG_REQUEST; } // namespace ErrorCodes DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info_, const tipb::DAGRequest & rqst_, + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos_, const tipb::DAGRequest & rqst_, ASTPtr dummy_query_) : context(context_), input_streams_vec(input_streams_vec_), query_block(query_block_), keep_session_timezone_info(keep_session_timezone_info_), - region_info(region_info_), + region_infos(region_infos_), rqst(rqst_), dummy_query(std::move(dummy_query_)), log(&Logger::get("DAGQueryBlockInterpreter")) @@ -155,11 +155,12 @@ void constructExprBasedOnRange(Int32 handle_col_id, tipb::Expr & expr, HandleRan template bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, const std::vector> & region_ranges, - Int32 handle_col_id, tipb::Expr & handle_filter) + Int32 handle_col_id, tipb::Expr & handle_filter, Logger *log) { if (ranges.empty()) { // generate an always false filter + LOG_WARNING(log, "income key ranges is empty"); constructInt64LiteralTiExpr(handle_filter, 0); return false; } @@ -204,6 +205,7 @@ bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, break; } } + LOG_DEBUG(log, "ret " <> & ranges, } bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, - const ImutRegionRangePtr & region_key_range, Int32 handle_col_id, tipb::Expr & handle_filter) + const ImutRegionRangePtr & region_key_range, Int32 handle_col_id, tipb::Expr & handle_filter, Logger * log) { + LOG_INFO(log, "pk_is_uint: " << pk_is_uint64); if (key_ranges.empty()) { + LOG_WARNING(log, "income key ranges is empty1"); constructInt64LiteralTiExpr(handle_filter, 0); return false; } @@ -277,18 +281,18 @@ bool checkKeyRanges(const std::vector> update_region_handle_ranges.emplace_back(new_range[i]); } } - return checkRangeAndGenExprIfNeeded(update_handle_ranges, update_region_handle_ranges, handle_col_id, handle_filter); + return checkRangeAndGenExprIfNeeded(update_handle_ranges, update_region_handle_ranges, handle_col_id, handle_filter, log); } else - return checkRangeAndGenExprIfNeeded(handle_ranges, region_handle_ranges, handle_col_id, handle_filter); + return checkRangeAndGenExprIfNeeded(handle_ranges, region_handle_ranges, handle_col_id, handle_filter, log); } RegionException::RegionReadStatus DAGQueryBlockInterpreter::getRegionReadStatus(const RegionPtr & current_region) { if (!current_region) return RegionException::NOT_FOUND; - if (current_region->version() != region_info.region_version || current_region->confVer() != region_info.region_conf_version) - return RegionException::VERSION_ERROR; + //if (current_region->version() != region_info.region_version || current_region->confVer() != region_info.region_conf_version) + // return RegionException::VERSION_ERROR; if (current_region->isPendingRemove()) return RegionException::PENDING_REMOVE; return RegionException::OK; @@ -355,22 +359,22 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & if (handle_col_id == -1) handle_col_id = required_columns.size(); - auto current_region = context.getTMTContext().getKVStore()->getRegion(region_info.region_id); - auto region_read_status = getRegionReadStatus(current_region); - if (region_read_status != RegionException::OK) - { - std::vector region_ids; - region_ids.push_back(region_info.region_id); - LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << region_info.region_id); - throw RegionException(std::move(region_ids), region_read_status); - } - const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; - if (!checkKeyRanges(region_info.key_ranges, table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) - { - // need to add extra filter on handle column - filter_on_handle = true; - conditions.push_back(&handle_filter_expr); - } +// auto current_region = context.getTMTContext().getKVStore()->getRegion(region_info.region_id); +// auto region_read_status = getRegionReadStatus(current_region); +// if (region_read_status != RegionException::OK) +// { +// std::vector region_ids; +// region_ids.push_back(region_info.region_id); +// LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << region_info.region_id); +// throw RegionException(std::move(region_ids), region_read_status); +// } +// const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; +// if (!checkKeyRanges(region_info.key_ranges, table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) +// { +// // need to add extra filter on handle column +// filter_on_handle = true; +// conditions.push_back(&handle_filter_expr); +// } bool has_handle_column = (handle_col_id != (Int32)required_columns.size()); @@ -434,6 +438,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & analyzer->makeExplicitSetForIndex(condition, storage); } } + SelectQueryInfo query_info; /// to avoid null point exception query_info.query = dummy_query; @@ -441,13 +446,33 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & query_info.mvcc_query_info = std::make_unique(); query_info.mvcc_query_info->resolve_locks = true; query_info.mvcc_query_info->read_tso = settings.read_tso; - RegionQueryInfo info; - info.region_id = region_info.region_id; - info.version = region_info.region_version; - info.conf_version = region_info.region_conf_version; - info.range_in_table = current_region->getHandleRangeByTable(table_id); - query_info.mvcc_query_info->regions_query_info.push_back(info); - query_info.mvcc_query_info->concurrent = 0.0; + for (auto & r : region_infos) + { + RegionQueryInfo info; + info.region_id = r.region_id; + info.version = r.region_version; + info.conf_version = r.region_conf_version; + auto current_region = context.getTMTContext().getKVStore()->getRegion(info.region_id); + if (!current_region) + { + std::vector region_ids; + for (auto & rr : region_infos) { + region_ids.push_back(rr.region_id); + } + throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); + } + info.range_in_table = current_region->getHandleRangeByTable(table_id); + query_info.mvcc_query_info->regions_query_info.push_back(info); + } + query_info.mvcc_query_info->concurrent = region_infos.size() > 1 ? 1.0 : 0.0; +// RegionQueryInfo info; +// info.region_id = region_info.region_id; +// info.version = region_info.region_version; +// info.conf_version = region_info.region_conf_version; +// info.range_in_table = current_region->getHandleRangeByTable(table_id); +// query_info.mvcc_query_info->regions_query_info.push_back(info); +// query_info.mvcc_query_info->concurrent = 0.0; + if (ts.next_read_engine() == tipb::EngineType::Local) { pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); @@ -466,6 +491,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & //} //pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); } + LOG_INFO(log, "dag execution stream size: " << region_infos.size()); if (pipeline.streams.empty()) { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index ce97ffb363a..1b517a2c8f8 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -78,7 +78,7 @@ class DAGQueryBlockInterpreter { public: DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const RegionInfo & region_info, const tipb::DAGRequest & rqst, + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos, const tipb::DAGRequest & rqst, ASTPtr dummp_query); ~DAGQueryBlockInterpreter() = default; @@ -112,7 +112,7 @@ class DAGQueryBlockInterpreter std::vector input_streams_vec; const DAGQueryBlock & query_block; const bool keep_session_timezone_info; - const RegionInfo & region_info; + const std::vector & region_infos; const tipb::DAGRequest & rqst; ASTPtr dummy_query; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 77fbd4d1664..03947ecfa8b 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -29,14 +29,12 @@ extern const int COP_BAD_DAG_REQUEST; // index = value; //} -DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, RegionID region_id_, UInt64 region_version_, - UInt64 region_conf_version_, const std::vector> & key_ranges_, +DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, const std::vector & regions_, + const std::vector> & key_ranges_, const tipb::DAGRequest & dag_request_) : context(context_), dag_context(dag_context_), - region_id(region_id_), - region_version(region_version_), - region_conf_version(region_conf_version_), + regions(regions_), key_ranges(key_ranges_), dag_request(dag_request_), metrics(context.getTiFlashMetrics()) diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index a76c116ebcc..29af0f80edc 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -11,6 +11,7 @@ #include #include #include "DAGQueryBlock.h" +#include "DAGDriver.h" namespace DB { @@ -30,7 +31,7 @@ class DAGQuerySource : public IQuerySource //static const String TOPN_NAME; //static const String LIMIT_NAME; - DAGQuerySource(Context & context_, DAGContext & dag_context_, RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, + DAGQuerySource(Context & context_, DAGContext & dag_context_, const std::vector & regions, const std::vector> & key_ranges_, const tipb::DAGRequest & dag_request_); std::tuple parse(size_t max_query_size) override; @@ -39,9 +40,6 @@ class DAGQuerySource : public IQuerySource //DAGContext & getDAGContext() const { return dag_context; }; - RegionID getRegionID() const { return region_id; } - UInt64 getRegionVersion() const { return region_version; } - UInt64 getRegionConfVersion() const { return region_conf_version; } const std::vector> & getKeyRanges() const { return key_ranges; } //bool hasSelection() const { return sel_index != -1; }; @@ -89,6 +87,7 @@ class DAGQuerySource : public IQuerySource tipb::EncodeType getEncodeType() const { return encode_type; } std::shared_ptr getQueryBlock() const { return query_block_tree; } + const std::vector & getRegions() const { return regions; } protected: //void assertValid(Int32 index, const String & name) const @@ -106,9 +105,7 @@ class DAGQuerySource : public IQuerySource Context & context; DAGContext & dag_context; - const RegionID region_id; - const UInt64 region_version; - const UInt64 region_conf_version; + const std::vector & regions; const std::vector> & key_ranges; const tipb::DAGRequest & dag_request; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 7b14a5a5483..ad69dafa72a 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -155,10 +155,11 @@ void constructExprBasedOnRange(Int32 handle_col_id, tipb::Expr & expr, HandleRan template bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, const std::vector> & region_ranges, - Int32 handle_col_id, tipb::Expr & handle_filter) + Int32 handle_col_id, tipb::Expr & handle_filter, Logger * log) { if (ranges.empty()) { + LOG_WARNING(log, "income key ranges is empty"); // generate an always false filter constructInt64LiteralTiExpr(handle_filter, 0); return false; @@ -204,6 +205,7 @@ bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, break; } } + LOG_DEBUG(log, "ret " <> & ranges, return ret; } +<<<<<<< HEAD static bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, const ImutRegionRangePtr & region_key_range, Int32 handle_col_id, tipb::Expr & handle_filter) +======= +bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, + const ImutRegionRangePtr & region_key_range, Int32 handle_col_id, tipb::Expr & handle_filter, Logger* log) +>>>>>>> c2d620553... support batch cop { + LOG_INFO(log, "pk_is_uint: " << pk_is_uint64); if (key_ranges.empty()) { + LOG_WARNING(log, "income key ranges is empty1"); constructInt64LiteralTiExpr(handle_filter, 0); return false; } @@ -277,18 +286,18 @@ static bool checkKeyRanges(const std::vector(update_handle_ranges, update_region_handle_ranges, handle_col_id, handle_filter); + return checkRangeAndGenExprIfNeeded(update_handle_ranges, update_region_handle_ranges, handle_col_id, handle_filter, log); } else - return checkRangeAndGenExprIfNeeded(handle_ranges, region_handle_ranges, handle_col_id, handle_filter); + return checkRangeAndGenExprIfNeeded(handle_ranges, region_handle_ranges, handle_col_id, handle_filter, log); } RegionException::RegionReadStatus InterpreterDAG::getRegionReadStatus(const RegionPtr & current_region) { if (!current_region) return RegionException::NOT_FOUND; - if (current_region->version() != dag.getRegionVersion() || current_region->confVer() != dag.getRegionConfVersion()) - return RegionException::VERSION_ERROR; +// if (current_region->version() != dag.getRegionVersion() || current_region->confVer() != dag.getRegionConfVersion()) + // return RegionException::VERSION_ERROR; if (current_region->isPendingRemove()) return RegionException::PENDING_REMOVE; return RegionException::OK; @@ -355,22 +364,22 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) if (handle_col_id == -1) handle_col_id = required_columns.size(); - auto current_region = context.getTMTContext().getKVStore()->getRegion(dag.getRegionID()); - auto region_read_status = getRegionReadStatus(current_region); - if (region_read_status != RegionException::OK) - { - std::vector region_ids; - region_ids.push_back(dag.getRegionID()); - LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << dag.getRegionID()); - throw RegionException(std::move(region_ids), region_read_status); - } - const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; - if (!checkKeyRanges(dag.getKeyRanges(), table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) - { - // need to add extra filter on handle column - filter_on_handle = true; - conditions.push_back(&handle_filter_expr); - } +// auto current_region = context.getTMTContext().getKVStore()->getRegion(dag.getRegionID()); +// auto region_read_status = getRegionReadStatus(current_region); +// if (region_read_status != RegionException::OK) +// { +// std::vector region_ids; +// region_ids.push_back(dag.getRegionID()); +// LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << dag.getRegionID()); +// throw RegionException(std::move(region_ids), region_read_status); +// } +// const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; +// if (!checkKeyRanges(dag.getKeyRanges(), table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) +// { +// // need to add extra filter on handle column +// filter_on_handle = true; +// conditions.push_back(&handle_filter_expr); +// } bool has_handle_column = (handle_col_id != (Int32)required_columns.size()); @@ -434,6 +443,7 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) query_info.mvcc_query_info = std::make_unique(); query_info.mvcc_query_info->resolve_locks = true; query_info.mvcc_query_info->read_tso = settings.read_tso; +<<<<<<< HEAD RegionQueryInfo info; info.region_id = dag.getRegionID(); info.version = dag.getRegionVersion(); @@ -448,6 +458,31 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) pipeline.streams = storage->remote_read(dag.getKeyRanges(), query_info, ts, context); } +======= + for (auto & r : dag.getRegions()) { + RegionQueryInfo info; + info.region_id = r.region_id; + info.version = r.region_version; + info.conf_version = r.region_conf_version; + auto current_region = context.getTMTContext().getKVStore()->getRegion(info.region_id); + if (!current_region) { + std::vector region_ids; + for (auto & rr : dag.getRegions()) { + region_ids.push_back(rr.region_id); + } + throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); + } + //if (!checkKeyRanges(dag.getKeyRanges(), table_id, storage->getPKType() == IManageableStorage::PKType::UINT64, current_region->getRange(), 0, handle_filter_expr, log)) + // throw Exception("Cop request only support full range scan for given region", + // ErrorCodes::COP_BAD_DAG_REQUEST); + info.range_in_table = current_region->getHandleRangeByTable(table_id); + query_info.mvcc_query_info->regions_query_info.push_back(info); + } + query_info.mvcc_query_info->concurrent = dag.getRegions().size() > 1 ? 1.0 : 0.0; + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); +>>>>>>> c2d620553... support batch cop + + LOG_INFO(log, "dag execution stream size: " << dag.getRegions().size()); if (pipeline.streams.empty()) { @@ -865,24 +900,24 @@ void InterpreterDAG::executeUnion(Pipeline & pipeline) } } -BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info) +BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const std::vector & region_infos) { if (!query_block.children.empty()) { std::vector input_streams_vec; for (auto & child : query_block.children) { - BlockInputStreams child_streams = executeQueryBlock(*child, region_info); + BlockInputStreams child_streams = executeQueryBlock(*child, region_infos); input_streams_vec.push_back(child_streams); } DAGQueryBlockInterpreter query_block_interpreter( - context, input_streams_vec, query_block, keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); + context, input_streams_vec, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST()); return query_block_interpreter.execute(); } else { DAGQueryBlockInterpreter query_block_interpreter( - context, {}, query_block, keep_session_timezone_info, region_info, dag.getDAGRequest(), dag.getAST()); + context, {}, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST()); return query_block_interpreter.execute(); } } @@ -892,8 +927,7 @@ BlockIO InterpreterDAG::execute() /// region_info should based on the source executor, however /// tidb does not support multi-table dag request yet, so /// it is ok to use the same region_info for the whole dag request - RegionInfo region_info(dag.getRegionID(), dag.getRegionVersion(), dag.getRegionConfVersion(), dag.getKeyRanges()); - BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), region_info); + BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), dag.getRegions()); Pipeline pipeline; pipeline.streams = streams; executeUnion(pipeline); diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index a4088ece56d..9e38b81b2e3 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -37,7 +37,7 @@ class InterpreterDAG : public IInterpreter BlockIO execute(); private: - BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block, const RegionInfo & region_info); + BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block, const std::vector & region_infos); void executeUnion(Pipeline & pipeline); /* void executeImpl(Pipeline & pipeline); diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp new file mode 100644 index 00000000000..efd7871c1ad --- /dev/null +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp @@ -0,0 +1,106 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNSUPPORTED_PARAMETER; +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes + +StreamingDAGBlockOutputStream::StreamingDAGBlockOutputStream(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, + std::vector && result_field_types_, Block && header_) + : writer(writer_), + result_field_types(std::move(result_field_types_)), + header(std::move(header_)), + records_per_chunk(records_per_chunk_), + current_records_num(0), + encode_type(encode_type_) +{ + if (encode_type == tipb::EncodeType::TypeDefault) + { + chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); + } + else if (encode_type == tipb::EncodeType::TypeChunk) + { + chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); + } + else if (encode_type == tipb::EncodeType::TypeCHBlock) + { + chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); + records_per_chunk = -1; + } + else + { + throw Exception("Only Default and Arrow encode type is supported in StreamingDAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); + } +} + +void StreamingDAGBlockOutputStream::writePrefix() +{ + //something to do here? +} + +void StreamingDAGBlockOutputStream::encodeChunkToDAGResponse() +{ + ::coprocessor::BatchResponse resp ; + + tipb::SelectResponse dag_response; + dag_response.set_encode_type(encode_type); + auto dag_chunk = dag_response.add_chunks(); + dag_chunk->set_rows_data(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); + dag_response.add_output_counts(current_records_num); + current_records_num = 0; + std::string dag_data; + dag_response.SerializeToString(&dag_data); + resp.set_data(dag_data); + + writer->Write(resp); +} + +void StreamingDAGBlockOutputStream::writeSuffix() +{ + // todo error handle + if (current_records_num > 0) + { + encodeChunkToDAGResponse(); + } +} + +void StreamingDAGBlockOutputStream::write(const Block & block) +{ + if (block.columns() != result_field_types.size()) + throw Exception("Output column size mismatch with field type size", ErrorCodes::LOGICAL_ERROR); + if (records_per_chunk == -1) + { + current_records_num = 0; + if (block.rows() > 0) + { + chunk_codec_stream->encode(block, 0, block.rows()); + encodeChunkToDAGResponse(); + } + } + else + { + size_t rows = block.rows(); + for (size_t row_index = 0; row_index < rows;) + { + if (current_records_num >= records_per_chunk) + { + encodeChunkToDAGResponse(); + } + const size_t upper = std::min(row_index + (records_per_chunk - current_records_num), rows); + chunk_codec_stream->encode(block, row_index, upper); + current_records_num += (upper - row_index); + row_index = upper; + } + } +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h new file mode 100644 index 00000000000..8edbf46d538 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +namespace DB +{ + +/// Serializes the stream of blocks in TiDB DAG response format. +/// TODO: May consider using some parallelism. +/// TODO: Consider using output schema in DAG request, do some conversion or checking between DAG schema and block schema. +class StreamingDAGBlockOutputStream : public IBlockOutputStream +{ +public: + StreamingDAGBlockOutputStream(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, + std::vector && result_field_types, Block && header_); + + Block getHeader() const override { return header; } + void write(const Block & block) override; + void writePrefix() override; + void writeSuffix() override; + void encodeChunkToDAGResponse(); + +private: + ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; + std::vector result_field_types; + Block header; + Int64 records_per_chunk; + std::unique_ptr chunk_codec_stream; + Int64 current_records_num; + tipb::EncodeType encode_type; +}; + +} // namespace DB diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp index 3713584be2e..dfdb2c86127 100644 --- a/dbms/src/Flash/CoprocessorHandler.cpp +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -59,8 +59,9 @@ grpc::Status CoprocessorHandler::execute() if (dag_request.has_is_rpn_expr() && dag_request.is_rpn_expr()) throw Exception("DAG request with rpn expression is not supported in TiFlash", ErrorCodes::NOT_IMPLEMENTED); tipb::SelectResponse dag_response; - DAGDriver driver(cop_context.db_context, dag_request, cop_context.kv_context.region_id(), - cop_context.kv_context.region_epoch().version(), cop_context.kv_context.region_epoch().conf_ver(), + std::vector regions; + regions.emplace_back(cop_context.kv_context.region_id(),cop_context.kv_context.region_epoch().version(), cop_context.kv_context.region_epoch().conf_ver()); + DAGDriver driver(cop_context.db_context, dag_request, regions, cop_request->start_ts() > 0 ? cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), std::move(key_ranges), dag_response); driver.execute(); diff --git a/dbms/src/Flash/CoprocessorHandler.h b/dbms/src/Flash/CoprocessorHandler.h index cf7aab438fe..afe6c8f4a74 100644 --- a/dbms/src/Flash/CoprocessorHandler.h +++ b/dbms/src/Flash/CoprocessorHandler.h @@ -8,6 +8,8 @@ #include #pragma GCC diagnostic pop #include +#include +#include namespace DB { diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 44c117c128e..19e63c76665 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -49,6 +49,25 @@ grpc::Status FlashService::Coprocessor( return ret; } +::grpc::Status FlashService:: BatchCoprocessor(::grpc::ServerContext* grpc_context, const ::coprocessor::BatchRequest* request, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer) +{ + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling batch coprocessor request: " << request->DebugString()); + + auto [context, status] = createDBContext(grpc_context); + if (!status.ok()) + { + return status; + } + + CoprocessorContext cop_context(context, request->context(), *grpc_context); + BatchCoprocessorHandler cop_handler(cop_context, request, writer); + + auto ret = cop_handler.execute(); + + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle coprocessor request done: " << ret.error_code() << ", " << ret.error_message()); + return ret; +} + grpc::Status FlashService::BatchCommands( grpc::ServerContext * grpc_context, grpc::ServerReaderWriter<::tikvpb::BatchCommandsResponse, tikvpb::BatchCommandsRequest> * stream) { diff --git a/dbms/src/Flash/FlashService.h b/dbms/src/Flash/FlashService.h index 83a678db5ec..b19139d1569 100644 --- a/dbms/src/Flash/FlashService.h +++ b/dbms/src/Flash/FlashService.h @@ -9,6 +9,8 @@ #include #pragma GCC diagnostic pop +#include "BatchCoprocessorHandler.h" + namespace DB { @@ -27,6 +29,8 @@ class FlashService final : public tikvpb::Tikv::Service, public std::enable_shar grpc::Status BatchCommands(grpc::ServerContext * grpc_context, grpc::ServerReaderWriter * stream) override; + ::grpc::Status BatchCoprocessor(::grpc::ServerContext* context, const ::coprocessor::BatchRequest* request, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer) override; + private: std::tuple createDBContext(const grpc::ServerContext * grpc_contex) const; diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index da4be172e44..25a3efd0f1d 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -7,6 +7,8 @@ #include +#include + namespace DB { From abbff57bcbb6390101b7b14ae9348624e7b75e77 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 12 Mar 2020 14:29:35 +0800 Subject: [PATCH 24/54] fix build error --- contrib/kvproto | 2 +- .../Coprocessor/DAGQueryBlockInterpreter.h | 22 +++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/contrib/kvproto b/contrib/kvproto index 80122c79a66..e9d8c4e28e7 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 80122c79a6646d84788559c14ea2bfdfde5812ff +Subproject commit e9d8c4e28e75c9e655e2038a40267bbd784302cc diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 1b517a2c8f8..1e9ff9c7a87 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -25,17 +25,17 @@ class Context; class Region; using RegionPtr = std::shared_ptr; -struct RegionInfo -{ - RegionInfo(RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, - const std::vector> & key_ranges_) - : region_id(region_id_), region_version(region_version_), region_conf_version(region_conf_version_), key_ranges(key_ranges_) - {} - const RegionID region_id; - const UInt64 region_version; - const UInt64 region_conf_version; - const std::vector> & key_ranges; -}; +//struct RegionInfo +//{ +// RegionInfo(RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, +// const std::vector> & key_ranges_) +// : region_id(region_id_), region_version(region_version_), region_conf_version(region_conf_version_), key_ranges(key_ranges_) +// {} +// const RegionID region_id; +// const UInt64 region_version; +// const UInt64 region_conf_version; +// const std::vector> & key_ranges; +//}; struct Pipeline { From 3ce43c348b6afadd299202cdfac085cc27cb73b4 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 12 Mar 2020 10:56:28 +0800 Subject: [PATCH 25/54] fix daily test fail --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 18 +++++++----------- .../fullstack-test/expr/datetime_literal.test | 14 ++++++++++++++ .../expr/empty_input_for_udaf.test | 3 --- 3 files changed, 21 insertions(+), 14 deletions(-) create mode 100644 tests/fullstack-test/expr/datetime_literal.test diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index ef4b3bd9bb5..20c79646e74 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -578,15 +578,11 @@ String DAGExpressionAnalyzer::alignReturnType( String DAGExpressionAnalyzer::appendCastIfNeeded( const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool explicit_cast) { - // do not append cast for column expr - // the main difficulty here is some column expr's field type - // does not have enough information, for example the enum type - if (isColumnExpr(expr)) + if (!isFunctionExpr(expr)) return expr_name; - if (isFunctionExpr(expr) && !expr.has_field_type()) + if (!expr.has_field_type()) { - // seems literal expr does not have field type, do not throw exception for literal expr throw Exception("Function Expression without field type", ErrorCodes::COP_BAD_DAG_REQUEST); } if (exprHasValidFieldType(expr)) @@ -668,15 +664,15 @@ String DAGExpressionAnalyzer::getActions(const tipb::Expr & expr, ExpressionActi if (isLiteralExpr(expr)) { Field value = decodeLiteral(expr); - DataTypePtr type = applyVisitor(FieldToDataType(), value); - ret = exprToString(expr, getCurrentInputColumns()) + "_" + type->getName(); + DataTypePtr flash_type = applyVisitor(FieldToDataType(), value); + DataTypePtr target_type = exprHasValidFieldType(expr) ? getDataTypeByFieldType(expr.field_type()) : flash_type; + ret = exprToString(expr, getCurrentInputColumns()) + "_" + target_type->getName(); if (!actions->getSampleBlock().has(ret)) { ColumnWithTypeAndName column; - column.column = type->createColumnConst(1, value); + column.column = target_type->createColumnConst(1, convertFieldToType(value, *target_type, flash_type.get())); column.name = ret; - column.type = type; - + column.type = target_type; actions->add(ExpressionAction::addColumn(column)); } } diff --git a/tests/fullstack-test/expr/datetime_literal.test b/tests/fullstack-test/expr/datetime_literal.test new file mode 100644 index 00000000000..1dc16629bee --- /dev/null +++ b/tests/fullstack-test/expr/datetime_literal.test @@ -0,0 +1,14 @@ +mysql> drop table if exists test.t +mysql> create table test.t(a int , b date) +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + +mysql> insert into test.t values (1, date'2020-01-01'); +mysql> insert into test.t values (1, date'2020-01-02'); +mysql> select /*+ read_from_storage(tiflash[t]) */ * from test.t where b > date'2020-01-01'; ++------+------------+ +| a | b | ++------+------------+ +| 1 | 2020-01-02 | ++------+------------+ diff --git a/tests/fullstack-test/expr/empty_input_for_udaf.test b/tests/fullstack-test/expr/empty_input_for_udaf.test index 3361930fe0d..8affefb7d7f 100644 --- a/tests/fullstack-test/expr/empty_input_for_udaf.test +++ b/tests/fullstack-test/expr/empty_input_for_udaf.test @@ -6,9 +6,6 @@ SLEEP 15 mysql> insert into test.t values (1, 1, 1, 1, 1, 1); mysql> insert into test.t values (1, 2, 3, NULL, NULL, 1); - -SLEEP 15 - mysql> select /*+ read_from_storage(tiflash[t]) */ count(1),count(a),count(b),count(d),count(NULL) from test.t where a > 10; +----------+----------+----------+----------+-------------+ | count(1) | count(a) | count(b) | count(d) | count(NULL) | From 7edcb4fb9d688f3bef29826b20eed8e1e02fafbb Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 12 Mar 2020 18:22:01 +0800 Subject: [PATCH 26/54] some bug fix --- .../DataStreams/CoprocessorBlockInputStream.h | 16 +++- .../DataStreams/NativeBlockInputStream.cpp | 11 +++ dbms/src/DataStreams/NativeBlockInputStream.h | 4 + .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 7 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 77 ++++++++++--------- dbms/src/Storages/IManageableStorage.cpp | 7 +- 6 files changed, 78 insertions(+), 44 deletions(-) diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h index dd767ebfa9f..26b87cb0f74 100644 --- a/dbms/src/DataStreams/CoprocessorBlockInputStream.h +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -1,9 +1,11 @@ #pragma once #include -#include +#include #include #include +#include +#include namespace DB { @@ -51,7 +53,16 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream } auto chunk = chunk_queue.front(); chunk_queue.pop(); - return codec.decode(chunk, schema); + switch (resp->encode_type()) { + case tipb::EncodeType::TypeCHBlock: + return CHBlockChunkCodec().decode(chunk, schema); + case tipb::EncodeType::TypeChunk: + return ArrowChunkCodec().decode(chunk, schema); + case tipb::EncodeType::TypeDefault: + return DefaultChunkCodec().decode(chunk, schema); + default: + throw Exception("Unsupported encode type", ErrorCodes::LOGICAL_ERROR); + } } private: @@ -85,7 +96,6 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream pingcap::coprocessor::ResponseIter resp_iter; DAGSchema schema; - ArrowChunkCodec codec; std::shared_ptr resp; std::queue chunk_queue; diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 787ba974546..ed0ac6b3a22 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -22,6 +22,10 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_, std::vector && output_names_) + : istr(istr_), server_revision(server_revision_), output_names(std::move(output_names_)) +{ +} NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_) : istr(istr_), server_revision(server_revision_) @@ -109,6 +113,11 @@ Block NativeBlockInputStream::readImpl() rows = index_block_it->num_rows; } + if (output_names.size() > 0 && output_names.size() != columns) + throw Exception("NativeBlockInputStream with explicity output name, but the block column size " + "is not equal to the size of output names", ErrorCodes::LOGICAL_ERROR); + bool explicit_output_name = output_names.size() > 0; + for (size_t i = 0; i < columns; ++i) { if (use_index) @@ -121,6 +130,8 @@ Block NativeBlockInputStream::readImpl() /// Name readBinary(column.name, istr); + if (explicit_output_name) + column.name = output_names[i]; /// Type String type_name; diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index 14ce18347e3..cc1ea5cb9e2 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -60,6 +60,8 @@ struct IndexForNativeFormat class NativeBlockInputStream : public IProfilingBlockInputStream { public: + /// provide output column names explicitly + NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_, std::vector && output_names_); /// If a non-zero server_revision is specified, additional block information may be expected and read. NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_); @@ -96,6 +98,8 @@ class NativeBlockInputStream : public IProfilingBlockInputStream PODArray avg_value_size_hints; + std::vector output_names; + void updateAvgValueSizeHints(const Block & block); }; diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 8f6e5d713d5..e8832666250 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -69,11 +69,14 @@ std::unique_ptr CHBlockChunkCodec::newCodecStream(const std::v return std::make_unique(field_types); } -Block CHBlockChunkCodec::decode(const tipb::Chunk & chunk, const DAGSchema &) +Block CHBlockChunkCodec::decode(const tipb::Chunk & chunk, const DAGSchema & schema) { const String & row_data = chunk.rows_data(); ReadBufferFromString read_buffer(row_data); - NativeBlockInputStream block_in(read_buffer, 0); + std::vector output_names; + for (const auto & c : schema) + output_names.push_back(c.first); + NativeBlockInputStream block_in(read_buffer, 0, std::move(output_names)); return block_in.read(); } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index a807712d356..27fb26e7d55 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -47,8 +47,8 @@ extern const int COP_BAD_DAG_REQUEST; } // namespace ErrorCodes DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos_, const tipb::DAGRequest & rqst_, - ASTPtr dummy_query_) + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos_, + const tipb::DAGRequest & rqst_, ASTPtr dummy_query_) : context(context_), input_streams_vec(input_streams_vec_), query_block(query_block_), @@ -63,6 +63,12 @@ DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std for (auto & condition : query_block.selection->selection().conditions()) conditions.push_back(&condition); } + const Settings & settings = context.getSettingsRef(); + max_streams = settings.max_threads; + if (max_streams > 1) + { + max_streams *= settings.max_streams_to_max_threads_ratio; + } } template @@ -155,7 +161,7 @@ void constructExprBasedOnRange(Int32 handle_col_id, tipb::Expr & expr, HandleRan template bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, const std::vector> & region_ranges, - Int32 handle_col_id, tipb::Expr & handle_filter, Logger *log) + Int32 handle_col_id, tipb::Expr & handle_filter, Logger * log) { if (ranges.empty()) { @@ -205,7 +211,7 @@ bool checkRangeAndGenExprIfNeeded(std::vector> & ranges, break; } } - LOG_DEBUG(log, "ret " <getRegion(region_info.region_id); -// auto region_read_status = getRegionReadStatus(current_region); -// if (region_read_status != RegionException::OK) -// { -// std::vector region_ids; -// region_ids.push_back(region_info.region_id); -// LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << region_info.region_id); -// throw RegionException(std::move(region_ids), region_read_status); -// } -// const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; -// if (!checkKeyRanges(region_info.key_ranges, table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) -// { -// // need to add extra filter on handle column -// filter_on_handle = true; -// conditions.push_back(&handle_filter_expr); -// } + // auto current_region = context.getTMTContext().getKVStore()->getRegion(region_info.region_id); + // auto region_read_status = getRegionReadStatus(current_region); + // if (region_read_status != RegionException::OK) + // { + // std::vector region_ids; + // region_ids.push_back(region_info.region_id); + // LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << region_info.region_id); + // throw RegionException(std::move(region_ids), region_read_status); + // } + // const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; + // if (!checkKeyRanges(region_info.key_ranges, table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) + // { + // // need to add extra filter on handle column + // filter_on_handle = true; + // conditions.push_back(&handle_filter_expr); + // } bool has_handle_column = (handle_col_id != (Int32)required_columns.size()); @@ -424,12 +430,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & } size_t max_block_size = settings.max_block_size; - max_streams = settings.max_threads; QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; - if (max_streams > 1) - { - max_streams *= settings.max_streams_to_max_threads_ratio; - } if (query_block.selection) { @@ -456,7 +457,8 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & if (!current_region) { std::vector region_ids; - for (auto & rr : region_infos) { + for (auto & rr : region_infos) + { region_ids.push_back(rr.region_id); } throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); @@ -465,13 +467,13 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & query_info.mvcc_query_info->regions_query_info.push_back(info); } query_info.mvcc_query_info->concurrent = region_infos.size() > 1 ? 1.0 : 0.0; -// RegionQueryInfo info; -// info.region_id = region_info.region_id; -// info.version = region_info.region_version; -// info.conf_version = region_info.region_conf_version; -// info.range_in_table = current_region->getHandleRangeByTable(table_id); -// query_info.mvcc_query_info->regions_query_info.push_back(info); -// query_info.mvcc_query_info->concurrent = 0.0; + // RegionQueryInfo info; + // info.region_id = region_info.region_id; + // info.version = region_info.region_version; + // info.conf_version = region_info.region_conf_version; + // info.range_in_table = current_region->getHandleRangeByTable(table_id); + // query_info.mvcc_query_info->regions_query_info.push_back(info); + // query_info.mvcc_query_info->concurrent = 0.0; if (ts.next_read_engine() == tipb::EngineType::Local) { @@ -586,7 +588,11 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p right_streams = input_streams_vec[1]; } std::vector join_output_columns; - for (auto const & p : left_streams[0]->getHeader().getNamesAndTypesList()) + for (auto const & p : input_streams_vec[0][0]->getHeader().getNamesAndTypesList()) + { + join_output_columns.emplace_back(p.name, p.type); + } + for (auto const & p : input_streams_vec[1][0]->getHeader().getNamesAndTypesList()) { join_output_columns.emplace_back(p.name, p.type); } @@ -595,7 +601,6 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p for (auto const & p : right_streams[0]->getHeader().getNamesAndTypesList()) { columns_added_by_join.emplace_back(p.name, p.type); - join_output_columns.emplace_back(p.name, p.type); } if (!query_block.aggregation) @@ -804,7 +809,7 @@ void DAGQueryBlockInterpreter::getAndLockStorageWithSchemaVersion(TableID table_ return std::make_tuple(nullptr, nullptr, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, false); } - if (storage_->engineType() != ::TiDB::StorageEngine::TMT && storage_->engineType() != ::TiDB::StorageEngine::DM) + if (storage_->engineType() != ::TiDB::StorageEngine::TMT && storage_->engineType() != ::TiDB::StorageEngine::DT) { throw Exception("Specifying schema_version for non-managed storage: " + storage_->getName() + ", table: " + storage_->getTableName() + ", id: " + DB::toString(table_id) + " is not allowed", diff --git a/dbms/src/Storages/IManageableStorage.cpp b/dbms/src/Storages/IManageableStorage.cpp index 8fc643c4bfa..e0278ef3775 100644 --- a/dbms/src/Storages/IManageableStorage.cpp +++ b/dbms/src/Storages/IManageableStorage.cpp @@ -1,3 +1,4 @@ +#include #include namespace DB @@ -96,7 +97,7 @@ BlockInputStreams IManageableStorage::remote_read(const std::vector( - sample_block, [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); + BlockInputStreamPtr input = std::make_shared(sample_block, + [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); return {input}; }; } // namespace DB From 72083dca4833f8d1a543b653e2b1dc84f93fa346 Mon Sep 17 00:00:00 2001 From: xufei Date: Sat, 14 Mar 2020 08:50:48 +0800 Subject: [PATCH 27/54] log dag execution time without encode to chunk --- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 5 +++++ dbms/src/Storages/IManageableStorage.cpp | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index a3f9773576b..55b88621dfe 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -56,6 +56,11 @@ try if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) return; + if (auto * p_stream = dynamic_cast(streams.in.get())) + { + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": dag request without encode cost: " + << p_stream->getProfileInfo().execution_time/(double)1000000 << " seconds."); + } // add ExecutorExecutionSummary info for (auto & p_streams : dag_context.profile_streams_list) { diff --git a/dbms/src/Storages/IManageableStorage.cpp b/dbms/src/Storages/IManageableStorage.cpp index e0278ef3775..b3f79962060 100644 --- a/dbms/src/Storages/IManageableStorage.cpp +++ b/dbms/src/Storages/IManageableStorage.cpp @@ -74,7 +74,7 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: new_ts->set_next_read_engine(tipb::EngineType::Local); exec->set_allocated_tbl_scan(new_ts); - dag_req.set_encode_type(tipb::EncodeType::TypeChunk); + dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); } BlockInputStreams IManageableStorage::remote_read(const std::vector> & key_ranges, From 1dbd6b836ebccf272ce824f4d77e950aa78987eb Mon Sep 17 00:00:00 2001 From: xufei Date: Sat, 14 Mar 2020 10:54:42 +0800 Subject: [PATCH 28/54] fix bug --- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 55b88621dfe..3c9da9f9190 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -59,7 +59,7 @@ try if (auto * p_stream = dynamic_cast(streams.in.get())) { LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": dag request without encode cost: " - << p_stream->getProfileInfo().execution_time/(double)1000000 << " seconds."); + << p_stream->getProfileInfo().execution_time/(double)1000000000 << " seconds."); } // add ExecutorExecutionSummary info for (auto & p_streams : dag_context.profile_streams_list) From 28114f9399f4495bf4449deb1f9bede9792fc4cd Mon Sep 17 00:00:00 2001 From: xufei Date: Sat, 14 Mar 2020 14:26:42 +0800 Subject: [PATCH 29/54] fix bug --- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 3c9da9f9190..d8f2f44f8cb 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -54,13 +54,13 @@ try dag_response, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), streams.in->getHeader()); copyData(*streams.in, *dag_output_stream); - if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) - return; if (auto * p_stream = dynamic_cast(streams.in.get())) { LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": dag request without encode cost: " - << p_stream->getProfileInfo().execution_time/(double)1000000000 << " seconds."); + << p_stream->getProfileInfo().execution_time/(double)1000000000 << " seconds."); } + if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) + return; // add ExecutorExecutionSummary info for (auto & p_streams : dag_context.profile_streams_list) { From 8106ac602be0d39856457a4fbc417d5b8a1d0537 Mon Sep 17 00:00:00 2001 From: xufei Date: Sat, 14 Mar 2020 15:11:08 +0800 Subject: [PATCH 30/54] log dag execution time without encode --- dbms/src/DataStreams/IProfilingBlockInputStream.cpp | 4 ++++ dbms/src/Flash/Coprocessor/DAGDriver.cpp | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 04907c9a710..d8512b04966 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -103,6 +103,7 @@ Block IProfilingBlockInputStream::read(FilterPtr & res_filter, bool return_filte void IProfilingBlockInputStream::readPrefix() { + auto start_time = info.total_stopwatch.elapsed(); readPrefixImpl(); forEachChild([&] (IBlockInputStream & child) @@ -110,11 +111,13 @@ void IProfilingBlockInputStream::readPrefix() child.readPrefix(); return false; }); + info.updateExecutionTime(info.total_stopwatch.elapsed() - start_time); } void IProfilingBlockInputStream::readSuffix() { + auto start_time = info.total_stopwatch.elapsed(); forEachChild([&] (IBlockInputStream & child) { child.readSuffix(); @@ -122,6 +125,7 @@ void IProfilingBlockInputStream::readSuffix() }); readSuffixImpl(); + info.updateExecutionTime(info.total_stopwatch.elapsed() - start_time); } diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index d8f2f44f8cb..984c7dfab7d 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -126,6 +126,11 @@ try writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), streams.in->getHeader()); copyData(*streams.in, *dag_output_stream); + if (auto * p_stream = dynamic_cast(streams.in.get())) + { + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": dag request without encode cost: " + << p_stream->getProfileInfo().execution_time/(double)1000000000 << " seconds."); + } if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) return; // add ExecutorExecutionSummary info From cb294a711b2eede3c4e2799a5656133379c05856 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Mon, 16 Mar 2020 12:34:01 +0800 Subject: [PATCH 31/54] make encode multi processors --- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 13 ++-- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 7 ++- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 6 +- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 8 +++ .../StreamingDAGBlockOutputStream.cpp | 61 +++++++++++-------- .../StreamingDAGBlockOutputStream.h | 15 +++-- dbms/src/Storages/IManageableStorage.cpp | 2 + 7 files changed, 70 insertions(+), 42 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 984c7dfab7d..1f8006ea274 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -115,16 +114,20 @@ void DAGDriver::batchExecute(::grpc::ServerWriter< ::coprocessor::BatchResponse> try { DAGContext dag_context(dag_request.executors_size()); - DAGQuerySource dag(context, dag_context, regions, key_ranges, dag_request); + DAGQuerySource dag(context, dag_context, regions, key_ranges, dag_request, writer); BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); if (!streams.in || streams.out) // Only query is allowed, so streams.in must not be null and streams.out must be null throw Exception("DAG is not query.", ErrorCodes::LOGICAL_ERROR); - BlockOutputStreamPtr dag_output_stream = std::make_shared( - writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), streams.in->getHeader()); - copyData(*streams.in, *dag_output_stream); + //BlockOutputStreamPtr dag_output_stream = std::make_shared( + // writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), streams.in->getHeader()); + //copyData(*streams.in, *dag_output_stream); + + streams.in->readPrefix(); + while(streams.in->read()); + streams.in->readSuffix(); if (auto * p_stream = dynamic_cast(streams.in.get())) { diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 03947ecfa8b..bd21cd7b5e6 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -31,8 +31,11 @@ extern const int COP_BAD_DAG_REQUEST; DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, const std::vector & regions_, const std::vector> & key_ranges_, - const tipb::DAGRequest & dag_request_) - : context(context_), + const tipb::DAGRequest & dag_request_, + ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) + : + writer(writer_), + context(context_), dag_context(dag_context_), regions(regions_), key_ranges(key_ranges_), diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index 29af0f80edc..7e3d83d632c 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -32,7 +32,8 @@ class DAGQuerySource : public IQuerySource //static const String LIMIT_NAME; DAGQuerySource(Context & context_, DAGContext & dag_context_, const std::vector & regions, - const std::vector> & key_ranges_, const tipb::DAGRequest & dag_request_); + const std::vector> & key_ranges_, const tipb::DAGRequest & dag_request_, + ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_ = nullptr); std::tuple parse(size_t max_query_size) override; String str(size_t max_query_size) override; @@ -89,6 +90,8 @@ class DAGQuerySource : public IQuerySource std::shared_ptr getQueryBlock() const { return query_block_tree; } const std::vector & getRegions() const { return regions; } + ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; + protected: //void assertValid(Int32 index, const String & name) const //{ @@ -121,7 +124,6 @@ class DAGQuerySource : public IQuerySource std::vector result_field_types; tipb::EncodeType encode_type; std::shared_ptr query_block_tree; - ASTPtr ast; }; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 87c6c56b651..ed561ba1d12 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -928,8 +929,15 @@ BlockIO InterpreterDAG::execute() /// tidb does not support multi-table dag request yet, so /// it is ok to use the same region_info for the whole dag request BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), dag.getRegions()); + + if (dag.writer != nullptr) { + for (auto & stream : streams) + stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); + } + Pipeline pipeline; pipeline.streams = streams; + executeUnion(pipeline); BlockIO res; diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp index efd7871c1ad..2ac9cc64016 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp @@ -13,9 +13,11 @@ extern const int UNSUPPORTED_PARAMETER; extern const int LOGICAL_ERROR; } // namespace ErrorCodes -StreamingDAGBlockOutputStream::StreamingDAGBlockOutputStream(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, +StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr input_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, std::vector && result_field_types_, Block && header_) - : writer(writer_), + : input(input_), + finished(false), + writer(writer_), result_field_types(std::move(result_field_types_)), header(std::move(header_)), records_per_chunk(records_per_chunk_), @@ -41,12 +43,12 @@ StreamingDAGBlockOutputStream::StreamingDAGBlockOutputStream(::grpc::ServerWrite } } -void StreamingDAGBlockOutputStream::writePrefix() +void StreamingDAGBlockInputStream::readPrefix() { - //something to do here? + input->readPrefix(); } -void StreamingDAGBlockOutputStream::encodeChunkToDAGResponse() +void StreamingDAGBlockInputStream::encodeChunkToDAGResponse() { ::coprocessor::BatchResponse resp ; @@ -64,43 +66,48 @@ void StreamingDAGBlockOutputStream::encodeChunkToDAGResponse() writer->Write(resp); } -void StreamingDAGBlockOutputStream::writeSuffix() +void StreamingDAGBlockInputStream::readSuffix() { // todo error handle if (current_records_num > 0) { encodeChunkToDAGResponse(); } + input->readSuffix(); } -void StreamingDAGBlockOutputStream::write(const Block & block) +Block StreamingDAGBlockInputStream::readImpl() { - if (block.columns() != result_field_types.size()) - throw Exception("Output column size mismatch with field type size", ErrorCodes::LOGICAL_ERROR); - if (records_per_chunk == -1) - { - current_records_num = 0; - if (block.rows() > 0) + if (finished) + return {}; + while(Block block = input->read()) { + if (!block) { - chunk_codec_stream->encode(block, 0, block.rows()); - encodeChunkToDAGResponse(); + finished = true; + return {}; } - } - else - { - size_t rows = block.rows(); - for (size_t row_index = 0; row_index < rows;) - { - if (current_records_num >= records_per_chunk) - { + if (block.columns() != result_field_types.size()) + throw Exception("Output column size mismatch with field type size", ErrorCodes::LOGICAL_ERROR); + if (records_per_chunk == -1) { + current_records_num = 0; + if (block.rows() > 0) { + chunk_codec_stream->encode(block, 0, block.rows()); encodeChunkToDAGResponse(); } - const size_t upper = std::min(row_index + (records_per_chunk - current_records_num), rows); - chunk_codec_stream->encode(block, row_index, upper); - current_records_num += (upper - row_index); - row_index = upper; + } else { + size_t rows = block.rows(); + for (size_t row_index = 0; row_index < rows;) { + if (current_records_num >= records_per_chunk) { + encodeChunkToDAGResponse(); + } + const size_t upper = std::min(row_index + (records_per_chunk - current_records_num), rows); + chunk_codec_stream->encode(block, row_index, upper); + current_records_num += (upper - row_index); + row_index = upper; + } } } + return {}; } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h index 8edbf46d538..08fae5198cb 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -17,19 +17,22 @@ namespace DB /// Serializes the stream of blocks in TiDB DAG response format. /// TODO: May consider using some parallelism. /// TODO: Consider using output schema in DAG request, do some conversion or checking between DAG schema and block schema. -class StreamingDAGBlockOutputStream : public IBlockOutputStream +class StreamingDAGBlockInputStream : public IProfilingBlockInputStream { public: - StreamingDAGBlockOutputStream(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, + StreamingDAGBlockInputStream(BlockInputStreamPtr input_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, std::vector && result_field_types, Block && header_); Block getHeader() const override { return header; } - void write(const Block & block) override; - void writePrefix() override; - void writeSuffix() override; + Block readImpl() override; + String getName() const override { return "StreamingWriter"; } + void readPrefix() override; + void readSuffix() override; void encodeChunkToDAGResponse(); private: + BlockInputStreamPtr input; + bool finished; ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; std::vector result_field_types; Block header; diff --git a/dbms/src/Storages/IManageableStorage.cpp b/dbms/src/Storages/IManageableStorage.cpp index b3f79962060..e8ad21f1ee2 100644 --- a/dbms/src/Storages/IManageableStorage.cpp +++ b/dbms/src/Storages/IManageableStorage.cpp @@ -80,6 +80,7 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: BlockInputStreams IManageableStorage::remote_read(const std::vector> & key_ranges, UInt64 read_tso, const DAGQueryBlock & remote_query_block, Context & context) { + std::cout<< "begin remote read" < cop_key_ranges; for (const auto & key_range : key_ranges) { @@ -114,6 +115,7 @@ BlockInputStreams IManageableStorage::remote_read(const std::vector(sample_block, [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); return {input}; From 63d7a9ff4c7fd3d4a2163e27374041c652b3d5e1 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Mon, 16 Mar 2020 20:43:43 +0800 Subject: [PATCH 32/54] fix --- .../src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 10 +++++++++- dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h | 3 ++- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 10 ++-------- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 27fb26e7d55..2f2024f2770 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -48,7 +49,7 @@ extern const int COP_BAD_DAG_REQUEST; DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos_, - const tipb::DAGRequest & rqst_, ASTPtr dummy_query_) + const tipb::DAGRequest & rqst_, ASTPtr dummy_query_, const DAGQuerySource & dag_) : context(context_), input_streams_vec(input_streams_vec_), query_block(query_block_), @@ -56,6 +57,7 @@ DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std region_infos(region_infos_), rqst(rqst_), dummy_query(std::move(dummy_query_)), + dag(dag_), log(&Logger::get("DAGQueryBlockInterpreter")) { if (query_block.selection != nullptr) @@ -1048,6 +1050,12 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) executeLimit(pipeline); //recordProfileStreams(pipeline, dag.getLimitIndex()); } + + if (dag.writer != nullptr) { + for (auto & stream : pipeline.streams) + stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); + } + if (query_block.source->tp() == tipb::ExecType::TypeJoin) { // add the diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 1e9ff9c7a87..d45920f32fd 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -79,7 +79,7 @@ class DAGQueryBlockInterpreter public: DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos, const tipb::DAGRequest & rqst, - ASTPtr dummp_query); + ASTPtr dummp_query, const DAGQuerySource & dag_); ~DAGQueryBlockInterpreter() = default; @@ -132,6 +132,7 @@ class DAGQueryBlockInterpreter tipb::Expr handle_filter_expr; Int32 handle_col_id = -1; std::vector conditions; + const DAGQuerySource & dag; Poco::Logger * log; }; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index ed561ba1d12..0e325f0dca5 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -912,13 +911,13 @@ BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, input_streams_vec.push_back(child_streams); } DAGQueryBlockInterpreter query_block_interpreter( - context, input_streams_vec, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST()); + context, input_streams_vec, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST(), dag); return query_block_interpreter.execute(); } else { DAGQueryBlockInterpreter query_block_interpreter( - context, {}, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST()); + context, {}, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST(), dag); return query_block_interpreter.execute(); } } @@ -930,11 +929,6 @@ BlockIO InterpreterDAG::execute() /// it is ok to use the same region_info for the whole dag request BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), dag.getRegions()); - if (dag.writer != nullptr) { - for (auto & stream : streams) - stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); - } - Pipeline pipeline; pipeline.streams = streams; From c8b66d1391cd128dd470fd7c28f3851bd9a49dc8 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 17 Mar 2020 12:45:08 +0800 Subject: [PATCH 33/54] parallel encode --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 4 ++-- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 17 ++++++++++++++++- .../StreamingDAGBlockOutputStream.cpp | 4 ++-- .../Coprocessor/StreamingDAGBlockOutputStream.h | 4 ++-- 5 files changed, 23 insertions(+), 8 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 2f2024f2770..c234a99b6e6 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -1051,8 +1051,8 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) //recordProfileStreams(pipeline, dag.getLimitIndex()); } - if (dag.writer != nullptr) { - for (auto & stream : pipeline.streams) + if (dag.writer != nullptr && query_block.isRootQueryBlock()) { + for (auto & stream : pipeline.streams ) stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); } diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index bd21cd7b5e6..6230c7bf6a4 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -34,7 +34,7 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, co const tipb::DAGRequest & dag_request_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) : - writer(writer_), + writer(std::make_shared(writer_)), context(context_), dag_context(dag_context_), regions(regions_), diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index 7e3d83d632c..abe730dee0c 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -20,6 +20,21 @@ class Context; class TiFlashMetrics; using TiFlashMetricsPtr = std::shared_ptr; +struct StreamWriter { + ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; + std::mutex write_mutex; + + StreamWriter(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) : writer(writer_) {} + + void write(const ::coprocessor::BatchResponse & data) + { + std::lock_guard lk(write_mutex); + writer->Write(data); + } +}; + +using StreamWriterPtr = std::shared_ptr; + /// Query source of a DAG request via gRPC. /// This is also an IR of a DAG. class DAGQuerySource : public IQuerySource @@ -90,7 +105,7 @@ class DAGQuerySource : public IQuerySource std::shared_ptr getQueryBlock() const { return query_block_tree; } const std::vector & getRegions() const { return regions; } - ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; + StreamWriterPtr writer; protected: //void assertValid(Int32 index, const String & name) const diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp index 2ac9cc64016..a77bb22a5db 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp @@ -13,7 +13,7 @@ extern const int UNSUPPORTED_PARAMETER; extern const int LOGICAL_ERROR; } // namespace ErrorCodes -StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr input_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, +StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, std::vector && result_field_types_, Block && header_) : input(input_), finished(false), @@ -63,7 +63,7 @@ void StreamingDAGBlockInputStream::encodeChunkToDAGResponse() dag_response.SerializeToString(&dag_data); resp.set_data(dag_data); - writer->Write(resp); + writer->write(resp); } void StreamingDAGBlockInputStream::readSuffix() diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h index 08fae5198cb..f2d7a57f0ad 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h @@ -20,7 +20,7 @@ namespace DB class StreamingDAGBlockInputStream : public IProfilingBlockInputStream { public: - StreamingDAGBlockInputStream(BlockInputStreamPtr input_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, + StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, std::vector && result_field_types, Block && header_); Block getHeader() const override { return header; } @@ -33,7 +33,7 @@ class StreamingDAGBlockInputStream : public IProfilingBlockInputStream private: BlockInputStreamPtr input; bool finished; - ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; + StreamWriterPtr writer; std::vector result_field_types; Block header; Int64 records_per_chunk; From b33da95abda9179757b709fe590ca0079269cd9c Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 17 Mar 2020 19:48:23 +0800 Subject: [PATCH 34/54] refine code of batch coprocessor --- dbms/src/Debug/dbgFuncCoprocessor.cpp | 7 +- dbms/src/Flash/BatchCoprocessorHandler.cpp | 241 +++++++++--------- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 25 +- dbms/src/Flash/Coprocessor/DAGDriver.h | 24 +- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 10 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 16 +- dbms/src/Flash/CoprocessorHandler.cpp | 11 +- 7 files changed, 169 insertions(+), 165 deletions(-) diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index 216f8329ed5..3037b20555c 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -74,7 +74,7 @@ BlockInputStreamPtr dbgFuncDAG(Context & context, const ASTs & args) auto managed_storage = std::dynamic_pointer_cast(storage); if (!managed_storage // || !(managed_storage->engineType() == ::TiDB::StorageEngine::DT - || managed_storage->engineType() == ::TiDB::StorageEngine::TMT)) + || managed_storage->engineType() == ::TiDB::StorageEngine::TMT)) throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); return managed_storage->getTableInfo(); }, @@ -644,9 +644,8 @@ tipb::SelectResponse executeDAGRequest(Context & context, const tipb::DAGRequest LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); tipb::SelectResponse dag_response; std::vector regions; - regions.emplace_back(region_id, region_version, region_conf_version); - DAGDriver driver(context, dag_request, regions, start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, - std::move(key_ranges), dag_response, true); + regions.emplace_back(RegionInfo(region_id, region_version, region_conf_version, std::move(key_ranges))); + DAGDriver driver(context, dag_request, regions, start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, dag_response, true); driver.execute(); LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); return dag_response; diff --git a/dbms/src/Flash/BatchCoprocessorHandler.cpp b/dbms/src/Flash/BatchCoprocessorHandler.cpp index c61de88c205..8109301510c 100644 --- a/dbms/src/Flash/BatchCoprocessorHandler.cpp +++ b/dbms/src/Flash/BatchCoprocessorHandler.cpp @@ -1,5 +1,4 @@ #include - #include #include #include @@ -12,136 +11,148 @@ namespace DB { - namespace ErrorCodes - { - extern const int NOT_IMPLEMENTED; - } +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} - BatchCoprocessorHandler::BatchCoprocessorHandler( - CoprocessorContext & cop_context_, const coprocessor::BatchRequest * cop_request_, ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) - : cop_context(cop_context_), cop_request(cop_request_), writer(writer_), log(&Logger::get("BatchCoprocessorHandler")) - {} +BatchCoprocessorHandler::BatchCoprocessorHandler(CoprocessorContext & cop_context_, const coprocessor::BatchRequest * cop_request_, + ::grpc::ServerWriter<::coprocessor::BatchResponse> * writer_) + : cop_context(cop_context_), cop_request(cop_request_), writer(writer_), log(&Logger::get("BatchCoprocessorHandler")) +{} - grpc::Status BatchCoprocessorHandler::execute() - try +grpc::Status BatchCoprocessorHandler::execute() +try +{ + switch (cop_request->tp()) { - switch (cop_request->tp()) + case COP_REQ_TYPE_DAG: { - case COP_REQ_TYPE_DAG: + tipb::DAGRequest dag_request; + dag_request.ParseFromString(cop_request->data()); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); + std::vector regions; + for (auto & r : cop_request->regions()) { std::vector> key_ranges; - tipb::DAGRequest dag_request; - dag_request.ParseFromString(cop_request->data()); - LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); - std::vector regions; - for (auto & r : cop_request->regions()) { - regions.emplace_back(r.region_id(), r.region_epoch().version(), r.region_epoch().conf_ver()); - } - tipb::SelectResponse dag_response; // unused - DAGDriver driver(cop_context.db_context, dag_request, regions, cop_request->start_ts() > 0 ? - cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), std::move(key_ranges), - dag_response); - // batch execution; - driver.batchExecute(writer); - if (dag_response.has_error()) + for (auto & range : r.ranges()) { - err_response.set_other_error(dag_response.error().msg()); - writer->Write(err_response); + std::string start_key(range.start()); + DecodedTiKVKey start(std::move(start_key)); + std::string end_key(range.end()); + DecodedTiKVKey end(std::move(end_key)); + key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); } - LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); - break; + regions.emplace_back( + RegionInfo(r.region_id(), r.region_epoch().version(), r.region_epoch().conf_ver(), std::move(key_ranges))); + } + tipb::SelectResponse dag_response; // unused + DAGDriver driver(cop_context.db_context, dag_request, regions, + cop_request->start_ts() > 0 ? cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), + dag_response); + // batch execution; + driver.batchExecute(writer); + if (dag_response.has_error()) + { + err_response.set_other_error(dag_response.error().msg()); + writer->Write(err_response); } - case COP_REQ_TYPE_ANALYZE: - case COP_REQ_TYPE_CHECKSUM: - default: - throw Exception( - "Coprocessor request type " + std::to_string(cop_request->tp()) + " is not implemented", ErrorCodes::NOT_IMPLEMENTED); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); + break; } - return grpc::Status::OK; + case COP_REQ_TYPE_ANALYZE: + case COP_REQ_TYPE_CHECKSUM: + default: + throw Exception( + "Coprocessor request type " + std::to_string(cop_request->tp()) + " is not implemented", ErrorCodes::NOT_IMPLEMENTED); } - catch (const LockException & e) - { - LOG_ERROR(log, - __PRETTY_FUNCTION__ << ": LockException: region " << cop_request->context().region_id() << "\n" - << e.getStackTrace().toString()); + return grpc::Status::OK; +} +catch (const LockException & e) +{ + LOG_ERROR(log, + __PRETTY_FUNCTION__ << ": LockException: region " << cop_request->context().region_id() << "\n" + << e.getStackTrace().toString()); - //for (int i = 0; i < cop_request->regions_size(); i++) { - // auto * status = cop_response->add_region_status(); - // if (cop_request->regions(i).region_id() == e.region_id) - // { - // auto * lock_info = status->mutable_locked(); - // lock_info->set_key(e.lock_infos[0]->key); - // lock_info->set_primary_lock(e.lock_infos[0]->primary_lock); - // lock_info->set_lock_ttl(e.lock_infos[0]->lock_ttl); - // lock_info->set_lock_version(e.lock_infos[0]->lock_version); - // } - // else - // { - // status->set_success(true); - // } - //} - // return ok so TiDB has the chance to see the LockException - return grpc::Status::OK; - } - catch (const RegionException & e) + //for (int i = 0; i < cop_request->regions_size(); i++) { + // auto * status = cop_response->add_region_status(); + // if (cop_request->regions(i).region_id() == e.region_id) + // { + // auto * lock_info = status->mutable_locked(); + // lock_info->set_key(e.lock_infos[0]->key); + // lock_info->set_primary_lock(e.lock_infos[0]->primary_lock); + // lock_info->set_lock_ttl(e.lock_infos[0]->lock_ttl); + // lock_info->set_lock_version(e.lock_infos[0]->lock_version); + // } + // else + // { + // status->set_success(true); + // } + //} + // return ok so TiDB has the chance to see the LockException + return grpc::Status::OK; +} +catch (const RegionException & e) +{ + LOG_ERROR(log, + __PRETTY_FUNCTION__ << ": RegionException: region " << cop_request->context().region_id() << "\n" + << e.getStackTrace().toString()); + //errorpb::Error * region_err; + switch (e.status) { - LOG_ERROR(log, - __PRETTY_FUNCTION__ << ": RegionException: region " << cop_request->context().region_id() << "\n" - << e.getStackTrace().toString()); - //errorpb::Error * region_err; - switch (e.status) - { - case RegionException::RegionReadStatus::NOT_FOUND: - case RegionException::RegionReadStatus::PENDING_REMOVE: - for (auto region_id : e.region_ids) { - auto *status = err_response.add_region_status(); - status->mutable_region_error()->mutable_region_not_found()->set_region_id(region_id); - status->set_region_id(region_id); - } - break; - case RegionException::RegionReadStatus::VERSION_ERROR: + case RegionException::RegionReadStatus::NOT_FOUND: + case RegionException::RegionReadStatus::PENDING_REMOVE: + for (auto region_id : e.region_ids) + { + auto * status = err_response.add_region_status(); + status->mutable_region_error()->mutable_region_not_found()->set_region_id(region_id); + status->set_region_id(region_id); + } + break; + case RegionException::RegionReadStatus::VERSION_ERROR: - for (auto region_id : e.region_ids) { - auto * status = err_response.add_region_status(); - status->mutable_region_error()->mutable_epoch_not_match(); - status->set_region_id(region_id); - } - break; - default: - // should not happen - break; - } - // TODO: This should not be the last response, we should return data and exception at the same time. - writer->Write(err_response); - // return ok so TiDB has the chance to see the LockException - return grpc::Status::OK; - } - catch (const Exception & e) - { - LOG_ERROR(log, __PRETTY_FUNCTION__ << ": DB Exception: " << e.message() << "\n" << e.getStackTrace().toString()); - return recordError(e.code() == ErrorCodes::NOT_IMPLEMENTED ? grpc::StatusCode::UNIMPLEMENTED : grpc::StatusCode::INTERNAL, e.message()); - } - catch (const pingcap::Exception & e) - { - LOG_ERROR(log, __PRETTY_FUNCTION__ << ": KV Client Exception: " << e.message()); - return recordError(e.code() == ErrorCodes::NOT_IMPLEMENTED ? grpc::StatusCode::UNIMPLEMENTED : grpc::StatusCode::INTERNAL, e.message()); - } - catch (const std::exception & e) - { - LOG_ERROR(log, __PRETTY_FUNCTION__ << ": std exception: " << e.what()); - return recordError(grpc::StatusCode::INTERNAL, e.what()); - } - catch (...) - { - LOG_ERROR(log, __PRETTY_FUNCTION__ << ": other exception"); - return recordError(grpc::StatusCode::INTERNAL, "other exception"); + for (auto region_id : e.region_ids) + { + auto * status = err_response.add_region_status(); + status->mutable_region_error()->mutable_epoch_not_match(); + status->set_region_id(region_id); + } + break; + default: + // should not happen + break; } + // TODO: This should not be the last response, we should return data and exception at the same time. + writer->Write(err_response); + // return ok so TiDB has the chance to see the LockException + return grpc::Status::OK; +} +catch (const Exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": DB Exception: " << e.message() << "\n" << e.getStackTrace().toString()); + return recordError(e.code() == ErrorCodes::NOT_IMPLEMENTED ? grpc::StatusCode::UNIMPLEMENTED : grpc::StatusCode::INTERNAL, e.message()); +} +catch (const pingcap::Exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": KV Client Exception: " << e.message()); + return recordError(e.code() == ErrorCodes::NOT_IMPLEMENTED ? grpc::StatusCode::UNIMPLEMENTED : grpc::StatusCode::INTERNAL, e.message()); +} +catch (const std::exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": std exception: " << e.what()); + return recordError(grpc::StatusCode::INTERNAL, e.what()); +} +catch (...) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": other exception"); + return recordError(grpc::StatusCode::INTERNAL, "other exception"); +} - grpc::Status BatchCoprocessorHandler::recordError(grpc::StatusCode err_code, const String & err_msg) - { - err_response.set_other_error(err_msg); +grpc::Status BatchCoprocessorHandler::recordError(grpc::StatusCode err_code, const String & err_msg) +{ + err_response.set_other_error(err_msg); - return grpc::Status(err_code, err_msg); - } + return grpc::Status(err_code, err_msg); +} } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 1f8006ea274..1a764f25bf8 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -21,13 +21,11 @@ extern const int LOGICAL_ERROR; extern const int UNKNOWN_EXCEPTION; } // namespace ErrorCodes -DAGDriver::DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, const std::vector & regions_, - UInt64 start_ts, UInt64 schema_ver, std::vector> && key_ranges_, - tipb::SelectResponse & dag_response_, bool internal_) +DAGDriver::DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, const std::vector & regions_, UInt64 start_ts, + UInt64 schema_ver, tipb::SelectResponse & dag_response_, bool internal_) : context(context_), dag_request(dag_request_), regions(regions_), - key_ranges(std::move(key_ranges_)), dag_response(dag_response_), internal(internal_), log(&Logger::get("DAGDriver")) @@ -42,7 +40,7 @@ void DAGDriver::execute() try { DAGContext dag_context(dag_request.executors_size()); - DAGQuerySource dag(context, dag_context, regions, key_ranges, dag_request); + DAGQuerySource dag(context, dag_context, regions, dag_request); BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); if (!streams.in || streams.out) @@ -55,8 +53,9 @@ try if (auto * p_stream = dynamic_cast(streams.in.get())) { - LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": dag request without encode cost: " - << p_stream->getProfileInfo().execution_time/(double)1000000000 << " seconds."); + LOG_DEBUG(log, + __PRETTY_FUNCTION__ << ": dag request without encode cost: " << p_stream->getProfileInfo().execution_time / (double)1000000000 + << " seconds."); } if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) return; @@ -110,11 +109,11 @@ catch (...) recordError(ErrorCodes::UNKNOWN_EXCEPTION, "other exception"); } -void DAGDriver::batchExecute(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer) +void DAGDriver::batchExecute(::grpc::ServerWriter<::coprocessor::BatchResponse> * writer) try { DAGContext dag_context(dag_request.executors_size()); - DAGQuerySource dag(context, dag_context, regions, key_ranges, dag_request, writer); + DAGQuerySource dag(context, dag_context, regions, dag_request, writer); BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); if (!streams.in || streams.out) @@ -126,13 +125,15 @@ try //copyData(*streams.in, *dag_output_stream); streams.in->readPrefix(); - while(streams.in->read()); + while (streams.in->read()) + ; streams.in->readSuffix(); if (auto * p_stream = dynamic_cast(streams.in.get())) { - LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": dag request without encode cost: " - << p_stream->getProfileInfo().execution_time/(double)1000000000 << " seconds."); + LOG_DEBUG(log, + __PRETTY_FUNCTION__ << ": dag request without encode cost: " << p_stream->getProfileInfo().execution_time / (double)1000000000 + << " seconds."); } if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) return; diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.h b/dbms/src/Flash/Coprocessor/DAGDriver.h index aa99aa2be34..d5e3ccb9c32 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.h +++ b/dbms/src/Flash/Coprocessor/DAGDriver.h @@ -3,22 +3,26 @@ #include #include #include -#include +#include #include #include -#include +#include namespace DB { class Context; -class RegionInfo { +class RegionInfo +{ public: RegionID region_id; UInt64 region_version; UInt64 region_conf_version; - RegionInfo(RegionID id, UInt64 ver, UInt64 conf_ver) : region_id(id), region_version(ver), region_conf_version(conf_ver) {} + std::vector> key_ranges; + RegionInfo(RegionID id, UInt64 ver, UInt64 conf_ver, std::vector> && key_ranges_) + : region_id(id), region_version(ver), region_conf_version(conf_ver), key_ranges(std::move(key_ranges_)) + {} }; /// An abstraction of driver running DAG request. @@ -26,14 +30,12 @@ class RegionInfo { class DAGDriver { public: - DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, const std::vector & regions_, - UInt64 start_ts, UInt64 schema_ver, - std::vector> && key_ranges_, tipb::SelectResponse & dag_response_, - bool internal_ = false); + DAGDriver(Context & context_, const tipb::DAGRequest & dag_request_, const std::vector & regions_, UInt64 start_ts, + UInt64 schema_ver, tipb::SelectResponse & dag_response_, bool internal_ = false); void execute(); - void batchExecute(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer); + void batchExecute(::grpc::ServerWriter<::coprocessor::BatchResponse> * writer); private: void recordError(Int32 err_code, const String & err_msg); @@ -44,10 +46,6 @@ class DAGDriver const tipb::DAGRequest & dag_request; const std::vector & regions; - //RegionID region_id; - //UInt64 region_version; - //UInt64 region_conf_version; - std::vector> key_ranges; tipb::SelectResponse & dag_response; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 6230c7bf6a4..5b0fbc74d12 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -30,15 +30,11 @@ extern const int COP_BAD_DAG_REQUEST; //} DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, const std::vector & regions_, - const std::vector> & key_ranges_, - const tipb::DAGRequest & dag_request_, - ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) - : - writer(std::make_shared(writer_)), - context(context_), + const tipb::DAGRequest & dag_request_, ::grpc::ServerWriter<::coprocessor::BatchResponse> * writer_) + : writer(std::make_shared(writer_)), + context(context_), dag_context(dag_context_), regions(regions_), - key_ranges(key_ranges_), dag_request(dag_request_), metrics(context.getTiFlashMetrics()) { diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index abe730dee0c..32c7498b205 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -10,8 +10,9 @@ #include #include #include -#include "DAGQueryBlock.h" + #include "DAGDriver.h" +#include "DAGQueryBlock.h" namespace DB { @@ -20,11 +21,12 @@ class Context; class TiFlashMetrics; using TiFlashMetricsPtr = std::shared_ptr; -struct StreamWriter { - ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer; +struct StreamWriter +{ + ::grpc::ServerWriter<::coprocessor::BatchResponse> * writer; std::mutex write_mutex; - StreamWriter(::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_) : writer(writer_) {} + StreamWriter(::grpc::ServerWriter<::coprocessor::BatchResponse> * writer_) : writer(writer_) {} void write(const ::coprocessor::BatchResponse & data) { @@ -47,8 +49,7 @@ class DAGQuerySource : public IQuerySource //static const String LIMIT_NAME; DAGQuerySource(Context & context_, DAGContext & dag_context_, const std::vector & regions, - const std::vector> & key_ranges_, const tipb::DAGRequest & dag_request_, - ::grpc::ServerWriter< ::coprocessor::BatchResponse>* writer_ = nullptr); + const tipb::DAGRequest & dag_request_, ::grpc::ServerWriter<::coprocessor::BatchResponse> * writer_ = nullptr); std::tuple parse(size_t max_query_size) override; String str(size_t max_query_size) override; @@ -56,8 +57,6 @@ class DAGQuerySource : public IQuerySource //DAGContext & getDAGContext() const { return dag_context; }; - const std::vector> & getKeyRanges() const { return key_ranges; } - //bool hasSelection() const { return sel_index != -1; }; //bool hasAggregation() const { return agg_index != -1; }; //bool hasTopN() const { return order_index != -1; }; @@ -124,7 +123,6 @@ class DAGQuerySource : public IQuerySource DAGContext & dag_context; const std::vector & regions; - const std::vector> & key_ranges; const tipb::DAGRequest & dag_request; diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp index dfdb2c86127..387c511e078 100644 --- a/dbms/src/Flash/CoprocessorHandler.cpp +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -33,9 +33,7 @@ CoprocessorHandler::CoprocessorHandler( grpc::Status CoprocessorHandler::execute() { Stopwatch watch; - SCOPE_EXIT({ - GET_METRIC(cop_context.metrics, tiflash_coprocessor_request_handle_seconds, type_cop).Observe(watch.elapsedSeconds()); - }); + SCOPE_EXIT({ GET_METRIC(cop_context.metrics, tiflash_coprocessor_request_handle_seconds, type_cop).Observe(watch.elapsedSeconds()); }); try { @@ -60,10 +58,13 @@ grpc::Status CoprocessorHandler::execute() throw Exception("DAG request with rpn expression is not supported in TiFlash", ErrorCodes::NOT_IMPLEMENTED); tipb::SelectResponse dag_response; std::vector regions; - regions.emplace_back(cop_context.kv_context.region_id(),cop_context.kv_context.region_epoch().version(), cop_context.kv_context.region_epoch().conf_ver()); + //regions.emplace_back(cop_context.kv_context.region_id(),cop_context.kv_context.region_epoch().version(), + // cop_context.kv_context.region_epoch().conf_ver(), key_ranges); + regions.emplace_back(RegionInfo(cop_context.kv_context.region_id(), cop_context.kv_context.region_epoch().version(), + cop_context.kv_context.region_epoch().conf_ver(), std::move(key_ranges))); DAGDriver driver(cop_context.db_context, dag_request, regions, cop_request->start_ts() > 0 ? cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), - std::move(key_ranges), dag_response); + dag_response); driver.execute(); cop_response->set_data(dag_response.SerializeAsString()); LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); From 347ec5ab2f73e2eadd9c7ce619893c062b1e56aa Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 17 Mar 2020 20:27:50 +0800 Subject: [PATCH 35/54] refine code --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 149 ++++++++++++++---- .../Coprocessor/DAGQueryBlockInterpreter.h | 5 +- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 10 +- dbms/src/Flash/Coprocessor/InterpreterDAG.h | 2 +- dbms/src/Storages/IManageableStorage.cpp | 123 --------------- dbms/src/Storages/IManageableStorage.h | 3 - 6 files changed, 129 insertions(+), 163 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index c234a99b6e6..40f2161fdd8 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -48,13 +48,12 @@ extern const int COP_BAD_DAG_REQUEST; } // namespace ErrorCodes DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos_, - const tipb::DAGRequest & rqst_, ASTPtr dummy_query_, const DAGQuerySource & dag_) + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const tipb::DAGRequest & rqst_, ASTPtr dummy_query_, + const DAGQuerySource & dag_) : context(context_), input_streams_vec(input_streams_vec_), query_block(query_block_), keep_session_timezone_info(keep_session_timezone_info_), - region_infos(region_infos_), rqst(rqst_), dummy_query(std::move(dummy_query_)), dag(dag_), @@ -449,7 +448,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & query_info.mvcc_query_info = std::make_unique(); query_info.mvcc_query_info->resolve_locks = true; query_info.mvcc_query_info->read_tso = settings.read_tso; - for (auto & r : region_infos) + for (auto & r : dag.getRegions()) { RegionQueryInfo info; info.region_id = r.region_id; @@ -459,7 +458,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & if (!current_region) { std::vector region_ids; - for (auto & rr : region_infos) + for (auto & rr : dag.getRegions()) { region_ids.push_back(rr.region_id); } @@ -468,7 +467,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); } - query_info.mvcc_query_info->concurrent = region_infos.size() > 1 ? 1.0 : 0.0; + query_info.mvcc_query_info->concurrent = dag.getRegions().size() > 1 ? 1.0 : 0.0; // RegionQueryInfo info; // info.region_id = region_info.region_id; // info.version = region_info.region_version; @@ -495,7 +494,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & //} //pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); } - LOG_INFO(log, "dag execution stream size: " << region_infos.size()); + LOG_INFO(log, "dag execution stream size: " << dag.getRegions().size()); if (pipeline.streams.empty()) { @@ -629,8 +628,6 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), kind, ASTTableJoin::Strictness::All); executeUnion(right_pipeline); - // todo clickhouse use LazyBlockInputStream to initialize the source, need to double check why and - // if we need to use LazyBlockInputStream here right_query.source = right_pipeline.firstStream(); right_query.join = joinPtr; right_query.join->setSampleBlock(right_query.source->getHeader()); @@ -964,26 +961,82 @@ void DAGQueryBlockInterpreter::executeSubqueryInJoin(Pipeline & pipeline, Subque SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); } -void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) +void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb::Executor * root) { - const auto & ts = query_block.source->tbl_scan(); - TableID table_id = ts.table_id(); - - const Settings & settings = context.getSettingsRef(); - - if (settings.schema_version == DEFAULT_UNSPECIFIED_SCHEMA_VERSION) + const tipb::Executor * current = root; + auto * exec = dag_req.add_executors(); + while (current->tp() != tipb::ExecType::TypeTableScan) { - storage = context.getTMTContext().getStorages().get(table_id); - if (storage == nullptr) + if (current->tp() == tipb::ExecType::TypeSelection) { - throw Exception("Table " + std::to_string(table_id) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + exec->set_tp(tipb::ExecType::TypeSelection); + auto * sel = exec->mutable_selection(); + for (auto const & condition : current->selection().conditions()) + { + auto * tmp = sel->add_conditions(); + tmp->CopyFrom(condition); + } + exec = sel->mutable_child(); + current = ¤t->selection().child(); + } + else if (current->tp() == tipb::ExecType::TypeAggregation || current->tp() == tipb::ExecType::TypeStreamAgg) + { + exec->set_tp(current->tp()); + auto * agg = exec->mutable_aggregation(); + for (auto const & expr : current->aggregation().agg_func()) + { + auto * tmp = agg->add_agg_func(); + tmp->CopyFrom(expr); + } + for (auto const & expr : current->aggregation().group_by()) + { + auto * tmp = agg->add_group_by(); + tmp->CopyFrom(expr); + } + agg->set_streamed(current->aggregation().streamed()); + exec = agg->mutable_child(); + current = ¤t->aggregation().child(); + } + else if (current->tp() == tipb::ExecType::TypeLimit) + { + exec->set_tp(current->tp()); + auto * limit = exec->mutable_limit(); + limit->set_limit(current->limit().limit()); + exec = limit->mutable_child(); + current = ¤t->limit().child(); + } + else if (current->tp() == tipb::ExecType::TypeTopN) + { + exec->set_tp(current->tp()); + auto * topn = exec->mutable_topn(); + topn->set_limit(current->topn().limit()); + for (auto const & expr : current->topn().order_by()) + { + auto * tmp = topn->add_order_by(); + tmp->CopyFrom(expr); + } + exec = topn->mutable_child(); + current = ¤t->topn().child(); + } + else + { + throw Exception("Not supported yet"); } - table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); - } - else - { - getAndLockStorageWithSchemaVersion(table_id, settings.schema_version); } + + if (current->tp() != tipb::ExecType::TypeTableScan) + throw Exception("Only support copy from table scan sourced query block"); + exec->set_tp(tipb::ExecType::TypeTableScan); + auto * new_ts = new tipb::TableScan(current->tbl_scan()); + new_ts->set_next_read_engine(tipb::EngineType::Local); + exec->set_allocated_tbl_scan(new_ts); + + dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); +} + +void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) +{ + const auto & ts = query_block.source->tbl_scan(); std::vector> key_ranges; for (auto & range : ts.ranges()) { @@ -993,7 +1046,45 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) DecodedTiKVKey end(std::move(end_key)); key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); } - pipeline.streams = storage->remote_read(key_ranges, context.getSettingsRef().read_tso, query_block, context); + std::cout << "begin remote read" << std::endl; + std::vector cop_key_ranges; + for (const auto & key_range : key_ranges) + { + cop_key_ranges.push_back( + pingcap::coprocessor::KeyRange{static_cast(key_range.first), static_cast(key_range.second)}); + } + + ::tipb::DAGRequest dag_req; + + copyExecutorTreeWithLocalTableScan(dag_req, query_block.root); + + DAGSchema schema; + ColumnsWithTypeAndName columns; + for (int i = 0; i < (int)query_block.output_field_types.size(); i++) + { + dag_req.add_output_offsets(i); + ColumnInfo info = fieldTypeToColumnInfo(query_block.output_field_types[i]); + String col_name = query_block.qb_column_prefix + "col_" + std::to_string(i); + schema.push_back(std::make_pair(col_name, info)); + auto tp = getDataTypeByColumnInfo(info); + ColumnWithTypeAndName col(tp, col_name); + columns.emplace_back(col); + } + Block sample_block = Block(columns); + + pingcap::coprocessor::Request req; + + dag_req.SerializeToString(&req.data); + req.tp = pingcap::coprocessor::ReqType::DAG; + req.start_ts = context.getSettingsRef().read_tso; + req.ranges = cop_key_ranges; + + pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); + pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; + std::cout << "begin remote read 1111" << std::endl; + BlockInputStreamPtr input = std::make_shared(sample_block, + [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); + pipeline.streams = {input}; } void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) @@ -1051,9 +1142,11 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) //recordProfileStreams(pipeline, dag.getLimitIndex()); } - if (dag.writer != nullptr && query_block.isRootQueryBlock()) { - for (auto & stream : pipeline.streams ) - stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); + if (dag.writer != nullptr && query_block.isRootQueryBlock()) + { + for (auto & stream : pipeline.streams) + stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, + dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); } if (query_block.source->tp() == tipb::ExecType::TypeJoin) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index d45920f32fd..8af1fb56e64 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -78,8 +78,8 @@ class DAGQueryBlockInterpreter { public: DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, - const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const std::vector & region_infos, const tipb::DAGRequest & rqst, - ASTPtr dummp_query, const DAGQuerySource & dag_); + const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const tipb::DAGRequest & rqst, ASTPtr dummp_query, + const DAGQuerySource & dag_); ~DAGQueryBlockInterpreter() = default; @@ -112,7 +112,6 @@ class DAGQueryBlockInterpreter std::vector input_streams_vec; const DAGQueryBlock & query_block; const bool keep_session_timezone_info; - const std::vector & region_infos; const tipb::DAGRequest & rqst; ASTPtr dummy_query; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 0e325f0dca5..2653a66e228 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -900,24 +900,24 @@ void InterpreterDAG::executeUnion(Pipeline & pipeline) } } -BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, const std::vector & region_infos) +BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block) { if (!query_block.children.empty()) { std::vector input_streams_vec; for (auto & child : query_block.children) { - BlockInputStreams child_streams = executeQueryBlock(*child, region_infos); + BlockInputStreams child_streams = executeQueryBlock(*child); input_streams_vec.push_back(child_streams); } DAGQueryBlockInterpreter query_block_interpreter( - context, input_streams_vec, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST(), dag); + context, input_streams_vec, query_block, keep_session_timezone_info, dag.getDAGRequest(), dag.getAST(), dag); return query_block_interpreter.execute(); } else { DAGQueryBlockInterpreter query_block_interpreter( - context, {}, query_block, keep_session_timezone_info, region_infos, dag.getDAGRequest(), dag.getAST(), dag); + context, {}, query_block, keep_session_timezone_info, dag.getDAGRequest(), dag.getAST(), dag); return query_block_interpreter.execute(); } } @@ -927,7 +927,7 @@ BlockIO InterpreterDAG::execute() /// region_info should based on the source executor, however /// tidb does not support multi-table dag request yet, so /// it is ok to use the same region_info for the whole dag request - BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), dag.getRegions()); + BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock()); Pipeline pipeline; pipeline.streams = streams; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index 9e38b81b2e3..8854c886412 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -37,7 +37,7 @@ class InterpreterDAG : public IInterpreter BlockIO execute(); private: - BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block, const std::vector & region_infos); + BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block); void executeUnion(Pipeline & pipeline); /* void executeImpl(Pipeline & pipeline); diff --git a/dbms/src/Storages/IManageableStorage.cpp b/dbms/src/Storages/IManageableStorage.cpp index e8ad21f1ee2..e69de29bb2d 100644 --- a/dbms/src/Storages/IManageableStorage.cpp +++ b/dbms/src/Storages/IManageableStorage.cpp @@ -1,123 +0,0 @@ -#include -#include - -namespace DB -{ - -void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb::Executor * root) -{ - const tipb::Executor * current = root; - auto * exec = dag_req.add_executors(); - while (current->tp() != tipb::ExecType::TypeTableScan) - { - if (current->tp() == tipb::ExecType::TypeSelection) - { - exec->set_tp(tipb::ExecType::TypeSelection); - auto * sel = exec->mutable_selection(); - for (auto const & condition : current->selection().conditions()) - { - auto * tmp = sel->add_conditions(); - tmp->CopyFrom(condition); - } - exec = sel->mutable_child(); - current = ¤t->selection().child(); - } - else if (current->tp() == tipb::ExecType::TypeAggregation || current->tp() == tipb::ExecType::TypeStreamAgg) - { - exec->set_tp(current->tp()); - auto * agg = exec->mutable_aggregation(); - for (auto const & expr : current->aggregation().agg_func()) - { - auto * tmp = agg->add_agg_func(); - tmp->CopyFrom(expr); - } - for (auto const & expr : current->aggregation().group_by()) - { - auto * tmp = agg->add_group_by(); - tmp->CopyFrom(expr); - } - agg->set_streamed(current->aggregation().streamed()); - exec = agg->mutable_child(); - current = ¤t->aggregation().child(); - } - else if (current->tp() == tipb::ExecType::TypeLimit) - { - exec->set_tp(current->tp()); - auto * limit = exec->mutable_limit(); - limit->set_limit(current->limit().limit()); - exec = limit->mutable_child(); - current = ¤t->limit().child(); - } - else if (current->tp() == tipb::ExecType::TypeTopN) - { - exec->set_tp(current->tp()); - auto * topn = exec->mutable_topn(); - topn->set_limit(current->topn().limit()); - for (auto const & expr : current->topn().order_by()) - { - auto * tmp = topn->add_order_by(); - tmp->CopyFrom(expr); - } - exec = topn->mutable_child(); - current = ¤t->topn().child(); - } - else - { - throw Exception("Not supported yet"); - } - } - - if (current->tp() != tipb::ExecType::TypeTableScan) - throw Exception("Only support copy from table scan sourced query block"); - exec->set_tp(tipb::ExecType::TypeTableScan); - auto * new_ts = new tipb::TableScan(current->tbl_scan()); - new_ts->set_next_read_engine(tipb::EngineType::Local); - exec->set_allocated_tbl_scan(new_ts); - - dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); -} - -BlockInputStreams IManageableStorage::remote_read(const std::vector> & key_ranges, - UInt64 read_tso, const DAGQueryBlock & remote_query_block, Context & context) -{ - std::cout<< "begin remote read" < cop_key_ranges; - for (const auto & key_range : key_ranges) - { - cop_key_ranges.push_back( - pingcap::coprocessor::KeyRange{static_cast(key_range.first), static_cast(key_range.second)}); - } - - ::tipb::DAGRequest dag_req; - - copyExecutorTreeWithLocalTableScan(dag_req, remote_query_block.root); - - DAGSchema schema; - ColumnsWithTypeAndName columns; - for (int i = 0; i < (int)remote_query_block.output_field_types.size(); i++) - { - dag_req.add_output_offsets(i); - ColumnInfo info = fieldTypeToColumnInfo(remote_query_block.output_field_types[i]); - String col_name = remote_query_block.qb_column_prefix + "col_" + std::to_string(i); - schema.push_back(std::make_pair(col_name, info)); - auto tp = getDataTypeByColumnInfo(info); - ColumnWithTypeAndName col(tp, col_name); - columns.emplace_back(col); - } - Block sample_block = Block(columns); - - pingcap::coprocessor::Request req; - - dag_req.SerializeToString(&req.data); - req.tp = pingcap::coprocessor::ReqType::DAG; - req.start_ts = read_tso; - req.ranges = cop_key_ranges; - - pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); - pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; - std::cout<< "begin remote read 1111" <(sample_block, - [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); - return {input}; -}; -} // namespace DB diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index 84d22250f3c..1285cf636ed 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -61,9 +61,6 @@ class IManageableStorage : public IStorage virtual const TiDB::TableInfo & getTableInfo() const = 0; - BlockInputStreams remote_read(const std::vector> & key_ranges, UInt64 read_tso, - const DAGQueryBlock & remote_query_block, Context & context); - // Apply AlterCommands synced from TiDB should use `alterFromTiDB` instead of `alter(...)` virtual void alterFromTiDB( const AlterCommands & commands, const TiDB::TableInfo & table_info, const String & database_name, const Context & context) From e9830a99747bd1aa1ca6044a71b6f51092f2f4a1 Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 17 Mar 2020 20:33:05 +0800 Subject: [PATCH 36/54] delete useless code --- .../Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 40f2161fdd8..1c60801bb2f 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -448,7 +448,8 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & query_info.mvcc_query_info = std::make_unique(); query_info.mvcc_query_info->resolve_locks = true; query_info.mvcc_query_info->read_tso = settings.read_tso; - for (auto & r : dag.getRegions()) + const auto & regions = dag.getRegions(); + for (auto & r : regions) { RegionQueryInfo info; info.region_id = r.region_id; @@ -458,7 +459,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & if (!current_region) { std::vector region_ids; - for (auto & rr : dag.getRegions()) + for (auto & rr : regions) { region_ids.push_back(rr.region_id); } @@ -467,7 +468,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); } - query_info.mvcc_query_info->concurrent = dag.getRegions().size() > 1 ? 1.0 : 0.0; + query_info.mvcc_query_info->concurrent = regions.size() > 1 ? 1.0 : 0.0; // RegionQueryInfo info; // info.region_id = region_info.region_id; // info.version = region_info.region_version; @@ -494,7 +495,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & //} //pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); } - LOG_INFO(log, "dag execution stream size: " << dag.getRegions().size()); + LOG_INFO(log, "dag execution stream size: " << regions.size()); if (pipeline.streams.empty()) { @@ -1046,7 +1047,6 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) DecodedTiKVKey end(std::move(end_key)); key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); } - std::cout << "begin remote read" << std::endl; std::vector cop_key_ranges; for (const auto & key_range : key_ranges) { @@ -1081,7 +1081,6 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; - std::cout << "begin remote read 1111" << std::endl; BlockInputStreamPtr input = std::make_shared(sample_block, [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); pipeline.streams = {input}; From 2c85b1d0d30d0a528d3327d37f870c4b420c8bd4 Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 17 Mar 2020 21:07:04 +0800 Subject: [PATCH 37/54] update kvproto and client-c --- contrib/client-c | 2 +- contrib/kvproto | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/client-c b/contrib/client-c index a31669150e5..a4b23dcdd28 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit a31669150e5dd12b269e8f389487d9305f053ddb +Subproject commit a4b23dcdd28f41b27a28a99eac1756370945c219 diff --git a/contrib/kvproto b/contrib/kvproto index e9d8c4e28e7..9bb769c3cee 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit e9d8c4e28e75c9e655e2038a40267bbd784302cc +Subproject commit 9bb769c3ceebbe3e66b6d4108d7e7f8106eb700f From f499cd7441042b3fb36c711b3bfa56e327e39fb5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 19 Mar 2020 17:17:13 +0800 Subject: [PATCH 38/54] [flash 1002]refine coprocessor read (#530) * refind coprocessor * fix * try fix ci --- contrib/client-c | 2 +- .../DataStreams/CoprocessorBlockInputStream.h | 50 +++++++++++-------- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 3 +- release-centos7/build/build-tiflash-ci.sh | 2 +- 4 files changed, 31 insertions(+), 26 deletions(-) diff --git a/contrib/client-c b/contrib/client-c index a4b23dcdd28..087ddad8378 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit a4b23dcdd28f41b27a28a99eac1756370945c219 +Subproject commit 087ddad837866d05fe615737479ba075abe41e6e diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h index 26b87cb0f74..6217ac66e62 100644 --- a/dbms/src/DataStreams/CoprocessorBlockInputStream.h +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -1,34 +1,18 @@ #pragma once #include +#include #include +#include #include #include -#include -#include namespace DB { class CoprocessorBlockInputStream : public IProfilingBlockInputStream { -public: - CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_, - pingcap::kv::StoreType store_type) - : req(req_), - resp_iter(pingcap::coprocessor::Client::send(cluster_, &req, store_type)), - schema(schema_), - log(&Logger::get("pingcap/coprocessor")) - { - pingcap::Exception error = resp_iter.prepare(); - if (!error.empty()) - { - LOG_WARNING(log, "coprocessor client meets error: " << error.displayText()); - throw error; - } - } - - Block getHeader() const override + Block getSampleBlock() const { ColumnsWithTypeAndName columns; for (auto name_and_column : schema) @@ -37,10 +21,23 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream ColumnWithTypeAndName col(tp, name_and_column.first); columns.emplace_back(col); } - LOG_DEBUG(log, "header columns: " + std::to_string(columns.size())); return Block(columns); } +public: + CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_, + int concurrency, pingcap::kv::StoreType store_type) + : req(req_), + resp_iter(pingcap::coprocessor::Client::send(cluster_, &req, concurrency, store_type)), + schema(schema_), + sample_block(getSampleBlock()), + log(&Logger::get("pingcap/coprocessor")) + { + resp_iter.open(); + } + + Block getHeader() const override { return sample_block; } + String getName() const override { return "Coprocessor"; } Block readImpl() override @@ -53,7 +50,8 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream } auto chunk = chunk_queue.front(); chunk_queue.pop(); - switch (resp->encode_type()) { + switch (resp->encode_type()) + { case tipb::EncodeType::TypeCHBlock: return CHBlockChunkCodec().decode(chunk, schema); case tipb::EncodeType::TypeChunk: @@ -70,13 +68,19 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream { LOG_DEBUG(log, "fetch new data"); - auto [data, has_next] = resp_iter.next(); + auto [result, has_next] = resp_iter.next(); + if (!result.error.empty()) + { + LOG_WARNING(log, "coprocessor client meets error: " << result.error.displayText()); + throw result.error; + } if (!has_next) { return false; } + const std::string & data = result.data(); resp = std::make_shared(); resp->ParseFromString(data); @@ -100,6 +104,8 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream std::queue chunk_queue; + Block sample_block; + Logger * log; }; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 1c60801bb2f..ac2e4fd29f9 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -1081,8 +1081,7 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; - BlockInputStreamPtr input = std::make_shared(sample_block, - [cluster, req, schema, store_type]() { return std::make_shared(cluster, req, schema, store_type); }); + BlockInputStreamPtr input = std::make_shared(cluster, req, schema, context.getSettingsRef().max_threads, store_type); pipeline.streams = {input}; } diff --git a/release-centos7/build/build-tiflash-ci.sh b/release-centos7/build/build-tiflash-ci.sh index f9cec626535..e557fd2c472 100755 --- a/release-centos7/build/build-tiflash-ci.sh +++ b/release-centos7/build/build-tiflash-ci.sh @@ -24,7 +24,7 @@ cp -r /flash_cluster_manager "$install_dir"/flash_cluster_manager if [ -d "$SRCPATH/contrib/kvproto" ]; then cd "$SRCPATH/contrib/kvproto" rm -rf cpp/kvproto - ./generate_cpp.sh + ./scripts/generate_cpp.sh cd - fi From 68b38152e05fc34848dff4ecf3160ef8c6c116e3 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 23 Mar 2020 16:12:12 +0800 Subject: [PATCH 39/54] support key ranges in batch coprocessor (#533) * save work * save work * save work * save work * dt support key ranges in dag request * fix bug * fix bug * fix bug * add some comments * fix bug * address comments --- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 1 - .../Coprocessor/DAGQueryBlockInterpreter.cpp | 96 +++----- .../Coprocessor/DAGQueryBlockInterpreter.h | 3 - dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 14 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 217 ++++++++++++++---- .../MergeTreeDataSelectExecutorCommon.hpp | 46 +++- dbms/src/Storages/RegionQueryInfo.h | 2 + dbms/src/Storages/StorageDeltaMerge.cpp | 18 +- dbms/src/Storages/StorageDeltaMergeHelpers.h | 47 ++-- .../Storages/Transaction/PartitionStreams.cpp | 6 +- .../Transaction/RegionBlockReader.cpp | 20 +- .../Storages/Transaction/RegionBlockReader.h | 51 +++- dbms/src/Storages/Transaction/RegionTable.h | 4 +- 16 files changed, 379 insertions(+), 154 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 1a764f25bf8..93fbd8fdcf7 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -132,7 +132,7 @@ try if (auto * p_stream = dynamic_cast(streams.in.get())) { LOG_DEBUG(log, - __PRETTY_FUNCTION__ << ": dag request without encode cost: " << p_stream->getProfileInfo().execution_time / (double)1000000000 + __PRETTY_FUNCTION__ << ": dag request with encode cost: " << p_stream->getProfileInfo().execution_time / (double)1000000000 << " seconds."); } if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index b2f17ce4f88..3bcd0928e4b 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -61,7 +61,7 @@ void collectOutPutFieldTypesFromAgg(std::vector & field_type, c } DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root_) - : id(id_), root(root_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") + : id(id_), root(root_), qb_column_prefix("__QB_" + std::to_string(id_) + "_") { const tipb::Executor * current = root; while (!isSourceNode(current)) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index 2e1afbd61d9..c19087c1730 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -33,7 +33,6 @@ class DAGQueryBlock UInt32 id; const tipb::Executor * root; String qb_column_prefix; - // todo use unique_ptr instead std::vector> children; std::vector output_field_types; // kinds of project diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index ac2e4fd29f9..88166553f0d 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -350,62 +350,25 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & auto pair = storage->getColumns().getPhysical(handle_column_name); source_columns.push_back(pair); is_ts_column.push_back(false); - handle_col_id = i; continue; } String name = storage->getTableInfo().getColumnName(cid); required_columns.push_back(name); - if (name == handle_column_name) - handle_col_id = i; auto pair = storage->getColumns().getPhysical(name); source_columns.emplace_back(std::move(pair)); is_ts_column.push_back(ci.tp() == TiDB::TypeTimestamp); } - if (handle_col_id == -1) - handle_col_id = required_columns.size(); - - // auto current_region = context.getTMTContext().getKVStore()->getRegion(region_info.region_id); - // auto region_read_status = getRegionReadStatus(current_region); - // if (region_read_status != RegionException::OK) - // { - // std::vector region_ids; - // region_ids.push_back(region_info.region_id); - // LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << region_info.region_id); - // throw RegionException(std::move(region_ids), region_read_status); - // } - // const bool pk_is_uint64 = storage->getPKType() == IManageableStorage::PKType::UINT64; - // if (!checkKeyRanges(region_info.key_ranges, table_id, pk_is_uint64, current_region->getRange(), handle_col_id, handle_filter_expr)) - // { - // // need to add extra filter on handle column - // filter_on_handle = true; - // conditions.push_back(&handle_filter_expr); - // } - - bool has_handle_column = (handle_col_id != (Int32)required_columns.size()); - - if (filter_on_handle && !has_handle_column) - { - // if need to add filter on handle column, and - // the handle column is not selected in ts, add - // the handle column - required_columns.push_back(handle_column_name); - auto pair = storage->getColumns().getPhysical(handle_column_name); - source_columns.push_back(pair); - is_ts_column.push_back(false); - } - analyzer = std::make_unique(std::move(source_columns), context); if (query_block.aggregation == nullptr) { - int extra_col_size = (filter_on_handle && !has_handle_column) ? 1 : 0; if (query_block.isRootQueryBlock()) { for (auto i : query_block.output_offsets) { - if ((size_t)i >= required_columns.size() - extra_col_size) + if ((size_t)i >= required_columns.size()) { // array index out of bound throw Exception("Output offset index is out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); @@ -416,7 +379,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & } else { - for (size_t i = 0; i < required_columns.size() - extra_col_size; i++) + for (size_t i = 0; i < required_columns.size(); i++) /// for child query block, add alias start with qb_column_prefix to avoid column name conflict final_project.emplace_back(required_columns[i], query_block.qb_column_prefix + required_columns[i]); } @@ -451,31 +414,39 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & const auto & regions = dag.getRegions(); for (auto & r : regions) { + if (r.key_ranges.empty()) + { + throw Exception("Income key ranges is empty for region: " + std::to_string(r.region_id), ErrorCodes::COP_BAD_DAG_REQUEST); + } RegionQueryInfo info; info.region_id = r.region_id; info.version = r.region_version; info.conf_version = r.region_conf_version; + for (const auto & p : r.key_ranges) + { + TiKVRange::Handle start = TiKVRange::getRangeHandle(p.first, table_id); + TiKVRange::Handle end = TiKVRange::getRangeHandle(p.second, table_id); + info.required_handle_ranges.emplace_back(std::make_pair(start, end)); + } auto current_region = context.getTMTContext().getKVStore()->getRegion(info.region_id); - if (!current_region) + auto region_read_status = getRegionReadStatus(current_region); + if (region_read_status != RegionException::OK) { std::vector region_ids; + region_ids.reserve(regions.size()); for (auto & rr : regions) { region_ids.push_back(rr.region_id); } + LOG_WARNING(log, __PRETTY_FUNCTION__ << " Meet region exception for region " << info.region_id); throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); } info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); } - query_info.mvcc_query_info->concurrent = regions.size() > 1 ? 1.0 : 0.0; - // RegionQueryInfo info; - // info.region_id = region_info.region_id; - // info.version = region_info.region_version; - // info.conf_version = region_info.region_conf_version; - // info.range_in_table = current_region->getHandleRangeByTable(table_id); - // query_info.mvcc_query_info->regions_query_info.push_back(info); - // query_info.mvcc_query_info->concurrent = 0.0; + if (query_info.mvcc_query_info->regions_query_info.empty()) + throw Exception("Dag Request does not have region to read. ", ErrorCodes::COP_BAD_DAG_REQUEST); + query_info.mvcc_query_info->concurrent = query_info.mvcc_query_info->regions_query_info.size() > 1 ? 1.0 : 0.0; if (ts.next_read_engine() == tipb::EngineType::Local) { @@ -484,16 +455,6 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & else { throw Exception("Should not reach here", ErrorCodes::LOGICAL_ERROR); - //std::vector> key_ranges; - //for (auto & range : ts.ranges()) - //{ - // std::string start_key(range.low()); - // DecodedTiKVKey start(std::move(start_key)); - // std::string end_key(range.high()); - // DecodedTiKVKey end(std::move(end_key)); - // key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); - //} - //pipeline.streams = storage->remote_read(key_ranges, query_info, ts, context); } LOG_INFO(log, "dag execution stream size: " << regions.size()); @@ -563,11 +524,6 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p if (join_type_it == join_type_map.end()) throw Exception("Unknown join type in dag request", ErrorCodes::COP_BAD_DAG_REQUEST); ASTTableJoin::Kind kind = join_type_it->second; - if (kind != ASTTableJoin::Kind::Inner) - { - // todo support left and right join - throw Exception("Only Inner join is supported", ErrorCodes::NOT_IMPLEMENTED); - } BlockInputStreams left_streams; BlockInputStreams right_streams; @@ -589,6 +545,13 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p left_streams = input_streams_vec[0]; right_streams = input_streams_vec[1]; } + + if (kind != ASTTableJoin::Kind::Inner) + { + // todo support left and right join + throw Exception("Only Inner join is supported", ErrorCodes::NOT_IMPLEMENTED); + } + std::vector join_output_columns; for (auto const & p : input_streams_vec[0][0]->getHeader().getNamesAndTypesList()) { @@ -640,6 +603,7 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p ExpressionActionsChain chain; dag_analyzer.appendJoin(chain, right_query, columns_added_by_join); pipeline.streams = left_streams; + /// add join input stream for (auto & stream : pipeline.streams) stream = std::make_shared(stream, chain.getLastActions()); @@ -1048,10 +1012,10 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) key_ranges.emplace_back(std::make_pair(std::move(start), std::move(end))); } std::vector cop_key_ranges; + cop_key_ranges.reserve(key_ranges.size()); for (const auto & key_range : key_ranges) { - cop_key_ranges.push_back( - pingcap::coprocessor::KeyRange{static_cast(key_range.first), static_cast(key_range.second)}); + cop_key_ranges.emplace_back(static_cast(key_range.first), static_cast(key_range.second)); } ::tipb::DAGRequest dag_req; @@ -1140,7 +1104,7 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) //recordProfileStreams(pipeline, dag.getLimitIndex()); } - if (dag.writer != nullptr && query_block.isRootQueryBlock()) + if (dag.writer->writer != nullptr && query_block.isRootQueryBlock()) { for (auto & stream : pipeline.streams) stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 8af1fb56e64..0c962dcb6ee 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -127,9 +127,6 @@ class DAGQueryBlockInterpreter std::unique_ptr analyzer; - bool filter_on_handle = false; - tipb::Expr handle_filter_expr; - Int32 handle_col_id = -1; std::vector conditions; const DAGQuerySource & dag; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 5b0fbc74d12..d5283467f84 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -73,28 +73,28 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, co */ //if (dag_request.has_executor_tree()) //{ - // query_block_tree = std::make_shared(1, &dag_request.executor_tree()); + // root_query_block = std::make_shared(1, &dag_request.executor_tree()); //} //else //{ // std::vector executors; // for (const tipb::Executor & executor : dag_request.executors()) // executors.push_back(&executor); - // query_block_tree = std::make_shared(1, executors, 0, (int)executors.size() - 1); + // root_query_block = std::make_shared(1, executors, 0, (int)executors.size() - 1); //} - query_block_tree = std::make_shared(1, &dag_request.executors(0)); + root_query_block = std::make_shared(1, &dag_request.executors(0)); for (Int32 i : dag_request.output_offsets()) - query_block_tree->output_offsets.push_back(i); - if (query_block_tree->aggregation != nullptr) + root_query_block->output_offsets.push_back(i); + if (root_query_block->aggregation != nullptr) { - for (auto & field_type : query_block_tree->output_field_types) + for (auto & field_type : root_query_block->output_field_types) result_field_types.push_back(field_type); } else { for (UInt32 i : dag_request.output_offsets()) { - result_field_types.push_back(query_block_tree->output_field_types[i]); + result_field_types.push_back(root_query_block->output_field_types[i]); } } //analyzeResultFieldTypes(); diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index 32c7498b205..afd7fe3b181 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -101,7 +101,7 @@ class DAGQuerySource : public IQuerySource tipb::EncodeType getEncodeType() const { return encode_type; } - std::shared_ptr getQueryBlock() const { return query_block_tree; } + std::shared_ptr getQueryBlock() const { return root_query_block; } const std::vector & getRegions() const { return regions; } StreamWriterPtr writer; @@ -136,7 +136,7 @@ class DAGQuerySource : public IQuerySource std::vector result_field_types; tipb::EncodeType encode_type; - std::shared_ptr query_block_tree; + std::shared_ptr root_query_block; ASTPtr ast; }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a212cfd5c66..97f74d530d8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -61,6 +61,7 @@ struct numeric_limits<__uint128_t> #include #include #include +#include #include #include #include @@ -167,6 +168,7 @@ struct RegionExecutorData { RegionQueryInfo info; Block block; + RegionScanFilterPtr range_scan_filter; RegionExecutorData() = default; RegionExecutorData(const RegionQueryInfo & info_) : info(info_) {} @@ -176,13 +178,115 @@ struct RegionExecutorData return *this; info = std::move(data.info); block = std::move(data.block); + range_scan_filter = std::move(data.range_scan_filter); return *this; } RegionExecutorData(const RegionExecutorData &) = delete; - RegionExecutorData(RegionExecutorData && data) : info(std::move(data.info)), block(std::move(data.block)) {} + RegionExecutorData(RegionExecutorData && data) : + info(std::move(data.info)), block(std::move(data.block)), range_scan_filter(std::move(data.range_scan_filter)) {} bool operator<(const RegionExecutorData & o) const { return info < o.info; } }; +template +bool isFullRegionScan(std::vector> & dag_required_handle_ranges, std::vector> & region_ranges) +{ + std::sort(dag_required_handle_ranges.begin(), dag_required_handle_ranges.end(), + [](const HandleRange & a, const HandleRange & b) { return a.first < b.first; }); + bool need_merge = false; + for(size_t i = 0; i < dag_required_handle_ranges.size()-1; i++) + need_merge |= dag_required_handle_ranges[i].second >= dag_required_handle_ranges[i+1].first; + if (need_merge) + { + // merge dag_required_handle_ranges + size_t new_size = 0; + HandleRange current_range; + current_range.first = dag_required_handle_ranges[0].first; + current_range.second = dag_required_handle_ranges[0].second; + for (size_t i = 1; i < dag_required_handle_ranges.size(); i++) + { + if (current_range.second >= dag_required_handle_ranges[i].first) + current_range.second = current_range.second >= dag_required_handle_ranges[i].second ? current_range.second : dag_required_handle_ranges[i].second; + else + { + if (current_range.second > current_range.first) + dag_required_handle_ranges[new_size++] = current_range; + current_range.first = dag_required_handle_ranges[i].first; + current_range.second = dag_required_handle_ranges[i].second; + } + } + if (current_range.second > current_range.first) + dag_required_handle_ranges[new_size++] = current_range; + dag_required_handle_ranges.resize(new_size); + } + + bool ret = true; + for (const auto & region_range : region_ranges) + { + bool covered = false; + for (const auto & range : dag_required_handle_ranges) + { + if (region_range.first >= range.first && region_range.second <= range.second) + { + covered = true; + break; + } + } + if (!covered && region_range.second > region_range.first) + { + ret = false; + break; + } + } + return ret; +} + +void checkAndSetRangeScanFilter(TMTPKType pk_type, std::vector & regions) { + for(auto & r : regions) + { + if (r.info.required_handle_ranges.empty()) + { + r.range_scan_filter = std::make_shared(true, + (std::vector>){}, (std::vector>){}); + continue; + } + if (pk_type == TMTPKType::UINT64) + { + std::vector> dag_handle_ranges; + for(const auto & dag_range : r.info.required_handle_ranges) + { + const auto [n, new_range] + = CHTableHandle::splitForUInt64TableHandle(dag_range); + for (int i = 0; i < n; i++) + dag_handle_ranges.push_back(new_range[i]); + } + std::vector> region_ranges; + // range_in_table should be initialized before calling this + const auto [n, new_range] = CHTableHandle::splitForUInt64TableHandle(r.info.range_in_table); + region_ranges.reserve(n); + for(int i = 0; i < n; i++) + region_ranges.push_back(new_range[i]); + if (isFullRegionScan(dag_handle_ranges, region_ranges)) + r.range_scan_filter = std::make_shared(true, + (std::vector>){}, (std::vector>){}); + else + r.range_scan_filter = std::make_shared(false, + (std::vector>){}, dag_handle_ranges); + } + else + { + std::vector> region_ranges; + // range_in_table should be initialized before calling this + region_ranges.push_back(r.info.range_in_table); + if (isFullRegionScan(r.info.required_handle_ranges, region_ranges)) + r.range_scan_filter = std::make_shared(true, + (std::vector>){}, (std::vector>){}); + else + r.range_scan_filter = std::make_shared(false, + r.info.required_handle_ranges, (std::vector>){}); + } + } +} + BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_to_return, const SelectQueryInfo & query_info, const Context & context, @@ -213,7 +317,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t const std::string handle_col_name = is_txn_engine ? data.getPrimarySortDescription()[0].column_name : ""; size_t region_cnt = 0; std::vector> region_group_range_parts; - std::vector>> region_group_mem_block; + std::vector> region_group_read_groups; std::vector>> region_group_handle_ranges; std::vector>> region_group_u64_handle_ranges; @@ -303,7 +407,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t regions_executor_data.reserve(regions.size()); for (const auto & [id, region] : regions) { - regions_executor_data.emplace_back(RegionQueryInfo{id, region->version(), region->confVer(), {0, 0}}); + regions_executor_data.emplace_back( + RegionQueryInfo{id, region->version(), region->confVer(), + region->getHandleRangeByTable(data.table_info->id), {}}); } } @@ -323,6 +429,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t if (kvstore_region.size() != regions_executor_data.size()) throw Exception("Duplicate region id", ErrorCodes::LOGICAL_ERROR); + checkAndSetRangeScanFilter(pk_type, regions_executor_data); + { // learner read concurrent_num = std::max(1, std::min(concurrent_num, regions_executor_data.size())); @@ -376,8 +484,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t } auto [block, status] = RegionTable::readBlockByRegion(*data.table_info, data.getColumns(), tmt_column_names_to_read, - kvstore_region[region_query_info.region_id], region_query_info.version, region_query_info.conf_version, - mvcc_query_info.resolve_locks, mvcc_query_info.read_tso, region_query_info.range_in_table); + kvstore_region[region_query_info.region_id], region_query_info.version, region_query_info.conf_version, mvcc_query_info.resolve_locks, + mvcc_query_info.read_tso, region_query_info.range_in_table, regions_executor_data[region_index].range_scan_filter); if (status != RegionException::RegionReadStatus::OK) { @@ -477,7 +585,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t } region_group_range_parts.assign(concurrent_num, {}); - region_group_mem_block.assign(concurrent_num, {}); + region_group_read_groups.assign(concurrent_num, {}); if (pk_type == TMTPKType::UINT64) region_group_u64_handle_ranges.assign(concurrent_num, {}); else @@ -917,21 +1025,32 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t for (size_t region_index = region_begin, region_end = region_begin + thread_region_size; region_index < region_end; ++region_index) { - const auto [n, new_range] - = CHTableHandle::splitForUInt64TableHandle(regions_executor_data[region_index].info.range_in_table); + const auto & region_data = regions_executor_data[region_index]; + if (region_data.range_scan_filter->isFullRangeScan()) + { + const auto[n, new_range] + = CHTableHandle::splitForUInt64TableHandle( + region_data.info.range_in_table); - if (n != 1) - throw Exception("split for uint64 handle should be only 1 ranges", ErrorCodes::LOGICAL_ERROR); + if (n != 1) + throw Exception("split for uint64 handle should be only 1 range", + ErrorCodes::LOGICAL_ERROR); - handle_ranges.emplace_back(new_range[0], region_index); - mem_rows += regions_executor_data[region_index].block.rows(); + handle_ranges.emplace_back(new_range[0], region_index); + } + else + { + for (const HandleRange & handle_range : region_data.range_scan_filter->getUInt64Ranges()) + handle_ranges.emplace_back(handle_range, region_index); + } + mem_rows += region_data.block.rows(); } // the order of uint64 is different with int64. std::sort(handle_ranges.begin(), handle_ranges.end(), [](const UInt64RangeElement & a, const UInt64RangeElement & b) { return a.first < b.first; }); - computeHandleRanges(region_group_mem_block[thread_idx], + computeHandleRanges(region_group_read_groups[thread_idx], handle_ranges, region_group_range_parts[thread_idx], region_group_u64_handle_ranges[thread_idx], @@ -949,13 +1068,20 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t for (size_t region_index = region_begin, region_end = region_begin + thread_region_size; region_index < region_end; ++region_index) { - handle_ranges.emplace_back(regions_executor_data[region_index].info.range_in_table, region_index); - mem_rows += regions_executor_data[region_index].block.rows(); + const auto & region_data = regions_executor_data[region_index]; + if (region_data.range_scan_filter->isFullRangeScan()) + handle_ranges.emplace_back(region_data.info.range_in_table, region_index); + else + { + for (const HandleRange &handle_range : region_data.range_scan_filter->getInt64Ranges()) + handle_ranges.emplace_back(handle_range, region_index); + } + mem_rows += region_data.block.rows(); } // handle_ranges is sorted. - computeHandleRanges(region_group_mem_block[thread_idx], + computeHandleRanges(region_group_read_groups[thread_idx], handle_ranges, region_group_range_parts[thread_idx], region_group_handle_ranges[thread_idx], @@ -1060,30 +1186,32 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t BlockInputStreams union_regions_stream; auto & range_parts = region_group_range_parts[thread_idx]; - auto & mem_blocks = region_group_mem_block[thread_idx]; - for (size_t range_idx = 0; range_idx < range_parts.size(); ++range_idx) + auto & read_groups = region_group_read_groups[thread_idx]; + for (size_t read_group_idx = 0; read_group_idx < read_groups.size(); ++read_group_idx) { - auto & data_parts = range_parts[range_idx]; - auto & block_indexes = mem_blocks[range_idx]; - + auto & read_group = read_groups[read_group_idx]; BlockInputStreams merging; - for (size_t part_idx = 0; part_idx < data_parts.size(); ++part_idx) + for (size_t range_idx = read_group.start_range_index; range_idx <= read_group.end_range_index; ++range_idx) { - const auto & part = data_parts[part_idx]; - BlockInputStreamPtr source_stream = func_make_merge_tree_input(part, part.ranges); + auto & data_parts = range_parts[range_idx]; + for (size_t part_idx = 0; part_idx < data_parts.size(); ++part_idx) + { + const auto & part = data_parts[part_idx]; + BlockInputStreamPtr source_stream = func_make_merge_tree_input(part, part.ranges); - if (pk_type == TMTPKType::UINT64) - source_stream - = func_make_uint64_range_filter_input(source_stream, region_group_u64_handle_ranges[thread_idx][range_idx]); - else - source_stream = func_make_range_filter_input(source_stream, region_group_handle_ranges[thread_idx][range_idx]); + if (pk_type == TMTPKType::UINT64) + source_stream + = func_make_uint64_range_filter_input(source_stream, region_group_u64_handle_ranges[thread_idx][range_idx]); + else + source_stream = func_make_range_filter_input(source_stream, region_group_handle_ranges[thread_idx][range_idx]); - source_stream = func_make_version_filter_input(source_stream); + source_stream = func_make_version_filter_input(source_stream); - merging.emplace_back(source_stream); + merging.emplace_back(source_stream); + } } - + auto & block_indexes = read_group.mem_block_indexes; { BlocksList blocks; for (size_t idx : block_indexes) @@ -1110,12 +1238,23 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t size_t region_sum_ranges = 0, region_sum_marks = 0; BlockInputStreams merging; + std::vector> ranges; const auto & special_region_info = regions_executor_data[special_region_index]; - const auto [n, new_range] = CHTableHandle::splitForUInt64TableHandle(special_region_info.info.range_in_table); + if (special_region_info.range_scan_filter->isFullRangeScan()) + { + const auto [n, new_range] = CHTableHandle::splitForUInt64TableHandle(special_region_info.info.range_in_table); + for (int i = 0; i < n; ++i) + ranges.push_back(new_range[i]); + } + else + { + for (const HandleRange & range : special_region_info.range_scan_filter->getUInt64Ranges()) + ranges.push_back(range); + } - for (int i = 0; i < n; ++i) + for (size_t i = 0; i < ranges.size(); ++i) { - const auto & handle_range = new_range[i]; + const auto & handle_range = ranges[i]; for (const RangesInDataPart & part : parts_with_ranges) { @@ -1143,17 +1282,17 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t { std::stringstream ss; - for (auto i = 0; i < n; ++i) + for (size_t i = 0; i < ranges.size(); ++i) { ss << "["; - new_range[i].first.toString(ss); + ranges[i].first.toString(ss); ss << ","; - new_range[i].second.toString(ss); + ranges[i].second.toString(ss); ss << ") "; } LOG_DEBUG(log, "[PK_IS_UINT64] special region " - << special_region_info.info.region_id << ", split range into " << n << ": " << ss.str() << ", " + << special_region_info.info.region_id << ", split range into " << ranges.size() << ": " << ss.str() << ", " << region_sum_marks << " marks to read from " << region_sum_ranges << " ranges, read " << regions_executor_data[special_region_index].block.rows() << " rows from memory"); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutorCommon.hpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutorCommon.hpp index 38a68e59c09..29267e4c9d3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutorCommon.hpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutorCommon.hpp @@ -111,8 +111,18 @@ static inline MarkRanges markRangesFromRegionRange(const MergeTreeData::DataPart return res; } +struct ReadGroup { + std::deque mem_block_indexes; + size_t start_range_index; + size_t end_range_index; +}; + +/// computeHandleRanges divide the input handle ranges into one or more read groups +/// the basic rule is +/// 1. the handle ranges will be combined if adjacent handle ranges is combinable +/// 2. a mem block only exists in a single read groups template -static inline void computeHandleRanges(std::vector> & block_data, +static inline void computeHandleRanges(std::vector & read_groups, std::vector, size_t>> & handle_ranges, std::vector & region_group_range_parts, std::vector> & region_group_handle_ranges, @@ -122,24 +132,48 @@ static inline void computeHandleRanges(std::vector> & block_d const Settings & settings, const size_t min_marks_for_seek) { - block_data.resize(handle_ranges.size()); + read_groups.resize(handle_ranges.size()); { size_t size = 0; + size_t read_group_size = 0; + std::unordered_set current_region_indexes; - block_data[0].emplace_back(handle_ranges[0].second); + read_groups[0].mem_block_indexes.emplace_back(handle_ranges[0].second); + current_region_indexes.insert(handle_ranges[0].second); + read_groups[0].start_range_index = 0; for (size_t i = 1; i < handle_ranges.size(); ++i) { if (handle_ranges[i].first.first == handle_ranges[size].first.second) handle_ranges[size].first.second = handle_ranges[i].first.second; else + { handle_ranges[++size] = handle_ranges[i]; - - block_data[size].emplace_back(handle_ranges[i].second); + /// check if it is ok to start new read group + /// a mem block should only exists in a single read group + if (current_region_indexes.find(handle_ranges[i].second) == current_region_indexes.end()) + { + /// if the region index in handle_ranges[i] is not in current_region_indexes + /// then it is safe to start a new read group + /// NOTE this assumes region index in handle_ranges is monotonous + read_groups[read_group_size++].end_range_index = size - 1; + read_groups[read_group_size].start_range_index = size; + current_region_indexes.clear(); + } + } + + /// to avoid duplicate mem block in one read group + if (current_region_indexes.find(handle_ranges[i].second) == current_region_indexes.end()) + { + read_groups[read_group_size].mem_block_indexes.emplace_back(handle_ranges[i].second); + current_region_indexes.insert(handle_ranges[i].second); + } } + read_groups[read_group_size].end_range_index = size; size = size + 1; + read_group_size = read_group_size + 1; handle_ranges.resize(size); - block_data.resize(size); + read_groups.resize(read_group_size); } region_group_range_parts.assign(handle_ranges.size(), {}); diff --git a/dbms/src/Storages/RegionQueryInfo.h b/dbms/src/Storages/RegionQueryInfo.h index ab709a8547d..a0b7ab7b5cd 100644 --- a/dbms/src/Storages/RegionQueryInfo.h +++ b/dbms/src/Storages/RegionQueryInfo.h @@ -11,6 +11,8 @@ struct RegionQueryInfo UInt64 version; UInt64 conf_version; HandleRange range_in_table; + // required handle ranges is the handle range specified in DAG request + std::vector> required_handle_ranges; bool operator<(const RegionQueryInfo & o) const { return range_in_table < o.range_in_table; } }; diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 3c90092dc86..22ba2ffb1d3 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -26,8 +26,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -349,7 +349,8 @@ RegionMap doLearnerRead(const TiDB::TableID table_id, // { if (region == nullptr) continue; - regions_info.emplace_back(RegionQueryInfo{id, region->version(), region->confVer(), {0, 0}}); + regions_info.emplace_back( + RegionQueryInfo{id, region->version(), region->confVer(), region->getHandleRangeByTable(table_id), {}}); } } @@ -587,8 +588,17 @@ BlockInputStreams StorageDeltaMerge::read( // std::stringstream ss; for (const auto & region : mvcc_query_info.regions_query_info) { - const auto & range = region.range_in_table; - ss << region.region_id << "[" << range.first.toString() << "," << range.second.toString() << "),"; + if (!region.required_handle_ranges.empty()) + { + for (const auto & range : region.required_handle_ranges) + ss << region.region_id << "[" << range.first.toString() << "," << range.second.toString() << "),"; + } + else + { + /// only used for test cases + const auto & range = region.range_in_table; + ss << region.region_id << "[" << range.first.toString() << "," << range.second.toString() << "),"; + } } std::stringstream ss_merged_range; for (const auto & range : ranges) diff --git a/dbms/src/Storages/StorageDeltaMergeHelpers.h b/dbms/src/Storages/StorageDeltaMergeHelpers.h index ea636817446..6fddb4d57f5 100644 --- a/dbms/src/Storages/StorageDeltaMergeHelpers.h +++ b/dbms/src/Storages/StorageDeltaMergeHelpers.h @@ -34,60 +34,73 @@ inline DM::HandleRange toDMHandleRange(const HandleRange & range) inline DM::HandleRanges getQueryRanges(const DB::MvccQueryInfo::RegionsQueryInfo & regions) { + std::vector> handle_ranges; + for (const auto & region_info : regions) + { + if (!region_info.required_handle_ranges.empty()) + { + for (const auto & handle_range : region_info.required_handle_ranges) + handle_ranges.push_back(handle_range); + } + else + { + /// only used for test cases + handle_ranges.push_back(region_info.range_in_table); + } + } DM::HandleRanges ranges; - if (regions.empty()) + if (handle_ranges.empty()) { // Just for test cases ranges.emplace_back(DB::DM::HandleRange::newAll()); return ranges; } - else if (regions.size() == 1) + else if (handle_ranges.size() == 1) { // Shortcut for only one region info - const auto & range_in_table = regions[0].range_in_table; + const auto & range_in_table = handle_ranges[0]; ranges.emplace_back(toDMHandleRange(range_in_table)); return ranges; } // Init index with [0, n) // http: //www.cplusplus.com/reference/numeric/iota/ - std::vector sort_index(regions.size()); + std::vector sort_index(handle_ranges.size()); std::iota(sort_index.begin(), sort_index.end(), 0); std::sort(sort_index.begin(), sort_index.end(), // - [®ions](const size_t lhs, const size_t rhs) { return regions[lhs] < regions[rhs]; }); + [&handle_ranges](const size_t lhs, const size_t rhs) { return handle_ranges[lhs] < handle_ranges[rhs]; }); - ranges.reserve(regions.size()); + ranges.reserve(handle_ranges.size()); DM::HandleRange current; - for (size_t i = 0; i < regions.size(); ++i) + for (size_t i = 0; i < handle_ranges.size(); ++i) { const size_t region_idx = sort_index[i]; - const auto & region = regions[region_idx]; - const auto & range_in_table = region.range_in_table; + const auto & handle_range = handle_ranges[region_idx]; if (i == 0) { - current.start = range_in_table.first.handle_id; - current.end = getRangeEndID(range_in_table.second); + current.start = handle_range.first.handle_id; + current.end = getRangeEndID(handle_range.second); } - else if (current.end == range_in_table.first.handle_id) + else if (current.end == handle_range.first.handle_id) { // concat this range_in_table to current - current.end = getRangeEndID(range_in_table.second); + current.end = getRangeEndID(handle_range.second); } - else if (current.end < range_in_table.first.handle_id) + else if (current.end < handle_range.first.handle_id) { ranges.emplace_back(current); // start a new range - current.start = range_in_table.first.handle_id; - current.end = getRangeEndID(range_in_table.second); + current.start = handle_range.first.handle_id; + current.end = getRangeEndID(handle_range.second); } else { throw Exception("Overlap region range between " + current.toString() + " and [" // - + range_in_table.first.toString() + "," + range_in_table.second.toString() + ")"); + + handle_range.first.toString() + "," + handle_range.second.toString() + ")"); } } ranges.emplace_back(current); diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 64d9b0cd49f..50eeafe6bc6 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -159,7 +159,8 @@ std::tuple RegionTable::readBlockByReg RegionVersion conf_version, bool resolve_locks, Timestamp start_ts, - DB::HandleRange & handle_range) + DB::HandleRange & handle_range, + RegionScanFilterPtr scan_filter) { if (!region) throw Exception(std::string(__PRETTY_FUNCTION__) + ": region is null", ErrorCodes::LOGICAL_ERROR); @@ -207,7 +208,8 @@ std::tuple RegionTable::readBlockByReg Block block; { bool ok = false; - std::tie(block, ok) = readRegionBlock(table_info, columns, column_names_to_read, data_list_read, start_ts, true); + std::tie(block, ok) = readRegionBlock(table_info, columns, column_names_to_read, data_list_read, + start_ts, true, scan_filter); if (!ok) // TODO: Enrich exception message. throw Exception("Read region " + std::to_string(region->id()) + " of table " + std::to_string(table_info.id) + " failed", diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.cpp b/dbms/src/Storages/Transaction/RegionBlockReader.cpp index 6deacf55602..73387aab4d6 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.cpp +++ b/dbms/src/Storages/Transaction/RegionBlockReader.cpp @@ -105,7 +105,8 @@ void setPKVersionDel(ColumnUInt8 & delmark_col, ColumnUInt64 & version_col, MutableColumnPtr & pk_column, const RegionDataReadInfoList & data_list, - const Timestamp tso) + const Timestamp tso, + RegionScanFilterPtr scan_filter) { ColumnUInt8::Container & delmark_data = delmark_col.getData(); ColumnUInt64::Container & version_data = version_col.getData(); @@ -121,6 +122,18 @@ void setPKVersionDel(ColumnUInt8 & delmark_col, if (commit_ts > tso) continue; + bool should_skip; + if constexpr (pk_type == TMTPKType::UINT64) + { + should_skip = scan_filter != nullptr && scan_filter->filter(static_cast(handle)); + } + else + { + should_skip = scan_filter != nullptr && scan_filter->filter(static_cast(handle)); + } + if(should_skip) + continue; + delmark_data.emplace_back(write_type == Region::DelFlag); version_data.emplace_back(commit_ts); @@ -138,7 +151,8 @@ std::tuple readRegionBlock(const TableInfo & table_info, const Names & column_names_to_read, RegionDataReadInfoList & data_list, Timestamp start_ts, - bool force_decode) + bool force_decode, + RegionScanFilterPtr scan_filter) { auto delmark_col = ColumnUInt8::create(); auto version_col = ColumnUInt64::create(); @@ -217,7 +231,7 @@ std::tuple readRegionBlock(const TableInfo & table_info, break; } - func(*delmark_col, *version_col, column_map.getMutableColumnPtr(handle_col_id), data_list, start_ts); + func(*delmark_col, *version_col, column_map.getMutableColumnPtr(handle_col_id), data_list, start_ts, scan_filter); } // optimize for only need handle, tso, delmark. diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.h b/dbms/src/Storages/Transaction/RegionBlockReader.h index ad78f98686c..115c40ed143 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.h +++ b/dbms/src/Storages/Transaction/RegionBlockReader.h @@ -19,6 +19,54 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class RegionScanFilter { + bool is_full_range_scan; + std::vector> int64_ranges; + std::vector> uint64_ranges; + + bool isValidHandle(UInt64 handle) + { + for(const auto & range : uint64_ranges) + { + if (handle >= range.first && handle < range.second) + { + return true; + } + } + return false; + } + bool isValidHandle(Int64 handle) + { + for(const auto & range : int64_ranges) + { + if (handle >= range.first && handle < range.second) + { + return true; + } + } + return false; + } +public: + RegionScanFilter(bool is_full_range_scan_, std::vector> int64_ranges_, std::vector> uint64_ranges_) + : is_full_range_scan(is_full_range_scan_), int64_ranges(std::move(int64_ranges_)), uint64_ranges(std::move(uint64_ranges_)) {} + bool filter(UInt64 handle) + { + return !is_full_range_scan && !isValidHandle(handle); + } + bool filter(Int64 handle) + { + return !is_full_range_scan && !isValidHandle(handle); + } + bool isFullRangeScan() + { + return is_full_range_scan; + } + const std::vector> & getUInt64Ranges() { return uint64_ranges; } + const std::vector> & getInt64Ranges() { return int64_ranges; } +}; + +using RegionScanFilterPtr = std::shared_ptr; + /// Read the region data in data_list, decode based on the given table_info and columns, as a block. /// /// Data with commit_ts > start_ts will be ignored. This is for the sake of decode safety on read, @@ -36,6 +84,7 @@ std::tuple readRegionBlock(const TiDB::TableInfo & table_info, const Names & column_names_to_read, RegionDataReadInfoList & data_list, Timestamp start_ts, - bool force_decode); + bool force_decode, + RegionScanFilterPtr scan_filter = nullptr); } // namespace DB diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 4d3ee753e83..4ce22d85b5d 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -138,7 +139,8 @@ class RegionTable : private boost::noncopyable RegionVersion conf_version, bool resolve_locks, Timestamp start_ts, - DB::HandleRange & handle_range); + DB::HandleRange & handle_range, + RegionScanFilterPtr scan_filter = nullptr); /// Check if there are any lock should be resolved, if so, throw LockException. static void resolveLocks(Region::CommittedScanner & scanner, const Timestamp start_ts); From 9b1690c1021db55c37ddd06a49032085e08221b8 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 26 Mar 2020 09:33:53 +0800 Subject: [PATCH 40/54] merge master branch (#556) * fix daily test fail (#520) * fix daily test fail * fix * Add fullstack test for engine DeltaTree (#524) ## Add fullstack test for engine DeltaTree. * Refine `tests/docker/run.sh` and split `tests/docker/docker-compose.yaml` into `tests/docker/{gtest/mock-test/cluster/tiflash-dt/tiflash-tmt}.yaml` `fullstack/ddl`,`fullstack-test/fault-inject` will be enabled in #526 ## Others * Add column `tidb_table_id` in `system.tables` * Add some debugging info Signed-off-by: JaySon-Huang * Bugfix: schrodinger bank2 fail (#521) * Flush committed data in Region after resolve locks * Stop append into last packs after split. * Remove last_cache in Delta to reduce code complexity. * Add system table: dt_tables and dt_segments, for debug. * [FLASH-1008] Support br restore & ingest sst (#529) * [flash-1018]fix bug of datetime default value (#534) * fix bug of datetime default value * address comment * Using SegmentSnapshotPtr instead of SegmentSnapshot (#532) * [Flash-664] Enable DDL for engine DeltaTree (#526) - [x] Enable unittest in gtest_dbms - [x] Enable mock test in tests/delta-merge-test - [x] Enable fullstack-test/ddl - [x] Enable fullstack-test/inject (Imported in https://github.com/pingcap/tics/pull/443) - [x] Refine exception while read / write to DeltaTree (FLASH-994) * Use `Exception::addMessage` to add more diagnostics for locate which table is wrong (commit: 716ae4a) - [x] shutdown should cancel all background tasks (FLASH-995) (commit: 7470c2f) - [x] Run schrodinger/sddl test ## Others * Add atomic-rename table test in `tests/fullstack-test/fault-inject/rename-table.test`, but did not enable this. We will fix it later. * "dt" engine ONLY support disable_bg_flush = true. If background flush is enabled, read will not triggle schema sync. Which means that we may not get the right result with out-dated schema. * Found that 'zero' value of type year is not match with tikv (FLASH-1023) Signed-off-by: JaySon-Huang * [FLASH-1027] Fix: proxy override system signal listening (#541) * remove signal listening from proxy. * while terminating, stop all learner read. * [FLASH-1026] Synchronization between drop table and remove region (#538) * Set default storage engine to DT (#547) * Do region flush in Region::handleWriteRaftCmd (#542) * [FLASH-1028] Region merge should not remove data (#544) * Remove region after region merge should not remove data Signed-off-by: JaySon-Huang * Fix region init index Signed-off-by: JaySon-Huang * Add region_merge.test for DT Signed-off-by: JaySon-Huang * Fix different behavior between DT and TMT Signed-off-by: JaySon-Huang * Fix mock remove region Signed-off-by: JaySon-Huang Co-authored-by: pingcap-github-bot * clean useless code Co-authored-by: JaySon Co-authored-by: Flowyi Co-authored-by: Tong Zhigao Co-authored-by: Han Fei Co-authored-by: pingcap-github-bot --- contrib/tiflash-proxy | 2 +- dbms/src/Common/FailPoint.h | 6 +- dbms/src/Core/ColumnWithTypeAndName.cpp | 20 +- dbms/src/Databases/DatabaseOrdinary.cpp | 7 + dbms/src/Debug/DBGInvoker.cpp | 10 +- dbms/src/Debug/MockTiDB.cpp | 2 +- dbms/src/Debug/MockTiKV.h | 7 +- dbms/src/Debug/dbgFuncMockRaftCommand.h | 2 + dbms/src/Debug/dbgFuncMockTiDBTable.cpp | 2 +- dbms/src/Debug/dbgFuncMockTiDBTable.h | 2 +- dbms/src/Debug/dbgFuncRegion.cpp | 77 ++- dbms/src/Debug/dbgFuncRegion.h | 1 + dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 13 + .../Interpreters/InterpreterDescribeQuery.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 43 +- .../Interpreters/InterpreterManageQuery.cpp | 4 +- dbms/src/Parsers/ASTSelectQuery.cpp | 6 + dbms/src/Parsers/ASTSelectQuery.h | 1 + dbms/src/Parsers/ExpressionElementParsers.cpp | 1 + dbms/src/Parsers/ParserSelectQuery.cpp | 8 + dbms/src/Server/ClusterManagerService.cpp | 11 + dbms/src/Server/Server.cpp | 28 +- .../DeltaMerge/Delta/CompactDelta.cpp | 7 +- .../Storages/DeltaMerge/Delta/FlushDelta.cpp | 29 +- dbms/src/Storages/DeltaMerge/Delta/Pack.cpp | 82 ++- dbms/src/Storages/DeltaMerge/Delta/Pack.h | 13 +- .../Storages/DeltaMerge/Delta/Snapshot.cpp | 18 +- dbms/src/Storages/DeltaMerge/DeltaMerge.h | 24 +- .../Storages/DeltaMerge/DeltaMergeHelpers.h | 11 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 148 +++-- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 47 +- .../Storages/DeltaMerge/DeltaValueSpace.cpp | 60 +- .../src/Storages/DeltaMerge/DeltaValueSpace.h | 67 ++- dbms/src/Storages/DeltaMerge/File/DMFile.cpp | 15 +- dbms/src/Storages/DeltaMerge/File/DMFile.h | 53 +- .../DeltaMerge/File/DMFilePackFilter.h | 2 +- .../Storages/DeltaMerge/File/DMFileReader.cpp | 92 ++-- dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp | 176 +++--- dbms/src/Storages/DeltaMerge/Segment.cpp | 171 +++--- dbms/src/Storages/DeltaMerge/Segment.h | 67 +-- .../Storages/DeltaMerge/SegmentReadTaskPool.h | 14 +- .../Storages/DeltaMerge/StableValueSpace.cpp | 19 +- .../DeltaMerge/convertColumnTypeHelpers.cpp | 360 ++++++++++++ .../DeltaMerge/convertColumnTypeHelpers.h | 22 + .../DeltaMerge/tests/gtest_convert_column.cpp | 246 +++++++++ .../DeltaMerge/tests/gtest_dm_file.cpp | 521 +++++++++++++++--- .../DeltaMerge/tests/gtest_dm_segment.cpp | 253 +++++++-- dbms/src/Storages/IManageableStorage.h | 6 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 15 +- dbms/src/Storages/MutableSupport.cpp | 3 +- dbms/src/Storages/MutableSupport.h | 3 +- dbms/src/Storages/StorageDebugging.cpp | 3 + dbms/src/Storages/StorageDebugging.h | 1 + dbms/src/Storages/StorageDeltaMerge.cpp | 124 ++++- dbms/src/Storages/StorageDeltaMerge.h | 6 +- dbms/src/Storages/StorageMergeTree.h | 2 +- .../System/StorageSystemDTSegment.cpp | 95 ++++ .../Storages/System/StorageSystemDTSegments.h | 34 ++ .../Storages/System/StorageSystemDTTable.cpp | 178 ++++++ .../Storages/System/StorageSystemDTTables.h | 34 ++ .../Storages/System/StorageSystemTables.cpp | 62 ++- .../Storages/System/attachSystemTables.cpp | 4 + .../Storages/Transaction/ApplySnapshot.cpp | 67 ++- dbms/src/Storages/Transaction/KVStore.cpp | 50 +- dbms/src/Storages/Transaction/KVStore.h | 18 +- .../Storages/Transaction/PartitionStreams.cpp | 172 ++++-- dbms/src/Storages/Transaction/ProxyFFI.cpp | 22 +- dbms/src/Storages/Transaction/ProxyFFIType.h | 21 +- dbms/src/Storages/Transaction/Region.cpp | 64 ++- dbms/src/Storages/Transaction/Region.h | 24 +- .../Storages/Transaction/RegionCFDataBase.cpp | 7 +- .../Storages/Transaction/RegionCFDataTrait.h | 2 + dbms/src/Storages/Transaction/RegionData.cpp | 10 +- dbms/src/Storages/Transaction/RegionMeta.cpp | 13 +- dbms/src/Storages/Transaction/RegionMeta.h | 3 +- dbms/src/Storages/Transaction/RegionTable.cpp | 36 +- dbms/src/Storages/Transaction/RegionTable.h | 19 +- .../Storages/Transaction/SchemaBuilder.cpp | 12 +- dbms/src/Storages/Transaction/TMTContext.cpp | 9 + dbms/src/Storages/Transaction/TMTContext.h | 5 + dbms/src/Storages/Transaction/TMTStorages.cpp | 9 +- dbms/src/Storages/Transaction/TiDB.cpp | 44 +- dbms/src/Storages/Transaction/TiDB.h | 9 +- dbms/src/test_utils/TiflashTestBasic.h | 16 + release-centos7/build/build-tiflash-ci.sh | 2 +- tests/_env.sh | 3 - tests/delta-merge-test/ddl/alter.test | 13 +- .../ddl/alter_default_value.test | 50 -- .../ddl/alter_joint_primary_key.test | 36 +- .../delta-merge-test/ddl/alter_nullable.test | 2 - .../delta-merge-test/raft/bugs/FLASH-484.test | 2 - tests/delta-merge-test/raft/region_merge.test | 107 ++++ .../raft/schema/alter_for_nullable.test | 70 +++ .../raft/schema/alter_on_read.test | 215 ++++++++ .../raft/schema/alter_on_write.test | 132 +++++ .../raft/schema/default_value.test | 42 ++ .../raft/schema/drop_on_read.test | 27 + .../raft/schema/drop_on_restart.test | 54 ++ .../raft/schema/drop_on_write.test | 67 +++ .../delta-merge-test/raft/schema/mydate.test | 73 +++ .../raft/schema/partition_table_restart.test | 43 ++ .../raft/schema/rename_column copy.test | 216 ++++++++ .../raft/schema/rename_column.test | 3 - .../raft/schema/rename_on_read.test | 24 + .../raft/schema/rename_on_write.test | 32 ++ .../raft/schema/truncate_on_read.test | 27 + .../raft/schema/truncate_on_write.test | 29 + tests/docker/cluster.yaml | 56 ++ tests/docker/config/tics_dt.toml | 23 + .../config/{config.toml => tics_tmt.toml} | 3 +- tests/docker/config/tiflash_dt.toml | 39 ++ .../config/{tiflash.toml => tiflash_tmt.toml} | 3 +- tests/docker/config/tikv-learner.toml | 1 + tests/docker/config/tikv.toml | 1 + tests/docker/docker-compose.yaml | 108 ---- tests/docker/gtest.yaml | 13 + tests/docker/mock-test-dt.yaml | 21 + tests/docker/mock-test-tmt.yaml | 21 + tests/docker/run.sh | 55 +- tests/docker/tiflash-dt.yaml | 28 + tests/docker/tiflash-tmt.yaml | 28 + .../ddl/alter_datetime_default_value.test | 90 +++ .../ddl/alter_decimal_default_value.test | 40 ++ .../ddl/alter_default_value.test | 21 + .../ddl/datetime_default_value.test | 36 ++ .../expr/single_ifnull_in_predicate.test | 2 + .../fault-inject/alter-table.test | 17 +- .../fault-inject/drop-table.test | 31 +- .../fault-inject/rename-table.test | 46 ++ tests/fullstack-test/sample.test | 5 +- tests/generate-fullstack-test.py | 10 +- tests/mutable-test/txn_mock/ingest_sst.test | 26 + tests/run-test.py | 1 + tests/run-test.sh | 2 +- 134 files changed, 4928 insertions(+), 1042 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.cpp create mode 100644 dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.h create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp create mode 100644 dbms/src/Storages/System/StorageSystemDTSegment.cpp create mode 100644 dbms/src/Storages/System/StorageSystemDTSegments.h create mode 100644 dbms/src/Storages/System/StorageSystemDTTable.cpp create mode 100644 dbms/src/Storages/System/StorageSystemDTTables.h delete mode 100644 tests/delta-merge-test/ddl/alter_default_value.test create mode 100644 tests/delta-merge-test/raft/region_merge.test create mode 100644 tests/delta-merge-test/raft/schema/alter_for_nullable.test create mode 100644 tests/delta-merge-test/raft/schema/alter_on_read.test create mode 100644 tests/delta-merge-test/raft/schema/alter_on_write.test create mode 100644 tests/delta-merge-test/raft/schema/default_value.test create mode 100644 tests/delta-merge-test/raft/schema/drop_on_read.test create mode 100644 tests/delta-merge-test/raft/schema/drop_on_restart.test create mode 100644 tests/delta-merge-test/raft/schema/drop_on_write.test create mode 100644 tests/delta-merge-test/raft/schema/mydate.test create mode 100644 tests/delta-merge-test/raft/schema/partition_table_restart.test create mode 100644 tests/delta-merge-test/raft/schema/rename_column copy.test create mode 100644 tests/delta-merge-test/raft/schema/rename_on_read.test create mode 100644 tests/delta-merge-test/raft/schema/rename_on_write.test create mode 100644 tests/delta-merge-test/raft/schema/truncate_on_read.test create mode 100644 tests/delta-merge-test/raft/schema/truncate_on_write.test create mode 100644 tests/docker/cluster.yaml create mode 100644 tests/docker/config/tics_dt.toml rename tests/docker/config/{config.toml => tics_tmt.toml} (88%) create mode 100644 tests/docker/config/tiflash_dt.toml rename tests/docker/config/{tiflash.toml => tiflash_tmt.toml} (92%) delete mode 100644 tests/docker/docker-compose.yaml create mode 100644 tests/docker/gtest.yaml create mode 100644 tests/docker/mock-test-dt.yaml create mode 100644 tests/docker/mock-test-tmt.yaml create mode 100644 tests/docker/tiflash-dt.yaml create mode 100644 tests/docker/tiflash-tmt.yaml create mode 100644 tests/fullstack-test/ddl/alter_datetime_default_value.test create mode 100644 tests/fullstack-test/ddl/alter_decimal_default_value.test create mode 100644 tests/fullstack-test/ddl/alter_default_value.test create mode 100644 tests/fullstack-test/ddl/datetime_default_value.test create mode 100644 tests/fullstack-test/fault-inject/rename-table.test create mode 100644 tests/mutable-test/txn_mock/ingest_sst.test diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 5885ec91490..ad4addbdcab 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 5885ec91490120b66c86fcb7d6aa7c1d4259c735 +Subproject commit ad4addbdcab1170f972d88723fe2840068851914 diff --git a/dbms/src/Common/FailPoint.h b/dbms/src/Common/FailPoint.h index 368a4e8edc5..f1edb3e1e61 100644 --- a/dbms/src/Common/FailPoint.h +++ b/dbms/src/Common/FailPoint.h @@ -15,11 +15,12 @@ extern const int FAIL_POINT_ERROR; #define FAIL_POINT_REGISTER(name) static constexpr char name[] = #name ""; -#define FAIL_POINT_ENABLE(trigger, name) else if (trigger == name) fiu_enable(name, 1, nullptr, FIU_ONETIME); +#define FAIL_POINT_ENABLE(trigger, name) else if (trigger == name) { fiu_enable(name, 1, nullptr, FIU_ONETIME); } FAIL_POINT_REGISTER(exception_between_drop_meta_and_data) FAIL_POINT_REGISTER(exception_between_alter_data_and_meta) FAIL_POINT_REGISTER(exception_drop_table_during_remove_meta) +FAIL_POINT_REGISTER(exception_between_rename_table_data_and_metadata); #define FAIL_POINT_TRIGGER_EXCEPTION(fail_point) \ fiu_do_on(fail_point, throw Exception("Fail point " #fail_point " is triggered.", ErrorCodes::FAIL_POINT_ERROR);) @@ -33,7 +34,8 @@ class FailPointHelper FAIL_POINT_ENABLE(fail_point_name, exception_between_alter_data_and_meta) FAIL_POINT_ENABLE(fail_point_name, exception_between_drop_meta_and_data) FAIL_POINT_ENABLE(fail_point_name, exception_drop_table_during_remove_meta) + FAIL_POINT_ENABLE(fail_point_name, exception_between_rename_table_data_and_metadata) else throw Exception("Cannot find fail point " + fail_point_name, ErrorCodes::FAIL_POINT_ERROR); } }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Core/ColumnWithTypeAndName.cpp b/dbms/src/Core/ColumnWithTypeAndName.cpp index 188bfc09b95..ee88f705a39 100644 --- a/dbms/src/Core/ColumnWithTypeAndName.cpp +++ b/dbms/src/Core/ColumnWithTypeAndName.cpp @@ -1,7 +1,7 @@ #include +#include #include #include -#include namespace DB @@ -9,14 +9,7 @@ namespace DB ColumnWithTypeAndName ColumnWithTypeAndName::cloneEmpty() const { - ColumnWithTypeAndName res; - - res.name = name; - res.type = type; - res.column_id = column_id; - if (column) - res.column = column->cloneEmpty(); - + ColumnWithTypeAndName res{(column != nullptr ? column->cloneEmpty() : nullptr), type, name, column_id, default_value}; return res; } @@ -24,15 +17,14 @@ ColumnWithTypeAndName ColumnWithTypeAndName::cloneEmpty() const bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) const { // TODO should we check column_id here? - return name == other.name - && ((!type && !other.type) || (type && other.type && type->equals(*other.type))) + return name == other.name && ((!type && !other.type) || (type && other.type && type->equals(*other.type))) && ((!column && !other.column) || (column && other.column && column->getName() == other.column->getName())); } void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const { - out << name; + out << name << ' ' << column_id; if (type) out << ' ' << type->getName(); @@ -43,6 +35,8 @@ void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const out << ' ' << column->dumpStructure(); else out << " nullptr"; + + out << " " << column_id; } String ColumnWithTypeAndName::dumpStructure() const @@ -52,4 +46,4 @@ String ColumnWithTypeAndName::dumpStructure() const return out.str(); } -} +} // namespace DB diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 1c0764bc798..85de32ac0d7 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -328,6 +328,9 @@ void DatabaseOrdinary::removeTable( try { + // If tiflash crash before remove metadata, next time it restart, will + // full apply schema from TiDB. And the old table's metadata and data + // will be removed. FAIL_POINT_TRIGGER_EXCEPTION(exception_drop_table_during_remove_meta); Poco::File(table_metadata_path).remove(); } @@ -410,6 +413,9 @@ void DatabaseOrdinary::renameTable( throw Exception{e}; } + // TODO: Atomic rename table is not fixed. + FAIL_POINT_TRIGGER_EXCEPTION(exception_between_rename_table_data_and_metadata); + ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name)); if (!ast) throw Exception("There is no metadata file for table " + table_name, ErrorCodes::FILE_DOESNT_EXIST); @@ -417,6 +423,7 @@ void DatabaseOrdinary::renameTable( ast_create_query.table = to_table_name; /// NOTE Non-atomic. + // Create new metadata and remove old metadata. to_database_concrete->createTable(context, to_table_name, table, ast); removeTable(context, table_name); } diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index 781f9de5ebd..5d12eea2a3c 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -1,18 +1,18 @@ -#include -#include - #include #include #include #include +#include #include #include #include #include #include -#include #include +#include +#include + namespace DB { @@ -88,6 +88,8 @@ DBGInvoker::DBGInvoker() regSchemafulFunc("dag", dbgFuncDAG); regSchemafulFunc("mock_dag", dbgFuncMockDAG); + + regSchemalessFunc("region_mock_ingest_sst", dbgFuncIngestSST); } void replaceSubstr(std::string & str, const std::string & target, const std::string & replacement) diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index fef86c8ad57..b740535e9e3 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -191,7 +191,7 @@ TableID MockTiDB::newTable(const String & database_name, const String & table_na else if (engine_type == "buggy") table_info.engine_type = TiDB::StorageEngine::DEBUGGING_MEMORY; else - throw Exception("Unknown engine type : " + engine_type + ", must be 'tmt' or 'dm'", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Unknown engine type : " + engine_type + ", must be 'tmt' or 'dt'", ErrorCodes::BAD_ARGUMENTS); auto table = std::make_shared(database_name, table_name, std::move(table_info)); tables_by_id.emplace(table->table_info.id, table); diff --git a/dbms/src/Debug/MockTiKV.h b/dbms/src/Debug/MockTiKV.h index 443cc1705bf..4a3e054fe6d 100644 --- a/dbms/src/Debug/MockTiKV.h +++ b/dbms/src/Debug/MockTiKV.h @@ -15,7 +15,12 @@ class MockTiKV : public ext::singleton std::lock_guard lock(mutex); auto it = raft_index.find(region_id); if (it == raft_index.end()) - it = raft_index.emplace_hint(it, region_id, RAFT_INIT_LOG_INDEX); + { + // Usually index 6 is empty and we ignore it. + // https://github.com/tikv/tikv/issues/7047 + auto init_index = RAFT_INIT_LOG_INDEX + 1; + it = raft_index.emplace_hint(it, region_id, init_index); + } ++(it->second); return it->second; } diff --git a/dbms/src/Debug/dbgFuncMockRaftCommand.h b/dbms/src/Debug/dbgFuncMockRaftCommand.h index 1c6b944d5af..3547753e322 100644 --- a/dbms/src/Debug/dbgFuncMockRaftCommand.h +++ b/dbms/src/Debug/dbgFuncMockRaftCommand.h @@ -29,6 +29,8 @@ struct MockRaftCommand // Usage: // ./storages-client.sh "DBGInvoke region_rollback_merge(region_id, database_name, table_name, start1, end1, start2, end2)" static void dbgFuncRollbackMerge(Context & context, const ASTs & args, DBGInvoker::Printer output); + + static void dbgFuncIngestSST(Context & context, const ASTs & args, DBGInvoker::Printer output); }; } // namespace DB diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp index 6cccf890a81..a037520c258 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp @@ -26,7 +26,7 @@ extern const int LOGICAL_ERROR; void MockTiDBTable::dbgFuncMockTiDBTable(Context & context, const ASTs & args, DBGInvoker::Printer output) { if (args.size() != 3 && args.size() != 4 && args.size() != 5) - throw Exception("Args not matched, should be: database-name, table-name, schema-string [, handle_pk_name], [, engine-type(tmt|dm|buggy)]", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Args not matched, should be: database-name, table-name, schema-string [, handle_pk_name], [, engine-type(tmt|dt|buggy)]", ErrorCodes::BAD_ARGUMENTS); const String & database_name = typeid_cast(*args[0]).name; const String & table_name = typeid_cast(*args[1]).name; diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.h b/dbms/src/Debug/dbgFuncMockTiDBTable.h index 4dd0ece00a7..28f8d7cbed6 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.h +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.h @@ -15,7 +15,7 @@ struct MockTiDBTable // Inject mocked TiDB table. // Usage: // ./storages-client.sh "DBGInvoke mock_tidb_table(database_name, table_name, 'col1 type1, col2 type2, ...'[, engine])" - // engine: [tmt, dm, buggy], tmt by default + // engine: [tmt, dt, buggy], tmt by default static void dbgFuncMockTiDBTable(Context & context, const ASTs & args, DBGInvoker::Printer output); // Inject mocked TiDB table. diff --git a/dbms/src/Debug/dbgFuncRegion.cpp b/dbms/src/Debug/dbgFuncRegion.cpp index 419bca7fb3f..ec768f886df 100644 --- a/dbms/src/Debug/dbgFuncRegion.cpp +++ b/dbms/src/Debug/dbgFuncRegion.cpp @@ -182,9 +182,6 @@ void dbgFuncRegionSnapshot(Context & context, const ASTs & args, DBGInvoker::Pri TMTContext & tmt = context.getTMTContext(); metapb::Region region_info; - SnapshotDataView lock_cf; - SnapshotDataView write_cf; - SnapshotDataView default_cf; region_info.set_id(region_id); region_info.set_start_key(RecordKVFormat::genKey(table_id, start).getStr()); @@ -194,14 +191,8 @@ void dbgFuncRegionSnapshot(Context & context, const ASTs & args, DBGInvoker::Pri *region_info.add_peers() = createPeer(2, true); auto peer_id = 1; - tmt.getKVStore()->handleApplySnapshot(std::move(region_info), - peer_id, - (lock_cf), - (write_cf), - (default_cf), - MockTiKV::instance().getRaftIndex(region_id), - RAFT_INIT_LOG_TERM, - tmt); + tmt.getKVStore()->handleApplySnapshot( + std::move(region_info), peer_id, SnapshotViewArray(), MockTiKV::instance().getRaftIndex(region_id), RAFT_INIT_LOG_TERM, tmt); std::stringstream ss; ss << "put region #" << region_id << ", range[" << start << ", " << end << ")" @@ -335,4 +326,68 @@ void dbgFuncRemoveRegion(Context & context, const ASTs & args, DBGInvoker::Print output(ss.str()); } +void dbgFuncIngestSST(Context & context, const ASTs & args, DBGInvoker::Printer) +{ + const String & database_name = typeid_cast(*args[0]).name; + const String & table_name = typeid_cast(*args[1]).name; + RegionID region_id = (RegionID)safeGet(typeid_cast(*args[2]).value); + RegionID start_handle = (RegionID)safeGet(typeid_cast(*args[3]).value); + RegionID end_handle = (RegionID)safeGet(typeid_cast(*args[4]).value); + MockTiDB::TablePtr table = MockTiDB::instance().getTableByName(database_name, table_name); + + std::vector> write_kv_list, default_kv_list; + + for (auto handle_id = start_handle; handle_id < end_handle; ++handle_id) + { + // make it have only one column Int64 just for test + std::vector fields; + fields.emplace_back(-handle_id); + { + TiKVKey key = RecordKVFormat::genKey(table->id(), handle_id); + std::stringstream ss; + RegionBench::encodeRow(table->table_info, fields, ss); + TiKVValue prewrite_value(ss.str()); + UInt64 commit_ts = handle_id; + UInt64 prewrite_ts = commit_ts; + TiKVValue commit_value = RecordKVFormat::encodeWriteCfValue(Region::PutFlag, prewrite_ts); + TiKVKey commit_key = RecordKVFormat::appendTs(key, commit_ts); + TiKVKey prewrite_key = RecordKVFormat::appendTs(key, prewrite_ts); + + write_kv_list.emplace_back(std::make_pair(std::move(commit_key), std::move(commit_value))); + default_kv_list.emplace_back(std::make_pair(std::move(prewrite_key), std::move(prewrite_value))); + } + } + + { + std::vector keys; + std::vector vals; + for (const auto & kv : write_kv_list) + { + keys.push_back({kv.first.data(), kv.first.dataSize()}); + vals.push_back({kv.second.data(), kv.second.dataSize()}); + } + std::vector snaps; + snaps.push_back(SnapshotView{keys.data(), vals.data(), ColumnFamilyType::Write, keys.size()}); + + auto & tmt = context.getTMTContext(); + tmt.getKVStore()->handleIngestSST(region_id, SnapshotViewArray{snaps.data(), snaps.size()}, + MockTiKV::instance().getRaftIndex(region_id), MockTiKV::instance().getRaftTerm(region_id), tmt); + } + + { + std::vector keys; + std::vector vals; + for (const auto & kv : default_kv_list) + { + keys.push_back({kv.first.data(), kv.first.dataSize()}); + vals.push_back({kv.second.data(), kv.second.dataSize()}); + } + std::vector snaps; + snaps.push_back(SnapshotView{keys.data(), vals.data(), ColumnFamilyType::Default, keys.size()}); + auto & tmt = context.getTMTContext(); + tmt.getKVStore()->handleIngestSST(region_id, SnapshotViewArray{snaps.data(), snaps.size()}, + MockTiKV::instance().getRaftIndex(region_id), MockTiKV::instance().getRaftTerm(region_id), tmt); + } +} + } // namespace DB diff --git a/dbms/src/Debug/dbgFuncRegion.h b/dbms/src/Debug/dbgFuncRegion.h index 6fd0ca91a1e..1dc02fcab55 100644 --- a/dbms/src/Debug/dbgFuncRegion.h +++ b/dbms/src/Debug/dbgFuncRegion.h @@ -50,5 +50,6 @@ void dbgFuncTryFlushRegion(Context & context, const ASTs & args, DBGInvoker::Pri // ./storage-client.sh "DBGInvoke remove_region(region_id)" void dbgFuncRemoveRegion(Context & context, const ASTs & args, DBGInvoker::Printer output); +void dbgFuncIngestSST(Context & context, const ASTs & args, DBGInvoker::Printer output); } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 2653a66e228..efc86ca16be 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -451,6 +451,7 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) info.range_in_table = current_region->getHandleRangeByTable(table_id); query_info.mvcc_query_info->regions_query_info.push_back(info); query_info.mvcc_query_info->concurrent = 0.0; +<<<<<<< HEAD if (ts.next_read_engine() == tipb::EngineType::Local) { pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); @@ -483,6 +484,18 @@ void InterpreterDAG::executeTS(const tipb::TableScan & ts, Pipeline & pipeline) >>>>>>> c2d620553... support batch cop LOG_INFO(log, "dag execution stream size: " << dag.getRegions().size()); +======= + try + { + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + } + catch (DB::Exception & e) + { + e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() + + "`, table_id: " + DB::toString(table_id) + ")"); + throw; + } +>>>>>>> b7776d3a8452a4288445a91a364f2e88c66f0fe6 if (pipeline.streams.empty()) { diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index 79369ad9c48..7954994a58d 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -109,7 +109,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() Block sample_block = getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); - OrderedNameSet filtered_names = MutableSupport::instance().hiddenColumns(table->getName()); + const OrderedNameSet filtered_names = MutableSupport::instance().hiddenColumns(table->getName()); for (const auto & column : columns) { diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 3708ee6269b..98e6443dbee 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -1,15 +1,14 @@ -#include #include - +#include +#include #include #include #include #include #include +#include #include #include -#include -#include namespace DB @@ -17,12 +16,12 @@ namespace DB namespace ErrorCodes { - extern const int TABLE_WAS_NOT_DROPPED; - extern const int DATABASE_NOT_EMPTY; - extern const int UNKNOWN_DATABASE; - extern const int READONLY; - extern const int FAIL_POINT_ERROR; -} +extern const int TABLE_WAS_NOT_DROPPED; +extern const int DATABASE_NOT_EMPTY; +extern const int UNKNOWN_DATABASE; +extern const int READONLY; +extern const int FAIL_POINT_ERROR; +} // namespace ErrorCodes InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} @@ -57,8 +56,8 @@ BlockIO InterpreterDropQuery::execute() { if (drop.database.empty() && !drop.temporary) { - LOG_WARNING((&Logger::get("InterpreterDropQuery")), - "It is recommended to use `DROP TEMPORARY TABLE` to delete temporary tables"); + LOG_WARNING( + (&Logger::get("InterpreterDropQuery")), "It is recommended to use `DROP TEMPORARY TABLE` to delete temporary tables"); } table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown @@ -119,15 +118,16 @@ BlockIO InterpreterDropQuery::execute() if (!drop.detach) { if (!table.first->checkTableCanBeDropped()) - throw Exception("Table " + database_name + "." + table.first->getTableName() + " couldn't be dropped due to failed pre-drop check", + throw Exception( + "Table " + database_name + "." + table.first->getTableName() + " couldn't be dropped due to failed pre-drop check", ErrorCodes::TABLE_WAS_NOT_DROPPED); } - table.first->shutdown(); - /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table.first->lockForAlter(__PRETTY_FUNCTION__); + table.first->shutdown(); + String current_table_name = table.first->getTableName(); if (drop.detach) @@ -147,13 +147,8 @@ BlockIO InterpreterDropQuery::execute() table.first->is_dropped = true; - // drop is complete, then clean tmt context; - if (auto storage = std::static_pointer_cast(table.first)) - storage->removeFromTMTContext(); - - String database_data_path = database->getDataPath(); - /// If it is not virtual database like Dictionary then drop remaining data dir + const String database_data_path = database->getDataPath(); if (!database_data_path.empty()) { String table_data_path = database_data_path + "/" + escapeForFileName(current_table_name); @@ -161,6 +156,10 @@ BlockIO InterpreterDropQuery::execute() if (Poco::File(table_data_path).exists()) Poco::File(table_data_path).remove(true); } + + // drop is complete, then clean tmt context + if (auto storage = std::dynamic_pointer_cast(table.first); storage) + storage->removeFromTMTContext(); } } @@ -214,4 +213,4 @@ void InterpreterDropQuery::checkAccess(const ASTDropQuery & drop) throw Exception("Cannot drop table in readonly mode", ErrorCodes::READONLY); } -} +} // namespace DB diff --git a/dbms/src/Interpreters/InterpreterManageQuery.cpp b/dbms/src/Interpreters/InterpreterManageQuery.cpp index ca3fd6b1828..4e4aec5ba2b 100644 --- a/dbms/src/Interpreters/InterpreterManageQuery.cpp +++ b/dbms/src/Interpreters/InterpreterManageQuery.cpp @@ -3,7 +3,7 @@ #include #include #include - +#include #include namespace DB @@ -14,7 +14,7 @@ BlockIO InterpreterManageQuery::execute() StoragePtr table = context.getTable(ast.database, ast.table); IManageableStorage * manageable_storage; - if (table->getName() == "DeltaMerge") + if (table->getName() == MutableSupport::delta_tree_storage_name) { manageable_storage = &dynamic_cast(*table); } diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 4db406f2a49..672c5a9aaca 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -97,6 +97,12 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F partition_expression_list->formatImpl(s, state, frame); } + if (partition_expression_list) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SEGMENT " << (s.hilite ? hilite_none : ""); + segment_expression_list->formatImpl(s, state, frame); + } + if (prewhere_expression) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "PREWHERE " << (s.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTSelectQuery.h b/dbms/src/Parsers/ASTSelectQuery.h index 96508073e17..70c27262de9 100644 --- a/dbms/src/Parsers/ASTSelectQuery.h +++ b/dbms/src/Parsers/ASTSelectQuery.h @@ -27,6 +27,7 @@ class ASTSelectQuery : public IAST ASTPtr select_expression_list; ASTPtr tables; ASTPtr partition_expression_list; + ASTPtr segment_expression_list; ASTPtr prewhere_expression; ASTPtr where_expression; ASTPtr group_expression_list; diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 5d7b3b10a78..14561cc62c9 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -667,6 +667,7 @@ const char * ParserAliasBase::restricted_keywords[] = "UNION", "INTO", "PARTITION", + "SEGMENT", nullptr }; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 1a7a9926443..9dd8c9c03af 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -32,6 +32,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_distinct("DISTINCT"); ParserKeyword s_from("FROM"); ParserKeyword s_partition("PARTITION"); + ParserKeyword s_segment("SEGMENT"); ParserKeyword s_prewhere("PREWHERE"); ParserKeyword s_where("WHERE"); ParserKeyword s_group_by("GROUP BY"); @@ -90,6 +91,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } + /// SEGMENT p or SEGMENT (p1, p2, ...) + if (s_segment.ignore(pos, expected)) + { + if (!ParserPartition().parse(pos, select_query->segment_expression_list, expected)) + return false; + } + /// PREWHERE expr if (s_prewhere.ignore(pos, expected)) { diff --git a/dbms/src/Server/ClusterManagerService.cpp b/dbms/src/Server/ClusterManagerService.cpp index f451fcee5d7..851c8e5b1e3 100644 --- a/dbms/src/Server/ClusterManagerService.cpp +++ b/dbms/src/Server/ClusterManagerService.cpp @@ -18,10 +18,21 @@ constexpr long MILLISECOND = 1000; constexpr long INIT_DELAY = 5; void ClusterManagerService::run(const std::string & bin_path, const std::vector & args) +try { auto proc = ShellCommand::executeDirect(bin_path, args); proc->wait(); } +catch (DB::Exception & e) +{ + std::stringstream ss; + ss << bin_path; + for (const auto & arg : args) + { + ss << " " << arg; + } + e.addMessage("(while running `" + ss.str() + "`)"); +} ClusterManagerService::ClusterManagerService(DB::Context & context_, const std::string & config_path) : context(context_), timer(), log(&Logger::get("ClusterManagerService")) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 3433b1869d6..849f25da121 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -395,7 +396,7 @@ int Server::main(const std::vector & /*args*/) bool disable_bg_flush = false; - ::TiDB::StorageEngine engine_if_empty = ::TiDB::StorageEngine::TMT; + ::TiDB::StorageEngine engine_if_empty = ::TiDB::StorageEngine::DT; ::TiDB::StorageEngine engine = engine_if_empty; if (config().has("raft")) @@ -447,22 +448,24 @@ int Server::main(const std::vector & /*args*/) } /// "tmt" engine ONLY support disable_bg_flush = false. - /// "dm" engine by default disable_bg_flush = true. + /// "dt" engine ONLY support disable_bg_flush = true. String disable_bg_flush_conf = "raft.disable_bg_flush"; if (engine == ::TiDB::StorageEngine::TMT) { if (config().has(disable_bg_flush_conf) && config().getBool(disable_bg_flush_conf)) - throw Exception( - "Illegal arguments: disable background flush while using engine TxnMergeTree.", ErrorCodes::INVALID_CONFIG_PARAMETER); + throw Exception("Illegal arguments: disable background flush while using engine " + MutableSupport::txn_storage_name, + ErrorCodes::INVALID_CONFIG_PARAMETER); disable_bg_flush = false; } else if (engine == ::TiDB::StorageEngine::DT) { - if (config().has(disable_bg_flush_conf)) - disable_bg_flush = config().getBool(disable_bg_flush_conf); - else - disable_bg_flush = true; + /// If background flush is enabled, read will not triggle schema sync. + /// Which means that we may get the wrong result with outdated schema. + if (config().has(disable_bg_flush_conf) && !config().getBool(disable_bg_flush_conf)) + throw Exception("Illegal arguments: enable background flush while using engine " + MutableSupport::delta_tree_storage_name, + ErrorCodes::INVALID_CONFIG_PARAMETER); + disable_bg_flush = true; } } @@ -558,10 +561,12 @@ int Server::main(const std::vector & /*args*/) .fn_handle_apply_snapshot = HandleApplySnapshot, .fn_atomic_update_proxy = AtomicUpdateProxy, .fn_handle_destroy = HandleDestroy, + .fn_handle_ingest_sst = HandleIngestSST, + .fn_handle_check_terminated = HandleCheckTerminated, // a special number, also defined in proxy .magic_number = 0x13579BDF, - .version = 2}; + .version = 4}; auto proxy_runner = std::thread([&proxy_conf, &log, &helper]() { if (!proxy_conf.inited) @@ -850,6 +855,11 @@ int Server::main(const std::vector & /*args*/) ClusterManagerService cluster_manager_service(*global_context, config_path); waitForTerminationRequest(); + + { + global_context->getTMTContext().setTerminated(); + LOG_INFO(log, "Set tmt context terminated"); + } } return Application::EXIT_OK; diff --git a/dbms/src/Storages/DeltaMerge/Delta/CompactDelta.cpp b/dbms/src/Storages/DeltaMerge/Delta/CompactDelta.cpp index e588763a916..946cc0d3e1d 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/CompactDelta.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/CompactDelta.cpp @@ -57,8 +57,8 @@ bool DeltaValueSpace::compact(DMContext & context) { if (!pack->isSaved()) break; - if ((unlikely(pack->isMutable()))) - throw Exception("Saved pack is mutable", ErrorCodes::LOGICAL_ERROR); + if ((unlikely(pack->dataFlushable()))) + throw Exception("Saved pack is data flushable", ErrorCodes::LOGICAL_ERROR); bool small_pack = !pack->isDeleteRange() && pack->rows < context.delta_small_pack_rows; bool schema_ok = task.to_compact.empty() || pack->schema == task.to_compact.back()->schema; @@ -118,10 +118,13 @@ bool DeltaValueSpace::compact(DMContext & context) if (unlikely(pack->isDeleteRange())) throw Exception("Unexpectedly selected a delete range to compact", ErrorCodes::LOGICAL_ERROR); + // We ensure schema of all packs are the same Block block = pack->isCached() ? readPackFromCache(pack) : readPackFromDisk(pack, reader); size_t block_rows = block.rows(); for (size_t i = 0; i < schema.columns(); ++i) + { compact_columns[i]->insertRangeFrom(*block.getByPosition(i).column, 0, block_rows); + } wbs.removed_log.delPage(pack->data_page); } diff --git a/dbms/src/Storages/DeltaMerge/Delta/FlushDelta.cpp b/dbms/src/Storages/DeltaMerge/Delta/FlushDelta.cpp index 6d87e8331bd..e501af2712e 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/FlushDelta.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/FlushDelta.cpp @@ -13,8 +13,9 @@ struct FlushPackTask { FlushPackTask(const PackPtr & pack_) : pack(pack_) {} - PackPtr pack; - PageId data_page = 0; + ConstPackPtr pack; + + PageId data_page = 0; }; using FlushPackTasks = std::vector; @@ -53,7 +54,7 @@ bool DeltaValueSpace::flush(DMContext & context) auto & task = tasks.emplace_back(pack); // We only write the pack's data if it is not a delete range, and it's data haven't been saved. // Otherwise, simply save it's metadata is enough. - if (pack->isMutable()) + if (pack->dataFlushable()) { if (unlikely(!pack->cache)) throw Exception("Mutable pack does not have cache", ErrorCodes::LOGICAL_ERROR); @@ -64,13 +65,13 @@ bool DeltaValueSpace::flush(DMContext & context) } total_rows += pack->rows; total_deletes += pack->isDeleteRange(); + + // Stop other threads appending to this pack. + pack->appendable = false; } if (unlikely(flush_rows != unsaved_rows || flush_deletes != unsaved_deletes || total_rows != rows || total_deletes != deletes)) throw Exception("Rows and deletes check failed", ErrorCodes::LOGICAL_ERROR); - - // Must remove the last_cache, so that later append operations won't append to last pack which we are flushing. - last_cache = {}; } // No update, return successfully. @@ -143,12 +144,6 @@ bool DeltaValueSpace::flush(DMContext & context) // If it's data have been updated, use the new pages info. if (task.data_page != 0) shadow->data_page = task.data_page; - if (task.pack->rows >= context.delta_small_pack_rows) - { - // This pack is too large to use cache. - task.pack->cache = {}; - task.pack->cache_offset = 0; - } packs_copy.push_back(shadow); } @@ -188,6 +183,16 @@ bool DeltaValueSpace::flush(DMContext & context) /// Commit updates in memory. packs.swap(packs_copy); + for (auto & pack : packs) + { + if (pack->cache && pack->data_page != 0 && pack->rows >= context.delta_small_pack_rows) + { + // This pack is too large to use cache. + pack->cache = {}; + pack->cache_offset = 0; + } + } + unsaved_rows -= flush_rows; unsaved_deletes -= flush_deletes; diff --git a/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp b/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp index 708aa63d3d4..20082392102 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp @@ -2,8 +2,9 @@ #include #include #include +#include #include - +#include namespace DB::DM { @@ -63,8 +64,9 @@ inline void serializePack(const Pack & pack, const BlockPtr & schema, WriteBuffe inline PackPtr deserializePack(ReadBuffer & buf) { - auto pack = std::make_shared(); - pack->saved = true; // Must be true, otherwise it should not be here. + auto pack = std::make_shared(); + pack->saved = true; // Must be true, otherwise it should not be here. + pack->appendable = false; // Must be false, otherwise it should not be here. readIntBinary(pack->rows, buf); readIntBinary(pack->bytes, buf); readPODBinary(pack->delete_range, buf); @@ -173,26 +175,46 @@ Block readPackFromCache(const PackPtr & pack) Columns readPackFromCache(const PackPtr & pack, const ColumnDefines & column_defines, size_t col_start, size_t col_end) { + if (unlikely(!(pack->cache))) + { + String msg = " Not a cache pack: " + pack->toString(); + LOG_ERROR(&Logger::get(__FUNCTION__), msg); + throw Exception(msg); + } + // TODO: should be able to use cache data directly, without copy. std::scoped_lock lock(pack->cache->mutex); - auto & cache_block = pack->cache->block; + const auto & cache_block = pack->cache->block; + if constexpr (0) + { + if (pack->schema == nullptr || !checkSchema(cache_block, *pack->schema)) + { + const String pack_schema_str = pack->schema ? pack->schema->dumpStructure() : "(none)"; + const String cache_schema_str = cache_block.dumpStructure(); + throw Exception("Pack[" + pack->toString() + "] schema not match its cache_block! pack: " + pack_schema_str + + ", cache: " + cache_schema_str, + ErrorCodes::LOGICAL_ERROR); + } + } Columns columns; for (size_t i = col_start; i < col_end; ++i) { - auto & col = column_defines[i]; - auto it = pack->colid_to_offset.find(col.id); - if (it == pack->colid_to_offset.end()) + const auto & cd = column_defines[i]; + if (auto it = pack->colid_to_offset.find(cd.id); it != pack->colid_to_offset.end()) { - // TODO: support DDL. - throw Exception("Cannot find column with id" + DB::toString(col.id)); + auto col_offset = it->second; + // Copy data from cache + auto [type, col_data] = pack->getDataTypeAndEmptyColumn(cd.id); + col_data->insertRangeFrom(*cache_block.getByPosition(col_offset).column, pack->cache_offset, pack->rows); + // Cast if need + auto col_converted = convertColumnByColumnDefineIfNeed(type, std::move(col_data), cd); + columns.push_back(std::move(col_converted)); } else { - auto col_offset = it->second; - auto col_data = col.type->createColumn(); - col_data->insertRangeFrom(*cache_block.getByPosition(col_offset).column, pack->cache_offset, pack->rows); - columns.push_back(std::move(col_data)); + ColumnPtr column = createColumnWithDefaultValue(cd, pack->rows - pack->cache_offset); + columns.emplace_back(std::move(column)); } } return columns; @@ -232,37 +254,47 @@ Columns readPackFromDisk(const PackPtr & pack, // size_t col_start, size_t col_end) { + const size_t num_columns_read = col_end - col_start; + + Columns columns(num_columns_read); // allocate empty columns + PageReadFields fields; fields.first = pack->data_page; for (size_t index = col_start; index < col_end; ++index) { - auto col_id = column_defines[index].id; - auto it = pack->colid_to_offset.find(col_id); - if (it == pack->colid_to_offset.end()) - // TODO: support DDL. - throw Exception("Cannot find column with id" + DB::toString(col_id)); - else + const auto & cd = column_defines[index]; + if (auto it = pack->colid_to_offset.find(cd.id); it != pack->colid_to_offset.end()) { auto col_index = it->second; fields.second.push_back(col_index); } + else + { + // New column after ddl is not exist in this pack, fill with default value + columns[index - col_start] = createColumnWithDefaultValue(cd, pack->rows); + } } auto page_map = page_reader.read({fields}); Page page = page_map[pack->data_page]; - - Columns columns; for (size_t index = col_start; index < col_end; ++index) { + const size_t index_in_read_columns = index - col_start; + if (columns[index_in_read_columns] != nullptr) + { + // the column is fill with default values. + continue; + } auto col_id = column_defines[index].id; auto col_index = pack->colid_to_offset[col_id]; auto data_buf = page.getFieldData(col_index); - auto & cd = column_defines[index]; - auto col = cd.type->createColumn(); - deserializeColumn(*col, cd.type, data_buf, pack->rows); + const auto & cd = column_defines[index]; + // Deserialize column by pack's schema + auto [type, col_data] = pack->getDataTypeAndEmptyColumn(cd.id); + deserializeColumn(*col_data, type, data_buf, pack->rows); - columns.push_back(std::move(col)); + columns[index_in_read_columns] = convertColumnByColumnDefineIfNeed(type, std::move(col_data), cd); } return columns; diff --git a/dbms/src/Storages/DeltaMerge/Delta/Pack.h b/dbms/src/Storages/DeltaMerge/Delta/Pack.h index 446dc2addc3..75713652267 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/Pack.h +++ b/dbms/src/Storages/DeltaMerge/Delta/Pack.h @@ -23,10 +23,17 @@ void serializeColumn(MemoryWriteBuffer & buf, const IColumn & column, const Data void serializeSavedPacks(WriteBuffer & buf, const Packs & packs); Packs deserializePacks(ReadBuffer & buf); -String packsToString(const Packs & packs); -Block readPackFromCache(const PackPtr & pack); +// Debugging string +String packsToString(const Packs & packs); + +// Read a block from cache / disk according to `pack->schema` +Block readPackFromCache(const PackPtr & pack); +Block readPackFromDisk(const PackPtr & pack, const PageReader & page_reader); + +// Read a block of columns in `column_defines` from cache / disk, +// if `pack->schema` is not match with `column_defines`, take good care of +// ddl cast Columns readPackFromCache(const PackPtr & pack, const ColumnDefines & column_defines, size_t col_start, size_t col_end); -Block readPackFromDisk(const PackPtr & pack, const PageReader & page_reader); Columns readPackFromDisk(const PackPtr & pack, // const PageReader & page_reader, const ColumnDefines & column_defines, diff --git a/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp b/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp index e41f956c8e9..4862df37f60 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB::DM { @@ -52,7 +53,7 @@ SnapshotPtr DeltaValueSpace::createSnapshot(const DMContext & context, bool is_u { if (!is_update || pack->isSaved()) { - auto pack_copy = pack->isMutable() ? std::make_shared(*pack) : pack; + auto pack_copy = pack->isAppendable() ? std::make_shared(*pack) : pack; snap->packs.push_back(std::move(pack_copy)); check_rows += pack->rows; @@ -169,15 +170,21 @@ std::pair findPack(const Packs & packs, size_t rows_offset, size const Columns & DeltaValueSpace::Snapshot::getColumnsOfPack(size_t pack_index, size_t col_num) { + // If some columns is already read in this snapshot, we can reuse `packs_data` auto & columns = packs_data[pack_index]; if (columns.size() < col_num) { size_t col_start = columns.size(); size_t col_end = col_num; - auto read_columns = packs[pack_index]->isCached() - ? readPackFromCache(packs[pack_index], column_defines, col_start, col_end) - : readPackFromDisk(packs[pack_index], storage_snap->log_reader, column_defines, col_start, col_end); + auto & pack = packs[pack_index]; + Columns read_columns; + if (pack->isCached()) + read_columns = readPackFromCache(packs[pack_index], column_defines, col_start, col_end); + else if (pack->data_page != 0) + read_columns = readPackFromDisk(packs[pack_index], storage_snap->log_reader, column_defines, col_start, col_end); + else + throw Exception("Pack is in illegal status: " + pack->toString(), ErrorCodes::LOGICAL_ERROR); columns.insert(columns.end(), read_columns.begin(), read_columns.end()); } @@ -206,8 +213,9 @@ size_t DeltaValueSpace::Snapshot::read(const HandleRange & range, MutableColumns if (rows_start_in_pack == rows_end_in_pack) continue; + // TODO: this get the full columns of pack, which may cause unnecessary copying auto & columns = getColumnsOfPack(pack_index, output_columns.size()); - auto & handle_col_data = toColumnVectorData(columns[0]); + auto & handle_col_data = toColumnVectorData(columns[0]); // TODO: Magic number of fixed position of pk if (rows_in_pack_limit == 1) { if (range.check(handle_col_data[rows_start_in_pack])) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMerge.h b/dbms/src/Storages/DeltaMerge/DeltaMerge.h index 3c72d67a70e..f7cdc9fe969 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMerge.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMerge.h @@ -81,6 +81,7 @@ class DeltaMergeBlockInputStream final : public SkippableBlockInputStream, Alloc size_t num_read = 0; Handle last_handle = N_INF_HANDLE; + UInt64 last_version = 0; size_t last_handle_pos = 0; size_t last_handle_read_num = 0; @@ -208,19 +209,30 @@ class DeltaMergeBlockInputStream final : public SkippableBlockInputStream, Alloc { if constexpr (DM_RUN_CHECK) { + // In some cases like Segment::getSplitPointSlow, only handle column in block. + if (block.columns() < 2 // + || block.getByPosition(0).column_id != EXTRA_HANDLE_COLUMN_ID // + || block.getByPosition(1).column_id != VERSION_COLUMN_ID) + return; + ++num_read; - auto & handle_column = toColumnVectorData(block.getByPosition(0).column); + auto & handle_column = toColumnVectorData(block.getByPosition(0).column); + auto & version_column = toColumnVectorData(block.getByPosition(1).column); for (size_t i = 0; i < handle_column.size(); ++i) { - if (handle_column[i] < last_handle) + auto handle = handle_column[i]; + auto version = version_column[i]; + if (handle < last_handle || (handle == last_handle && version < last_version)) { - throw Exception("DeltaMerge return wrong result, current handle [" + DB::toString(handle_column[i]) + "]@read[" - + DB::toString(num_read) + "]@pos[" + DB::toString(i) + "] is expected >= last handle [" - + DB::toString(last_handle) + "]@read[" + DB::toString(last_handle_read_num) + "]@pos[" + throw Exception("DeltaMerge return wrong result, current handle[" + DB::toString(handle) + "]version[" + + DB::toString(version) + "]@read[" + DB::toString(num_read) + "]@pos[" + DB::toString(i) + + "] is expected >= last_handle[" + DB::toString(last_handle) + "]last_version[" + + DB::toString(last_version) + "]@read[" + DB::toString(last_handle_read_num) + "]@pos[" + DB::toString(last_handle_pos) + "]"); } - last_handle = handle_column[i]; + last_handle = handle; + last_version = version; last_handle_pos = i; last_handle_read_num = num_read; } diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h b/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h index 66769a7b52d..e96ff5d7e68 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h @@ -152,11 +152,16 @@ inline void addColumnToBlock(Block & block, block.insert(std::move(column)); } -inline Block toEmptyBlock(const ColumnDefines & columns) +/// Generate a block from column_defines +inline Block toEmptyBlock(const ColumnDefines & column_defines) { Block block; - for (auto & c : columns) + for (const auto & c : column_defines) + { + /// Usually we use this function to get a header block, + /// maybe columns of all nullptr in this block is enough? addColumnToBlock(block, c.id, c.name, c.type, c.type->createColumn(), c.default_value); + } return block; } @@ -194,7 +199,7 @@ inline bool checkSchema(const Block & a, const Block & b) for (size_t i = 0; i < a.columns(); ++i) { auto & ca = a.getByPosition(i); - auto & cb = a.getByPosition(i); + auto & cb = b.getByPosition(i); bool col_ok = ca.column_id == cb.column_id; bool name_ok = ca.name == cb.name; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index fb578e74850..8d4cdc41330 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -71,6 +71,27 @@ DeltaMergeStore::BackgroundTask DeltaMergeStore::MergeDeltaTaskPool::nextTask(Lo // DeltaMergeStore // ================================================ +namespace +{ +// Actually we will always store a column of `_tidb_rowid`, no matter it +// exist in `table_columns` or not. +ColumnDefinesPtr getStoreColumns(const ColumnDefines & table_columns) +{ + auto columns = std::make_shared(); + // First three columns are always _tidb_rowid, _INTERNAL_VERSION, _INTERNAL_DELMARK + columns->emplace_back(getExtraHandleColumnDefine()); + columns->emplace_back(getVersionColumnDefine()); + columns->emplace_back(getTagColumnDefine()); + // Add other columns + for (const auto & col : table_columns) + { + if (col.name != EXTRA_HANDLE_COLUMN_NAME && col.name != VERSION_COLUMN_NAME && col.name != TAG_COLUMN_NAME) + columns->emplace_back(col); + } + return columns; +} +} // namespace + DeltaMergeStore::DeltaMergeStore(Context & db_context, const String & path_, const String & db_name_, @@ -99,20 +120,14 @@ DeltaMergeStore::DeltaMergeStore(Context & db_context, original_table_columns.emplace_back(original_table_handle_define); original_table_columns.emplace_back(getVersionColumnDefine()); original_table_columns.emplace_back(getTagColumnDefine()); - - store_columns = std::make_shared(); - store_columns->emplace_back(getExtraHandleColumnDefine()); - store_columns->emplace_back(getVersionColumnDefine()); - store_columns->emplace_back(getTagColumnDefine()); - - for (auto & col : columns) + for (const auto & col : columns) { if (col.name != original_table_handle_define.name && col.name != VERSION_COLUMN_NAME && col.name != TAG_COLUMN_NAME) original_table_columns.emplace_back(col); - if (col.name != EXTRA_HANDLE_COLUMN_NAME && col.name != VERSION_COLUMN_NAME && col.name != TAG_COLUMN_NAME) - store_columns->emplace_back(col); } - original_header = toEmptyBlock(original_table_columns); + + store_columns = getStoreColumns(original_table_columns); + auto dm_context = newDMContext(db_context, db_context.getSettingsRef()); @@ -192,12 +207,26 @@ DeltaMergeStore::~DeltaMergeStore() { LOG_INFO(log, "Release DeltaMerge Store start [" << db_name << "." << table_name << "]"); - background_pool.removeTask(gc_handle); - background_pool.removeTask(background_task_handle); + shutdown(); LOG_INFO(log, "Release DeltaMerge Store end [" << db_name << "." << table_name << "]"); } +void DeltaMergeStore::shutdown() +{ + bool v = false; + if (!shutdown_called.compare_exchange_strong(v, true)) + return; + + LOG_DEBUG(log, "Shutdown DeltaMerge Store start [" << db_name << "." << table_name << "]"); + background_pool.removeTask(gc_handle); + gc_handle = nullptr; + + background_pool.removeTask(background_task_handle); + background_task_handle = nullptr; + LOG_DEBUG(log, "Shutdown DeltaMerge Store start [" << db_name << "." << table_name << "]"); +} + DMContextPtr DeltaMergeStore::newDMContext(const Context & db_context, const DB::Settings & db_settings) { std::shared_lock lock(read_write_mutex); @@ -211,7 +240,7 @@ DMContextPtr DeltaMergeStore::newDMContext(const Context & db_context, const DB: storage_pool, hash_salt, store_columns, - /* min_version */ 0, + latest_gc_safe_point, settings.not_compress_columns, db_settings); return DMContextPtr(ctx); @@ -527,7 +556,8 @@ void DeltaMergeStore::compact(const Context & db_context, const HandleRange & ra BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, const DB::Settings & db_settings, const ColumnDefines & columns_to_read, - size_t num_streams) + size_t num_streams, + const SegmentIdSet & read_segments) { SegmentReadTasks tasks; @@ -539,7 +569,13 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, for (const auto & [handle, segment] : segments) { (void)handle; - tasks.push(std::make_shared(segment, segment->createSnapshot(*dm_context), HandleRanges{segment->getRange()})); + if (read_segments.empty() || read_segments.count(segment->segmentId())) + { + auto segment_snap = segment->createSnapshot(*dm_context); + if (unlikely(!segment_snap)) + throw Exception("Failed to get segment snap", ErrorCodes::LOGICAL_ERROR); + tasks.push(std::make_shared(segment, segment_snap, HandleRanges{segment->getRange()})); + } } } @@ -574,7 +610,8 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, size_t num_streams, UInt64 max_version, const RSOperatorPtr & filter, - size_t expected_block_size) + size_t expected_block_size, + const SegmentIdSet & read_segments) { LOG_DEBUG(log, "Read with " << sorted_ranges.size() << " ranges"); @@ -599,12 +636,15 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, { auto & req_range = *range_it; auto & seg_range = seg_it->second->getRange(); - if (req_range.intersect(seg_range)) + if (req_range.intersect(seg_range) && (read_segments.empty() || read_segments.count(seg_it->second->segmentId()))) { if (tasks.empty() || tasks.back()->segment != seg_it->second) { - auto segment = seg_it->second; - tasks.push(std::make_shared(segment, segment->createSnapshot(*dm_context))); + auto segment = seg_it->second; + auto segment_snap = segment->createSnapshot(*dm_context); + if (unlikely(!segment_snap)) + throw Exception("Failed to get segment snap", ErrorCodes::LOGICAL_ERROR); + tasks.push(std::make_shared(segment, segment_snap)); } tasks.back()->addRange(req_range); @@ -773,6 +813,9 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto try_add_background_task = [&](const BackgroundTask & task) { if (background_tasks.length() <= std::max(id_to_segment.size() * 2, background_pool.getNumberOfThreads() * 5)) { + if (shutdown_called.load(std::memory_order_relaxed)) + return; + // Prevent too many tasks. background_tasks.addTask(task, thread_type, log); background_task_handle->wake(); @@ -934,6 +977,10 @@ bool DeltaMergeStore::handleBackgroundTask() /* ignore_cache= */ false, global_context.getSettingsRef().safe_point_update_interval_seconds); + LOG_DEBUG(log, "Task" << toString(task.type) << " GC safe point: " << safe_point); + + // Foreground task don't get GC safe point from remote, but we better make it as up to date as possible. + latest_gc_safe_point = safe_point; task.dm_context->min_version = safe_point; } @@ -992,7 +1039,7 @@ SegmentPair DeltaMergeStore::segmentSplit(DMContext & dm_context, const SegmentP { LOG_DEBUG(log, "Split segment " << segment->info()); - SegmentSnapshot segment_snap; + SegmentSnapshotPtr segment_snap; { std::shared_lock lock(read_write_mutex); @@ -1069,8 +1116,8 @@ void DeltaMergeStore::segmentMerge(DMContext & dm_context, const SegmentPtr & le { LOG_DEBUG(log, "Merge Segment [" << left->info() << "] and [" << right->info() << "]"); - SegmentSnapshot left_snap; - SegmentSnapshot right_snap; + SegmentSnapshotPtr left_snap; + SegmentSnapshotPtr right_snap; { std::shared_lock lock(read_write_mutex); @@ -1151,7 +1198,7 @@ SegmentPtr DeltaMergeStore::segmentMergeDelta(DMContext & dm_context, const Segm { LOG_DEBUG(log, (is_foreground ? "Foreground" : "Background") << " merge delta, segment [" << segment->segmentId() << "]"); - SegmentSnapshot segment_snap; + SegmentSnapshotPtr segment_snap; { std::shared_lock lock(read_write_mutex); @@ -1265,24 +1312,28 @@ void DeltaMergeStore::check(const Context & /*db_context*/) } +Block DeltaMergeStore::getHeader() const +{ + return toEmptyBlock(original_table_columns); +} + void DeltaMergeStore::applyAlters(const AlterCommands & commands, const OptionTableInfoConstRef table_info, ColumnID & max_column_id_used, - const Context & /*context*/) + const Context & /* context */) { std::unique_lock lock(read_write_mutex); - ColumnDefines original_table_columns_copy(original_table_columns.begin(), original_table_columns.end()); - auto store_columns_copy = std::make_shared(store_columns->begin(), store_columns->end()); - + ColumnDefines new_original_table_columns(original_table_columns.begin(), original_table_columns.end()); for (const auto & command : commands) { - applyAlter(original_table_columns_copy, command, table_info, max_column_id_used); - applyAlter(*store_columns_copy, command, table_info, max_column_id_used); + applyAlter(new_original_table_columns, command, table_info, max_column_id_used); } - original_table_columns.swap(original_table_columns); - store_columns.swap(store_columns_copy); + auto new_store_columns = getStoreColumns(new_original_table_columns); + + original_table_columns.swap(new_original_table_columns); + store_columns.swap(new_store_columns); } @@ -1411,5 +1462,40 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() return stat; } + +SegmentStats DeltaMergeStore::getSegmentStats() +{ + std::shared_lock lock(read_write_mutex); + + SegmentStats stats; + for (const auto & [handle, segment] : segments) + { + (void)handle; + + SegmentStat stat; + auto & delta = segment->getDelta(); + auto & stable = segment->getStable(); + + stat.segment_id = segment->segmentId(); + stat.range = segment->getRange(); + + stat.rows = segment->getEstimatedRows(); + stat.size = delta->getBytes() + stable->getBytes(); + stat.delete_ranges = delta->getDeletes(); + + stat.delta_pack_count = delta->getPackCount(); + stat.stable_pack_count = stable->getPacks(); + + stat.avg_delta_pack_rows = (Float64)delta->getRows() / stat.delta_pack_count; + stat.avg_stable_pack_rows = (Float64)stable->getRows() / stat.stable_pack_count; + + stat.delta_rate = (Float64)delta->getRows() / stat.rows; + stat.delta_cache_size = delta->getTotalCacheBytes(); + + stats.push_back(stat); + } + return stats; +} + } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 928bb636810..a56bca76cef 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -5,11 +5,11 @@ #include #include #include +#include #include #include #include #include -#include #include @@ -26,9 +26,30 @@ using RSOperatorPtr = std::shared_ptr; struct DMContext; using DMContextPtr = std::shared_ptr; using NotCompress = std::unordered_set; +using SegmentIdSet = std::unordered_set; static const PageId DELTA_MERGE_FIRST_SEGMENT_ID = 1; +struct SegmentStat +{ + UInt64 segment_id; + HandleRange range; + + UInt64 rows = 0; + UInt64 size = 0; + UInt64 delete_ranges = 0; + + UInt64 delta_pack_count = 0; + UInt64 stable_pack_count = 0; + + Float64 avg_delta_pack_rows = 0; + Float64 avg_stable_pack_rows = 0; + + Float64 delta_rate = 0; + UInt64 delta_cache_size = 0; +}; +using SegmentStats = std::vector; + struct DeltaMergeStoreStat { UInt64 segment_count = 0; @@ -201,12 +222,21 @@ class DeltaMergeStore : private boost::noncopyable const Settings & settings_); ~DeltaMergeStore(); + const String & getDatabaseName() const { return db_name; } + const String & getTableName() const { return table_name; } + + // Stop all background tasks. + void shutdown(); + void write(const Context & db_context, const DB::Settings & db_settings, const Block & block); void deleteRange(const Context & db_context, const DB::Settings & db_settings, const HandleRange & delete_range); - BlockInputStreams - readRaw(const Context & db_context, const DB::Settings & db_settings, const ColumnDefines & column_defines, size_t num_streams); + BlockInputStreams readRaw(const Context & db_context, + const DB::Settings & db_settings, + const ColumnDefines & column_defines, + size_t num_streams, + const SegmentIdSet & read_segments = {}); /// ranges should be sorted and merged already. BlockInputStreams read(const Context & db_context, @@ -216,7 +246,8 @@ class DeltaMergeStore : private boost::noncopyable size_t num_streams, UInt64 max_version, const RSOperatorPtr & filter, - size_t expected_block_size = DEFAULT_BLOCK_SIZE); + size_t expected_block_size = DEFAULT_BLOCK_SIZE, + const SegmentIdSet & read_segments = {}); /// Force flush all data to disk. void flushCache(const Context & context, const HandleRange & range = HandleRange::newAll()) @@ -238,13 +269,14 @@ class DeltaMergeStore : private boost::noncopyable const ColumnDefines & getTableColumns() const { return original_table_columns; } const ColumnDefine & getHandle() const { return original_table_handle_define; } - const Block & getHeader() const { return original_header; } + Block getHeader() const; const Settings & getSettings() const { return settings; } DataTypePtr getPKDataType() const { return original_table_handle_define.type; } SortDescription getPrimarySortDescription() const; void check(const Context & db_context); DeltaMergeStoreStat getStat(); + SegmentStats getSegmentStats(); private: DMContextPtr newDMContext(const Context & db_context, const DB::Settings & db_settings); @@ -280,11 +312,12 @@ class DeltaMergeStore : private boost::noncopyable ColumnDefines original_table_columns; const ColumnDefine original_table_handle_define; - Block original_header; // The columns we actually store. ColumnDefinesPtr store_columns; + std::atomic shutdown_called{false}; + BackgroundProcessingPool & background_pool; BackgroundProcessingPool::TaskHandle gc_handle; BackgroundProcessingPool::TaskHandle background_task_handle; @@ -299,6 +332,8 @@ class DeltaMergeStore : private boost::noncopyable MergeDeltaTaskPool background_tasks; + DB::Timestamp latest_gc_safe_point = 0; + // Synchronize between write threads and read threads. std::shared_mutex read_write_mutex; diff --git a/dbms/src/Storages/DeltaMerge/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/DeltaValueSpace.cpp index 3f93f958a54..4893ef8ed03 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaValueSpace.cpp @@ -19,7 +19,6 @@ namespace DB { namespace DM { - const UInt64 DeltaValueSpace::CURRENT_VERSION = 1; using BlockPtr = DeltaValueSpace::BlockPtr; @@ -157,7 +156,9 @@ Packs DeltaValueSpace::checkHeadAndCloneTail(DMContext & context, wbs.log.putRefPage(new_page_id, pack->data_page); new_pack->data_page = new_page_id; } - + // No matter or what, don't append to packs which cloned from old packs again. + // Because they could shared the same cache. And the cache can NOT be inserted from different packs in different delta. + new_pack->appendable = false; tail_clone.push_back(new_pack); } } @@ -244,6 +245,7 @@ PackPtr DeltaValueSpace::writePack(DMContext & context, const Block & block, siz pack->bytes = block.bytes() * ((double)limit / block.rows()); pack->data_page = writePackData(context, block, offset, limit, wbs); pack->setSchema(std::make_shared(block.cloneEmpty())); + pack->appendable = false; return pack; } @@ -258,6 +260,9 @@ bool DeltaValueSpace::appendToDisk(DMContext & /*context*/, const PackPtr & pack if (last_schema && checkSchema(*pack->schema, *last_schema)) pack->schema = last_schema; + if (!packs.empty()) + packs.back()->appendable = false; + packs.push_back(pack); rows += pack->rows; @@ -278,33 +283,49 @@ bool DeltaValueSpace::appendToCache(DMContext & context, const Block & block, si // And, if last pack is mutable (haven't been saved to disk yet), we will merge the newly block into last pack. // Otherwise, create a new cache block and write into it. - PackPtr mutable_pack; - CachePtr cache; - if (!packs.empty() && last_cache) + PackPtr mutable_pack{}; + CachePtr cache{}; + if (!packs.empty()) { - std::scoped_lock cache_lock(last_cache->mutex); - - auto & last_pack = packs.back(); - bool is_overflow = last_cache->block.rows() >= context.delta_cache_limit_rows; - bool is_same_schema = checkSchema(block, last_cache->block); - - if (!is_overflow && is_same_schema) + auto & last_pack = packs.back(); + if (last_pack->isAppendable()) { - // The last cache block is available - cache = last_cache; - if (last_pack->isMutable()) + if constexpr (DM_RUN_CHECK) + { + if (unlikely(!checkSchema(*last_pack->schema, last_pack->cache->block))) + throw Exception("Mutable pack's structure of schema and block are different: " + last_pack->toString()); + } + + bool is_overflow = last_pack->cache->block.rows() >= context.delta_cache_limit_rows; + bool is_same_schema = checkSchema(block, last_pack->cache->block); + if (!is_overflow && is_same_schema) { - if (unlikely(last_pack->cache != last_cache)) - throw Exception("Last mutable pack's cache is not equal to last cache", ErrorCodes::LOGICAL_ERROR); + // The last cache block is available + cache = last_pack->cache; mutable_pack = last_pack; } + else + { + last_pack->appendable = false; + } } } if (!cache) { - cache = std::make_shared(block); - last_cache = cache; + cache = std::make_shared(block); + } + + if constexpr (0) + { + if (unlikely(!checkSchema(cache->block, block))) + { + const String block_schema_str = block.dumpStructure(); + const String cache_schema_str = cache->block.dumpStructure(); + throw Exception("Try to append block(rows:" + DB::toString(block.rows()) + + ") to a cache but schema not match! block: " + block_schema_str + ", cache: " + cache_schema_str, + ErrorCodes::LOGICAL_ERROR); + } } size_t cache_offset; @@ -361,6 +382,7 @@ bool DeltaValueSpace::appendDeleteRange(DMContext & /*context*/, const HandleRan auto pack = std::make_shared(); pack->delete_range = delete_range; + pack->appendable = false; packs.push_back(pack); ++deletes; diff --git a/dbms/src/Storages/DeltaMerge/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/DeltaValueSpace.h index 49168399a0f..7c903dbba26 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/DeltaValueSpace.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -17,8 +18,6 @@ class DeltaValueSpace; using DeltaValueSpacePtr = std::shared_ptr; struct WriteBatches; class StoragePool; -struct StorageSnapshot; -using StorageSnapshotPtr = std::shared_ptr; struct DMContext; struct BlockOrDelete @@ -65,11 +64,15 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr // Already persisted to disk or not. bool saved = false; + // Can be appended into new rows or not. + bool appendable = true; bool isDeleteRange() const { return !delete_range.none(); } bool isCached() const { return !isDeleteRange() && (bool)cache; } - /// This pack is not a delete range, the data in it has not been saved to disk. - bool isMutable() const { return !isDeleteRange() && data_page == 0; } + /// Whether its column data can be flushed. + bool dataFlushable() const { return !isDeleteRange() && data_page == 0; } + /// This pack is the last one, and not a delete range, and can be appended into new rows. + bool isAppendable() const { return !isDeleteRange() && data_page == 0 && appendable && (bool)cache; } /// This pack's metadata has been saved to disk. bool isSaved() const { return saved; } void setSchema(const BlockPtr & schema_) @@ -80,20 +83,33 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr colid_to_offset.emplace(schema->getByPosition(i).column_id, i); } + std::pair getDataTypeAndEmptyColumn(ColId column_id) const + { + // Note that column_id must exist + auto index = colid_to_offset.at(column_id); + auto col_type = schema->getByPosition(index).type; + return { col_type, col_type->createColumn() }; + } + String toString() { - return "{rows:" + DB::toString(rows) // - + ",bytes:" + DB::toString(bytes) // - + ",has_schema:" + DB::toString((bool)schema) // - + ",delete_range:" + delete_range.toString() // - + ",data_page:" + DB::toString(data_page) // - + ",has_cache:" + DB::toString((bool)cache) // - + ",cache_offset:" + DB::toString(cache_offset) // - + ",saved:" + DB::toString(saved) + "}"; + String s = "{rows:" + DB::toString(rows) // + + ",bytes:" + DB::toString(bytes) // + + ",has_schema:" + DB::toString((bool)schema) // + + ",delete_range:" + delete_range.toString() // + + ",data_page:" + DB::toString(data_page) // + + ",has_cache:" + DB::toString((bool)cache) // + + ",cache_offset:" + DB::toString(cache_offset) // + + ",saved:" + DB::toString(saved) // + + ",appendable:" + DB::toString(appendable) // + + ",schema:" + (schema ? schema->dumpStructure() : "none") // + + ",cache_block:" + (cache ? cache->block.dumpStructure() : "none") + ")"; + return s; } }; - using PackPtr = std::shared_ptr; - using Packs = std::vector; + using PackPtr = std::shared_ptr; + using ConstPackPtr = std::shared_ptr; + using Packs = std::vector; struct Snapshot : public std::enable_shared_from_this, private boost::noncopyable { @@ -128,19 +144,25 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr } } - size_t getPackCount() { return packs.size(); } - size_t getRows() { return rows; } - size_t getDeletes() { return deletes; } + size_t getPackCount() const { return packs.size(); } + size_t getRows() const { return rows; } + size_t getDeletes() const { return deletes; } void prepare(const DMContext & context, const ColumnDefines & column_defines_); BlockInputStreamPtr prepareForStream(const DMContext & context, const ColumnDefines & column_defines_); const Columns & getColumnsOfPack(size_t pack_index, size_t col_num); - size_t read(const HandleRange & range, MutableColumns & output_columns, size_t offset, size_t limit); - Block read(size_t col_num, size_t offset, size_t limit); - Block read(size_t pack_index); + // Get blocks or delete_ranges of `ExtraHandleColumn` and `VersionColumn`. + // If there are continuous blocks, they will be squashed into one block. + // We use the result to update DeltaTree. BlockOrDeletes getMergeBlocks(size_t rows_begin, size_t deletes_begin, size_t rows_end, size_t deletes_end); + + Block read(size_t pack_index); + size_t read(const HandleRange & range, MutableColumns & output_columns, size_t offset, size_t limit); + + private: + Block read(size_t col_num, size_t offset, size_t limit); }; using SnapshotPtr = std::shared_ptr; @@ -168,8 +190,6 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr std::atomic last_try_merge_delta_rows = 0; std::atomic last_try_split_rows = 0; - CachePtr last_cache; - // Protects the operations in this instance. mutable std::mutex mutex; @@ -210,7 +230,7 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr throw Exception("Try to abandon a already abandoned DeltaValueSpace", ErrorCodes::LOGICAL_ERROR); } - bool hasAbandoned() { return abandoned.load(std::memory_order_relaxed); } + bool hasAbandoned() const { return abandoned.load(std::memory_order_relaxed); } /// Restore the metadata of this instance. /// Only called after reboot. @@ -275,6 +295,7 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr using Pack = DeltaValueSpace::Pack; using PackPtr = DeltaValueSpace::PackPtr; +using ConstPackPtr = DeltaValueSpace::ConstPackPtr; using Packs = DeltaValueSpace::Packs; using DeltaSnapshot = DeltaValueSpace::Snapshot; using DeltaSnapshotPtr = DeltaValueSpace::SnapshotPtr; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp index 2bb574b2e41..b396b300262 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp @@ -1,19 +1,26 @@ -#include -#include -#include - #include #include #include #include +#include #include #include +#include +#include + namespace DB { namespace DM { +static constexpr const char * NGC_FILE_NAME = "NGC"; + +String DMFile::ngcPath() const +{ + return path() + "/" + NGC_FILE_NAME; +} + DMFilePtr DMFile::create(UInt64 file_id, const String & parent_path) { Logger * log = &Logger::get("DMFile"); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h index 0fdde57f665..2f012c6288d 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h @@ -15,8 +15,6 @@ class DMFile; using DMFilePtr = std::shared_ptr; using DMFiles = std::vector; -static const String NGC_FILE_NAME = "NGC"; - class DMFile : private boost::noncopyable { public: @@ -65,26 +63,18 @@ class DMFile : private boost::noncopyable void enableGC(); void remove(); - UInt64 fileId() { return file_id; } - UInt64 refId() { return ref_id; } - String path() { return parent_path + (status == Status::READABLE ? "/dmf_" : "/.tmp.dmf_") + DB::toString(file_id); } - String metaPath() { return path() + "/meta.txt"; } - String packStatPath() { return path() + "/pack"; } + UInt64 fileId() const { return file_id; } + UInt64 refId() const { return ref_id; } + String path() const { return parent_path + (status == Status::READABLE ? "/dmf_" : "/.tmp.dmf_") + DB::toString(file_id); } + String metaPath() const { return path() + "/meta.txt"; } + String packStatPath() const { return path() + "/pack"; } // Do not gc me. - String ngcPath() { return path() + "/" + NGC_FILE_NAME; } - String colDataPath(const String & file_name_base) { return path() + "/" + file_name_base + ".dat"; } - String colIndexPath(const String & file_name_base) { return path() + "/" + file_name_base + ".idx"; } - String colMarkPath(const String & file_name_base) { return path() + "/" + file_name_base + ".mrk"; } + String ngcPath() const; + String colDataPath(const String & file_name_base) const { return path() + "/" + file_name_base + ".dat"; } + String colIndexPath(const String & file_name_base) const { return path() + "/" + file_name_base + ".idx"; } + String colMarkPath(const String & file_name_base) const { return path() + "/" + file_name_base + ".mrk"; } - const ColumnStat & getColumnStat(ColId col_id) - { - auto it = column_stats.find(col_id); - if (it == column_stats.end()) - throw Exception("Column [" + DB::toString(col_id) + "] not found in dm file [" + path() + "]"); - return it->second; - } - - size_t getRows() + size_t getRows() const { size_t rows = 0; for (auto & s : pack_stats) @@ -92,7 +82,7 @@ class DMFile : private boost::noncopyable return rows; } - size_t getBytes() + size_t getBytes() const { size_t bytes = 0; for (auto & s : pack_stats) @@ -100,11 +90,22 @@ class DMFile : private boost::noncopyable return bytes; } - size_t getPacks() { return pack_stats.size(); } - const PackStats & getPackStats() { return pack_stats; } - const PackStat & getPackStat(size_t pack_index) { return pack_stats[pack_index]; } - const ColumnStats & getColumnStats() { return column_stats; } - Status getStatus() { return status; } + size_t getPacks() const { return pack_stats.size(); } + const PackStats & getPackStats() const { return pack_stats; } + const PackStat & getPackStat(size_t pack_index) const { return pack_stats[pack_index]; } + + const ColumnStats & getColumnStats() const { return column_stats; } + const ColumnStat & getColumnStat(ColId col_id) const + { + if (auto it = column_stats.find(col_id); it != column_stats.end()) + { + return it->second; + } + throw Exception("Column [" + DB::toString(col_id) + "] not found in dm file [" + path() + "]"); + } + bool isColumnExist(ColId col_id) const { return column_stats.find(col_id) != column_stats.end(); } + + Status getStatus() const { return status; } static String getFileNameBase(ColId col_id, const IDataType::SubstreamPath & substream = {}) { diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h index 935b42934cb..826ebfde96c 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h @@ -151,4 +151,4 @@ class DMFilePackFilter }; } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index a0a14a79d05..f52955f2d62 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp @@ -1,8 +1,8 @@ -#include - #include #include +#include #include +#include #include namespace DB @@ -18,8 +18,8 @@ DMFileReader::Stream::Stream(DMFileReader & reader, // Logger * log) : avg_size_hint(reader.dmfile->getColumnStat(col_id).avg_size) { - String mark_path = reader.dmfile->colMarkPath(file_name_base); - String data_path = reader.dmfile->colDataPath(file_name_base); + const String mark_path = reader.dmfile->colMarkPath(file_name_base); + const String data_path = reader.dmfile->colDataPath(file_name_base); auto mark_load = [&]() -> MarksInCompressedFilePtr { auto res = std::make_shared(reader.dmfile->getPacks()); @@ -39,7 +39,7 @@ DMFileReader::Stream::Stream(DMFileReader & reader, // marks = mark_load(); size_t data_file_size = Poco::File(data_path).getSize(); - size_t packs = reader.dmfile->getPacks(); + size_t packs = reader.dmfile->getPacks(); size_t buffer_size = 0; size_t estimated_size = 0; for (size_t i = 0; i < packs;) @@ -125,19 +125,27 @@ DMFileReader::DMFileReader(bool enable_clean_read_, throw Exception("DMFile [" + DB::toString(dmfile->fileId()) + "] is expected to be in READABLE status, but: " + DMFile::statusString(dmfile->getStatus())); - for (auto & cd : read_columns) + for (const auto & cd : read_columns) { + // New inserted column, fill them with default value later + if (!dmfile->isColumnExist(cd.id)) + continue; + auto callback = [&](const IDataType::SubstreamPath & substream) { String stream_name = DMFile::getFileNameBase(cd.id, substream); - auto stream = std::make_unique(*this, // - cd.id, - stream_name, - aio_threshold, - max_read_buffer_size, - log); + auto stream = std::make_unique( // + *this, + cd.id, + stream_name, + aio_threshold, + max_read_buffer_size, + log); column_streams.emplace(stream_name, std::move(stream)); }; - cd.type->enumerateStreams(callback, {}); + + // Load stream according to DataType in disk + const auto data_type = dmfile->getColumnStat(cd.id).type; + data_type->enumerateStreams(callback, {}); } } @@ -174,7 +182,7 @@ Block DMFileReader::read() // Find max continuing rows we can read. size_t start_pack_id = next_pack_id; - auto & pack_stats = dmfile->getPackStats(); + auto & pack_stats = dmfile->getPackStats(); size_t read_rows = 0; size_t not_clean_rows = 0; @@ -188,7 +196,7 @@ Block DMFileReader::read() not_clean_rows += pack_stats[next_pack_id].not_clean; } - if (!read_rows) + if (read_rows == 0) return {}; Block res; @@ -232,28 +240,42 @@ Block DMFileReader::read() } else { - String stream_name = DMFile::getFileNameBase(cd.id); - auto & stream = column_streams.at(stream_name); - if (shouldSeek(start_pack_id) || skip_packs_by_column[i] > 0) + const String stream_name = DMFile::getFileNameBase(cd.id); + if (auto iter = column_streams.find(stream_name); iter != column_streams.end()) { - auto & mark = (*stream->marks)[start_pack_id]; - stream->buf->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + auto & stream = iter->second; + if (shouldSeek(start_pack_id) || skip_packs_by_column[i] > 0) + { + auto & mark = (*stream->marks)[start_pack_id]; + stream->buf->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + } + + auto data_type = dmfile->getColumnStat(cd.id).type; + auto column = data_type->createColumn(); + data_type->deserializeBinaryBulkWithMultipleStreams( // + *column, + [&](const IDataType::SubstreamPath & substream_path) { + String substream_name = DMFile::getFileNameBase(cd.id, substream_path); + auto & sub_stream = column_streams.at(substream_name); + return sub_stream->buf.get(); + }, + read_rows, + stream->avg_size_hint, + true, + {}); + IDataType::updateAvgValueSizeHint(*column, stream->avg_size_hint); + + // Cast column's data from DataType in disk to what we need now + auto converted_column = convertColumnByColumnDefineIfNeed(data_type, std::move(column), cd); + + res.insert(ColumnWithTypeAndName{std::move(converted_column), cd.type, cd.name, cd.id}); + } + else + { + // New column after ddl is not exist in this DMFile, fill with default value + ColumnPtr column = createColumnWithDefaultValue(cd, read_rows); + res.insert(ColumnWithTypeAndName{std::move(column), cd.type, cd.name, cd.id}); } - - auto column = cd.type->createColumn(); - cd.type->deserializeBinaryBulkWithMultipleStreams(*column, // - [&](const IDataType::SubstreamPath & substream) { - String name = DMFile::getFileNameBase(cd.id, substream); - auto & stream = column_streams.at(name); - return stream->buf.get(); - }, - read_rows, - stream->avg_size_hint, - true, - {}); - IDataType::updateAvgValueSizeHint(*column, stream->avg_size_hint); - - res.insert(ColumnWithTypeAndName{std::move(column), cd.type, cd.name, cd.id}); skip_packs_by_column[i] = 0; } diff --git a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp index 45849090036..ebb30826cd2 100644 --- a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp +++ b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -11,15 +12,21 @@ namespace DB namespace DM { +String astToDebugString(const IAST * const ast) +{ + std::stringstream ss; + ast->dumpTree(ss); + return ss.str(); +} + inline void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & define) { - std::function castDefaultValue; // for lazy bind - castDefaultValue = [&](Field value, DataTypePtr type) -> Field { + std::function castDefaultValue; // for lazy bind + castDefaultValue = [&](const Field & value, const DataTypePtr & type) -> Field { switch (type->getTypeId()) { case TypeIndex::Float32: - case TypeIndex::Float64: - { + case TypeIndex::Float64: { if (value.getType() == Field::Types::Float64) { Float64 res = applyVisitor(FieldVisitorConvertToNumber(), value); @@ -39,117 +46,148 @@ inline void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefi } else { - throw Exception("Unknown float number literal"); + throw Exception("Unknown float number literal: " + applyVisitor(FieldVisitorToString(), value) + + ", value type: " + value.getTypeName()); } } - case TypeIndex::FixedString: - { + case TypeIndex::String: + case TypeIndex::FixedString: { String res = get(value); return toField(res); } case TypeIndex::Int8: case TypeIndex::Int16: case TypeIndex::Int32: - case TypeIndex::Int64: - { + case TypeIndex::Int64: { Int64 res = applyVisitor(FieldVisitorConvertToNumber(), value); return toField(res); } case TypeIndex::UInt8: case TypeIndex::UInt16: case TypeIndex::UInt32: - case TypeIndex::UInt64: - { + case TypeIndex::UInt64: { UInt64 res = applyVisitor(FieldVisitorConvertToNumber(), value); return toField(res); } - case TypeIndex::DateTime: - { + case TypeIndex::DateTime: { auto date = safeGet(value); time_t time = 0; ReadBufferFromMemory buf(date.data(), date.size()); readDateTimeText(time, buf); return toField((Int64)time); } - case TypeIndex::Decimal32: - { - auto dec = std::dynamic_pointer_cast(type); - Int64 v = applyVisitor(FieldVisitorConvertToNumber(), value); - ScaleType scale = dec->getScale(); - return DecimalField(Decimal32(v), scale); + case TypeIndex::Decimal32: { + auto v = safeGet>(value); + return v; } - case TypeIndex::Decimal64: - { - auto dec = std::dynamic_pointer_cast(type); - Int64 v = applyVisitor(FieldVisitorConvertToNumber(), value); - ScaleType scale = dec->getScale(); - return DecimalField(Decimal64(v), scale); + case TypeIndex::Decimal64: { + auto v = safeGet>(value); + return v; } - case TypeIndex::Decimal128: - { - auto dec = std::dynamic_pointer_cast(type); - Int64 v = applyVisitor(FieldVisitorConvertToNumber(), value); - ScaleType scale = dec->getScale(); - return DecimalField(Decimal128(v), scale); + case TypeIndex::Decimal128: { + auto v = safeGet>(value); + return v; } - case TypeIndex::Decimal256: - { - auto dec = std::dynamic_pointer_cast(type); - Int64 v = applyVisitor(FieldVisitorConvertToNumber(), value); - ScaleType scale = dec->getScale(); - return DecimalField(Decimal256(v), scale); + case TypeIndex::Decimal256: { + auto v = safeGet>(value); + return v; } - case TypeIndex::Nullable: - { + case TypeIndex::Enum16: { + // According to `Storages/Transaction/TiDB.h` and MySQL 5.7 + // document(https://dev.mysql.com/doc/refman/5.7/en/enum.html), + // enum support 65,535 distinct value at most, so only Enum16 is supported here. + // Default value of Enum should be store as a Int64 Field (Storages/Transaction/Datum.cpp) + Int64 res = applyVisitor(FieldVisitorConvertToNumber(), value); + return toField(res); + } + case TypeIndex::MyDate: + case TypeIndex::MyDateTime: { + static_assert(std::is_same_v); + static_assert(std::is_same_v); + UInt64 res = applyVisitor(FieldVisitorConvertToNumber(), value); + return toField(res); + } + case TypeIndex::Nullable: { if (value.isNull()) return value; auto nullable = std::dynamic_pointer_cast(type); DataTypePtr nested_type = nullable->getNestedType(); - return castDefaultValue(value, nested_type); + return castDefaultValue(value, nested_type); // Recursive call on nested type } default: - throw Exception("Unsupported data type: " + type->getName()); + throw Exception("Unsupported to setColumnDefineDefaultValue with data type: " + type->getName() + + " value: " + applyVisitor(FieldVisitorToString(), value) + ", type: " + value.getTypeName()); } }; if (command.default_expression) { - // a cast function - // change column_define.default_value - - if (auto default_literal = typeid_cast(command.default_expression.get()); - default_literal && default_literal->value.getType() == Field::Types::String) - { - define.default_value = default_literal->value; - } - else if (auto default_cast_expr = typeid_cast(command.default_expression.get()); - default_cast_expr && default_cast_expr->name == "CAST" /* ParserCastExpression::name */) + try { - // eg. CAST('1.234' AS Float32); CAST(999 AS Int32) - if (default_cast_expr->arguments->children.size() != 2) + // a cast function + // change column_define.default_value + + if (auto default_literal = typeid_cast(command.default_expression.get()); + default_literal && default_literal->value.getType() == Field::Types::String) { - throw Exception("Unknown CAST expression in default expr", ErrorCodes::NOT_IMPLEMENTED); + define.default_value = default_literal->value; } - - auto default_literal_in_cast = typeid_cast(default_cast_expr->arguments->children[0].get()); - if (default_literal_in_cast) + else if (auto default_cast_expr = typeid_cast(command.default_expression.get()); + default_cast_expr && default_cast_expr->name == "CAST" /* ParserCastExpression::name */) { - Field default_value = castDefaultValue(default_literal_in_cast->value, define.type); - define.default_value = default_value; + // eg. CAST('1.234' AS Float32); CAST(999 AS Int32) + if (default_cast_expr->arguments->children.size() != 2) + { + throw Exception("Unknown CAST expression in default expr", ErrorCodes::NOT_IMPLEMENTED); + } + + auto default_literal_in_cast = typeid_cast(default_cast_expr->arguments->children[0].get()); + if (default_literal_in_cast) + { + Field default_value = castDefaultValue(default_literal_in_cast->value, define.type); + define.default_value = default_value; + } + else + { + throw Exception("Invalid CAST expression", ErrorCodes::BAD_ARGUMENTS); + } } else { - throw Exception("Invalid CAST expression", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Default value must be a string or CAST('...' AS WhatType)", ErrorCodes::BAD_ARGUMENTS); } } - else + catch (DB::Exception & e) + { + e.addMessage("(in setColumnDefineDefaultValue for default_expression:" + astToDebugString(command.default_expression.get()) + + ")"); + throw; + } + catch (const Poco::Exception & e) + { + DB::Exception ex(e); + ex.addMessage("(in setColumnDefineDefaultValue for default_expression:" + astToDebugString(command.default_expression.get()) + + ")"); + throw ex; + } + catch (std::exception & e) { - throw Exception("Default value must be a string or CAST('...' AS WhatType)", ErrorCodes::BAD_ARGUMENTS); + std::stringstream ss; + ss << "std::exception: " << e.what() + << " (in setColumnDefineDefaultValue for default_expression:" + astToDebugString(command.default_expression.get()) << ")"; + DB::Exception ex(ss.str(), ErrorCodes::LOGICAL_ERROR); + throw ex; } } } +inline void setColumnDefineDefaultValue(const OptionTableInfoConstRef table_info, ColumnDefine & define) +{ + const auto col_info = table_info->get().getColumnInfo(define.id); + define.default_value = col_info.defaultValueToField(); +} + void applyAlter(ColumnDefines & table_columns, const AlterCommand & command, const OptionTableInfoConstRef table_info, @@ -170,7 +208,14 @@ void applyAlter(ColumnDefines & table_columns, { exist_column = true; column_define.type = command.data_type; - setColumnDefineDefaultValue(command, column_define); + if (table_info) + { + setColumnDefineDefaultValue(table_info, column_define); + } + else + { + setColumnDefineDefaultValue(command, column_define); + } break; } } @@ -206,12 +251,13 @@ void applyAlter(ColumnDefines & table_columns, if (table_info) { define.id = table_info->get().getColumnID(command.column_name); + setColumnDefineDefaultValue(table_info, define); } else { define.id = max_column_id_used++; + setColumnDefineDefaultValue(command, define); } - setColumnDefineDefaultValue(command, define); table_columns.emplace_back(std::move(define)); } else if (command.type == AlterCommand::DROP_COLUMN) @@ -238,4 +284,4 @@ void applyAlter(ColumnDefines & table_columns, } } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index d32c6b0800e..3521a3ffdae 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -216,6 +216,8 @@ bool Segment::writeToDisk(DMContext & dm_context, const PackPtr & pack) bool Segment::writeToCache(DMContext & dm_context, const Block & block, size_t offset, size_t limit) { LOG_TRACE(log, "Segment [" << segment_id << "] write to cache rows: " << limit); + if (unlikely(limit == 0)) + return true; return delta->appendToCache(dm_context, block, offset, limit); } @@ -251,18 +253,21 @@ bool Segment::write(DMContext & dm_context, const HandleRange & delete_range) return delta->appendDeleteRange(dm_context, delete_range); } -SegmentSnapshot Segment::createSnapshot(const DMContext & dm_context, bool is_update) const +SegmentSnapshotPtr Segment::createSnapshot(const DMContext & dm_context, bool is_update) const { - return {delta->createSnapshot(dm_context, is_update), stable}; + auto delta_snap = delta->createSnapshot(dm_context, is_update); + if (!delta_snap) + return {}; + return std::make_shared(delta_snap, stable); } -BlockInputStreamPtr Segment::getInputStream(const DMContext & dm_context, - const ColumnDefines & columns_to_read, - SegmentSnapshot & segment_snap, - const HandleRanges & read_ranges, - const RSOperatorPtr & filter, - UInt64 max_version, - size_t expected_block_size) +BlockInputStreamPtr Segment::getInputStream(const DMContext & dm_context, + const ColumnDefines & columns_to_read, + const SegmentSnapshotPtr & segment_snap, + const HandleRanges & read_ranges, + const RSOperatorPtr & filter, + UInt64 max_version, + size_t expected_block_size) { LOG_TRACE(log, "Segment [" << segment_id << "] create InputStream"); @@ -276,15 +281,15 @@ BlockInputStreamPtr Segment::getInputStream(const DMContext & dm_context, } else if (dm_context.read_stable_only) { - stream = segment_snap.stable->getInputStream(dm_context, read_info.read_columns, read_range, filter, max_version, false); + stream = segment_snap->stable->getInputStream(dm_context, read_info.read_columns, read_range, filter, max_version, false); } - else if (segment_snap.delta->rows == 0 && segment_snap.delta->deletes == 0 // - && !hasColumn(columns_to_read, EXTRA_HANDLE_COLUMN_ID) // - && !hasColumn(columns_to_read, VERSION_COLUMN_ID) // + else if (segment_snap->delta->rows == 0 && segment_snap->delta->deletes == 0 // + && !hasColumn(columns_to_read, EXTRA_HANDLE_COLUMN_ID) // + && !hasColumn(columns_to_read, VERSION_COLUMN_ID) // && !hasColumn(columns_to_read, TAG_COLUMN_ID)) { // No delta, let's try some optimizations. - stream = segment_snap.stable->getInputStream(dm_context, read_info.read_columns, read_range, filter, max_version, true); + stream = segment_snap->stable->getInputStream(dm_context, read_info.read_columns, read_range, filter, max_version, true); } else { @@ -292,8 +297,8 @@ BlockInputStreamPtr Segment::getInputStream(const DMContext & dm_context, read_info.read_columns, read_range, filter, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), @@ -339,14 +344,16 @@ BlockInputStreamPtr Segment::getInputStream(const DMContext & dm_context, size_t expected_block_size) { - SegmentSnapshot segment_snap = createSnapshot(dm_context); + auto segment_snap = createSnapshot(dm_context); + if (!segment_snap) + return {}; return getInputStream(dm_context, columns_to_read, segment_snap, read_ranges, filter, max_version, expected_block_size); } -BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, - const ColumnDefines & columns_to_read, - SegmentSnapshot & segment_snap, - bool do_range_filter) +BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, + const ColumnDefines & columns_to_read, + const SegmentSnapshotPtr & segment_snap, + bool do_range_filter) { ColumnDefines new_columns_to_read; @@ -365,10 +372,10 @@ BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, } } - BlockInputStreamPtr delta_stream = segment_snap.delta->prepareForStream(dm_context, new_columns_to_read); + BlockInputStreamPtr delta_stream = segment_snap->delta->prepareForStream(dm_context, new_columns_to_read); BlockInputStreamPtr stable_stream - = segment_snap.stable->getInputStream(dm_context, new_columns_to_read, range, EMPTY_FILTER, MAX_UINT64, false); + = segment_snap->stable->getInputStream(dm_context, new_columns_to_read, range, EMPTY_FILTER, MAX_UINT64, false); if (do_range_filter) { @@ -399,7 +406,9 @@ BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, const ColumnDefines & columns_to_read) { - SegmentSnapshot segment_snap = createSnapshot(dm_context); + auto segment_snap = createSnapshot(dm_context); + if (!segment_snap) + return {}; return getInputStreamRaw(dm_context, columns_to_read, segment_snap, true); } @@ -422,12 +431,12 @@ SegmentPtr Segment::mergeDelta(DMContext & dm_context) const return new_segment; } -StableValueSpacePtr Segment::prepareMergeDelta(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs) const +StableValueSpacePtr Segment::prepareMergeDelta(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs) const { LOG_DEBUG(log, "Segment [" << DB::toString(segment_id) - << "] prepare merge delta start. delta packs: " << DB::toString(segment_snap.delta->getPackCount()) - << ", delta total rows: " << DB::toString(segment_snap.delta->getRows())); + << "] prepare merge delta start. delta packs: " << DB::toString(segment_snap->delta->getPackCount()) + << ", delta total rows: " << DB::toString(segment_snap->delta->getRows())); EventRecorder recorder(ProfileEvents::DMDeltaMerge, ProfileEvents::DMDeltaMergeNS); @@ -437,8 +446,8 @@ StableValueSpacePtr Segment::prepareMergeDelta(DMContext & dm_context, SegmentSn read_info.read_columns, range, EMPTY_FILTER, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), @@ -449,7 +458,7 @@ StableValueSpacePtr Segment::prepareMergeDelta(DMContext & dm_context, SegmentSn data_stream = std::make_shared>( data_stream, read_info.read_columns, dm_context.min_version); - auto new_stable = createNewStable(dm_context, data_stream, segment_snap.stable->getId(), wbs); + auto new_stable = createNewStable(dm_context, data_stream, segment_snap->stable->getId(), wbs); LOG_DEBUG(log, "Segment [" << DB::toString(segment_id) << "] prepare merge delta done."); @@ -457,13 +466,13 @@ StableValueSpacePtr Segment::prepareMergeDelta(DMContext & dm_context, SegmentSn } SegmentPtr Segment::applyMergeDelta(DMContext & context, - SegmentSnapshot & segment_snap, + const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs, const StableValueSpacePtr & new_stable) const { LOG_DEBUG(log, "Before apply merge delta: " << info()); - auto later_packs = delta->checkHeadAndCloneTail(context, range, segment_snap.delta->packs, wbs); + auto later_packs = delta->checkHeadAndCloneTail(context, range, segment_snap->delta->packs, wbs); // Created references to tail pages' pages in "log" storage, we need to write them down. wbs.writeLogAndData(); @@ -576,7 +585,7 @@ Handle Segment::getSplitPointFast(DMContext & dm_context, const StableValueSpace return block.getByPosition(0).column->getInt(read_row_in_pack); } -Handle Segment::getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_info, SegmentSnapshot & segment_snap) const +Handle Segment::getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_info, const SegmentSnapshotPtr & segment_snap) const { EventRecorder recorder(ProfileEvents::DMSegmentGetSplitPoint, ProfileEvents::DMSegmentGetSplitPointNS); @@ -588,8 +597,8 @@ Handle Segment::getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_ {handle}, range, EMPTY_FILTER, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), @@ -608,8 +617,8 @@ Handle Segment::getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_ {handle}, range, EMPTY_FILTER, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), @@ -644,15 +653,15 @@ Handle Segment::getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_ return split_handle; } -Segment::SplitInfo Segment::prepareSplit(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs) const +Segment::SplitInfo Segment::prepareSplit(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs) const { - if (!dm_context.enable_logical_split // - || segment_snap.stable->getPacks() <= 3 // - || segment_snap.delta->getRows() > segment_snap.stable->getRows()) + if (!dm_context.enable_logical_split // + || segment_snap->stable->getPacks() <= 3 // + || segment_snap->delta->getRows() > segment_snap->stable->getRows()) return prepareSplitPhysical(dm_context, segment_snap, wbs); else { - Handle split_point = getSplitPointFast(dm_context, segment_snap.stable); + Handle split_point = getSplitPointFast(dm_context, segment_snap->stable); bool bad_split_point = !range.check(split_point) || split_point == range.start; if (bad_split_point) return prepareSplitPhysical(dm_context, segment_snap, wbs); @@ -662,7 +671,7 @@ Segment::SplitInfo Segment::prepareSplit(DMContext & dm_context, SegmentSnapshot } Segment::SplitInfo -Segment::prepareSplitLogical(DMContext & dm_context, SegmentSnapshot & segment_snap, Handle split_point, WriteBatches & wbs) const +Segment::prepareSplitLogical(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, Handle split_point, WriteBatches & wbs) const { LOG_DEBUG(log, "Segment [" << segment_id << "] prepare split logical start"); @@ -682,10 +691,10 @@ Segment::prepareSplitLogical(DMContext & dm_context, SegmentSnapshot & segment_s DMFiles my_stable_files; DMFiles other_stable_files; - for (auto & dmfile : segment_snap.stable->getDMFiles()) + for (auto & dmfile : segment_snap->stable->getDMFiles()) { auto ori_ref_id = dmfile->refId(); - auto file_id = segment_snap.delta->storage_snap->data_reader.getNormalPageId(ori_ref_id); + auto file_id = segment_snap->delta->storage_snap->data_reader.getNormalPageId(ori_ref_id); auto file_parent_path = dm_context.extra_paths.getPath(file_id) + "/" + STABLE_FOLDER_NAME; auto my_dmfile_id = storage_pool.newDataPageId(); @@ -704,7 +713,7 @@ Segment::prepareSplitLogical(DMContext & dm_context, SegmentSnapshot & segment_s auto other_stable_id = storage_pool.newMetaPageId(); - auto my_stable = std::make_shared(segment_snap.stable->getId()); + auto my_stable = std::make_shared(segment_snap->stable->getId()); auto other_stable = std::make_shared(other_stable_id); my_stable->setFiles(my_stable_files, &dm_context, my_range); @@ -715,9 +724,9 @@ Segment::prepareSplitLogical(DMContext & dm_context, SegmentSnapshot & segment_s return {true, split_point, my_stable, other_stable}; } -Segment::SplitInfo Segment::prepareSplitPhysical(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs) const +Segment::SplitInfo Segment::prepareSplitPhysical(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs) const { - LOG_DEBUG(log, "Segment [" << segment_id << "] physical split physical start"); + LOG_DEBUG(log, "Segment [" << segment_id << "] prepare split physical start"); EventRecorder recorder(ProfileEvents::DMSegmentSplit, ProfileEvents::DMSegmentSplitNS); @@ -740,34 +749,40 @@ Segment::SplitInfo Segment::prepareSplitPhysical(DMContext & dm_context, Segment read_info.read_columns, my_range, EMPTY_FILTER, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), dm_context.stable_pack_rows); + LOG_DEBUG(log, "Created my placed stream"); + my_data = std::make_shared>(my_data, my_range, 0); my_data = std::make_shared(my_data, EXTRA_HANDLE_COLUMN_NAME); my_data = std::make_shared>( my_data, read_info.read_columns, dm_context.min_version); - auto my_stable_id = segment_snap.stable->getId(); + auto my_stable_id = segment_snap->stable->getId(); my_new_stable = createNewStable(dm_context, my_data, my_stable_id, wbs); } + LOG_DEBUG(log, "prepare my_new_stable done"); + { // Write new segment's data BlockInputStreamPtr other_data = getPlacedStream(dm_context, read_info.read_columns, other_range, EMPTY_FILTER, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), dm_context.stable_pack_rows); + LOG_DEBUG(log, "Created other placed stream"); + other_data = std::make_shared>(other_data, other_range, 0); other_data = std::make_shared(other_data, EXTRA_HANDLE_COLUMN_NAME); other_data = std::make_shared>( @@ -776,6 +791,8 @@ Segment::SplitInfo Segment::prepareSplitPhysical(DMContext & dm_context, Segment other_stable = createNewStable(dm_context, other_data, other_stable_id, wbs); } + LOG_DEBUG(log, "prepare other_stable done"); + // Remove old stable's files. for (auto & file : stable->getDMFiles()) { @@ -784,15 +801,15 @@ Segment::SplitInfo Segment::prepareSplitPhysical(DMContext & dm_context, Segment wbs.removed_data.delPage(file->refId()); } - LOG_DEBUG(log, "Segment [" << segment_id << "] physical split physical end"); + LOG_DEBUG(log, "Segment [" << segment_id << "] prepare split physical end"); return {false, split_point, my_new_stable, other_stable}; } -SegmentPair Segment::applySplit(DMContext & dm_context, // - SegmentSnapshot & segment_snap, - WriteBatches & wbs, - SplitInfo & split_info) const +SegmentPair Segment::applySplit(DMContext & dm_context, // + const SegmentSnapshotPtr & segment_snap, + WriteBatches & wbs, + SplitInfo & split_info) const { LOG_DEBUG(log, "Segment [" << segment_id << "] apply split"); @@ -800,7 +817,7 @@ SegmentPair Segment::applySplit(DMContext & dm_context, // HandleRange other_range = {split_info.split_point, range.end}; Packs empty_packs; - Packs * head_packs = split_info.is_logical ? &empty_packs : &segment_snap.delta->packs; + Packs * head_packs = split_info.is_logical ? &empty_packs : &segment_snap->delta->packs; auto my_delta_packs = delta->checkHeadAndCloneTail(dm_context, my_range, *head_packs, wbs); auto other_delta_packs = delta->checkHeadAndCloneTail(dm_context, other_range, *head_packs, wbs); @@ -869,12 +886,12 @@ SegmentPtr Segment::merge(DMContext & dm_context, const SegmentPtr & left, const return merged; } -StableValueSpacePtr Segment::prepareMerge(DMContext & dm_context, // - const SegmentPtr & left, - SegmentSnapshot & left_snap, - const SegmentPtr & right, - SegmentSnapshot & right_snap, - WriteBatches & wbs) +StableValueSpacePtr Segment::prepareMerge(DMContext & dm_context, // + const SegmentPtr & left, + const SegmentSnapshotPtr & left_snap, + const SegmentPtr & right, + const SegmentSnapshotPtr & right_snap, + WriteBatches & wbs) { LOG_DEBUG(left->log, "Segment [" << left->segmentId() << "] and [" << right->segmentId() << "] prepare merge start"); @@ -882,14 +899,14 @@ StableValueSpacePtr Segment::prepareMerge(DMContext & dm_context, // throw Exception("The ranges of merge segments are not consecutive: first end: " + DB::toString(left->range.end) + ", second start: " + DB::toString(right->range.start)); - auto getStream = [&](const SegmentPtr & segment, SegmentSnapshot & segment_snap) { + auto getStream = [&](const SegmentPtr & segment, const SegmentSnapshotPtr & segment_snap) { auto read_info = segment->getReadInfo(dm_context, *dm_context.store_columns, segment_snap); BlockInputStreamPtr stream = segment->getPlacedStream(dm_context, read_info.read_columns, segment->range, EMPTY_FILTER, - segment_snap.stable, - segment_snap.delta, + segment_snap->stable, + segment_snap->delta, read_info.index_begin, read_info.index_end, read_info.index->entryCount(), @@ -918,9 +935,9 @@ StableValueSpacePtr Segment::prepareMerge(DMContext & dm_context, // SegmentPtr Segment::applyMerge(DMContext & dm_context, // const SegmentPtr & left, - SegmentSnapshot & left_snap, + const SegmentSnapshotPtr & left_snap, const SegmentPtr & right, - SegmentSnapshot & right_snap, + const SegmentSnapshotPtr & right_snap, WriteBatches & wbs, const StableValueSpacePtr & merged_stable) { @@ -928,8 +945,8 @@ SegmentPtr Segment::applyMerge(DMContext & dm_context, // HandleRange merged_range = {left->range.start, right->range.end}; - auto left_tail_packs = left->delta->checkHeadAndCloneTail(dm_context, merged_range, left_snap.delta->packs, wbs); - auto right_tail_packs = right->delta->checkHeadAndCloneTail(dm_context, merged_range, right_snap.delta->packs, wbs); + auto left_tail_packs = left->delta->checkHeadAndCloneTail(dm_context, merged_range, left_snap->delta->packs, wbs); + auto right_tail_packs = right->delta->checkHeadAndCloneTail(dm_context, merged_range, right_snap->delta->packs, wbs); // Created references to tail pages' pages in "log" storage, we need to write them down. wbs.writeLogAndData(); @@ -1022,19 +1039,19 @@ String Segment::info() const template Segment::ReadInfo -Segment::getReadInfo(const DMContext & dm_context, const ColumnDefines & read_columns, SegmentSnapshot & segment_snap) const +Segment::getReadInfo(const DMContext & dm_context, const ColumnDefines & read_columns, const SegmentSnapshotPtr & segment_snap) const { - LOG_TRACE(log, "getReadInfo start"); + LOG_DEBUG(log, "getReadInfo start"); auto new_read_columns = arrangeReadColumns(getExtraHandleColumnDefine(), read_columns); - segment_snap.delta->prepare(dm_context, new_read_columns); + segment_snap->delta->prepare(dm_context, new_read_columns); - DeltaIndexPtr delta_index = ensurePlace(dm_context, segment_snap.stable, segment_snap.delta); + DeltaIndexPtr delta_index = ensurePlace(dm_context, segment_snap->stable, segment_snap->delta); auto index_begin = DeltaIndex::begin(delta_index); auto index_end = DeltaIndex::end(delta_index); - LOG_TRACE(log, "getReadInfo end"); + LOG_DEBUG(log, "getReadInfo end"); return { .index = delta_index, diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index cc7509d9f12..b9d689b9626 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -18,6 +18,7 @@ namespace DM class Segment; struct SegmentSnapshot; +using SegmentSnapshotPtr = std::shared_ptr; class StableValueSpace; using StableValueSpacePtr = std::shared_ptr; class DeltaValueSpace; @@ -28,14 +29,12 @@ using SegmentPair = std::pair; using Segments = std::vector; /// A structure stores the informations to constantly read a segment instance. -struct SegmentSnapshot +struct SegmentSnapshot : private boost::noncopyable { DeltaSnapshotPtr delta; StableValueSpacePtr stable; - SegmentSnapshot() = default; - - explicit operator bool() { return (bool)delta; } + SegmentSnapshot(const DeltaSnapshotPtr & delta_, const StableValueSpacePtr & stable_) : delta(delta_), stable(stable_) {} }; /// A segment contains many rows of a table. A table is split into segments by consecutive ranges. @@ -101,15 +100,15 @@ class Segment : private boost::noncopyable bool write(DMContext & dm_context, const Block & block); // For test only bool write(DMContext & dm_context, const HandleRange & delete_range); - SegmentSnapshot createSnapshot(const DMContext & dm_context, bool is_update = false) const; + SegmentSnapshotPtr createSnapshot(const DMContext & dm_context, bool is_update = false) const; - BlockInputStreamPtr getInputStream(const DMContext & dm_context, - const ColumnDefines & columns_to_read, - SegmentSnapshot & segment_snap, - const HandleRanges & read_ranges, - const RSOperatorPtr & filter, - UInt64 max_version, - size_t expected_block_size); + BlockInputStreamPtr getInputStream(const DMContext & dm_context, + const ColumnDefines & columns_to_read, + const SegmentSnapshotPtr & segment_snap, + const HandleRanges & read_ranges, + const RSOperatorPtr & filter, + UInt64 max_version, + size_t expected_block_size); BlockInputStreamPtr getInputStream(const DMContext & dm_context, const ColumnDefines & columns_to_read, @@ -118,10 +117,10 @@ class Segment : private boost::noncopyable UInt64 max_version = MAX_UINT64, size_t expected_block_size = DEFAULT_BLOCK_SIZE); - BlockInputStreamPtr getInputStreamRaw(const DMContext & dm_context, - const ColumnDefines & columns_to_read, - SegmentSnapshot & segment_snap, - bool do_range_filter); + BlockInputStreamPtr getInputStreamRaw(const DMContext & dm_context, + const ColumnDefines & columns_to_read, + const SegmentSnapshotPtr & segment_snap, + bool do_range_filter); BlockInputStreamPtr getInputStreamRaw(const DMContext & dm_context, const ColumnDefines & columns_to_read); @@ -129,28 +128,29 @@ class Segment : private boost::noncopyable /// split(), merge() and mergeDelta() are only used in test cases. SegmentPair split(DMContext & dm_context) const; - SplitInfo prepareSplit(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs) const; - SegmentPair applySplit(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs, SplitInfo & split_info) const; + SplitInfo prepareSplit(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs) const; + SegmentPair + applySplit(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs, SplitInfo & split_info) const; static SegmentPtr merge(DMContext & dm_context, const SegmentPtr & left, const SegmentPtr & right); - static StableValueSpacePtr prepareMerge(DMContext & dm_context, // - const SegmentPtr & left, - SegmentSnapshot & left_snap, - const SegmentPtr & right, - SegmentSnapshot & right_snap, - WriteBatches & wbs); + static StableValueSpacePtr prepareMerge(DMContext & dm_context, // + const SegmentPtr & left, + const SegmentSnapshotPtr & left_snap, + const SegmentPtr & right, + const SegmentSnapshotPtr & right_snap, + WriteBatches & wbs); static SegmentPtr applyMerge(DMContext & dm_context, // const SegmentPtr & left, - SegmentSnapshot & left_snap, + const SegmentSnapshotPtr & left_snap, const SegmentPtr & right, - SegmentSnapshot & right_snap, + const SegmentSnapshotPtr & right_snap, WriteBatches & wbs, const StableValueSpacePtr & merged_stable); SegmentPtr mergeDelta(DMContext & dm_context) const; - StableValueSpacePtr prepareMergeDelta(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs) const; + StableValueSpacePtr prepareMergeDelta(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs) const; SegmentPtr applyMergeDelta(DMContext & dm_context, - SegmentSnapshot & segment_snap, + const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs, const StableValueSpacePtr & new_stable) const; @@ -200,7 +200,7 @@ class Segment : private boost::noncopyable private: template - ReadInfo getReadInfo(const DMContext & dm_context, const ColumnDefines & read_columns, SegmentSnapshot & segment_snap) const; + ReadInfo getReadInfo(const DMContext & dm_context, const ColumnDefines & read_columns, const SegmentSnapshotPtr & segment_snap) const; template static ColumnDefines arrangeReadColumns(const ColumnDefine & handle, const ColumnDefines & columns_to_read); @@ -218,12 +218,15 @@ class Segment : private boost::noncopyable size_t expected_block_size) const; /// Merge delta & stable, and then take the middle one. - Handle getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_info, SegmentSnapshot & segment_snap) const; + Handle getSplitPointSlow(DMContext & dm_context, const ReadInfo & read_info, const SegmentSnapshotPtr & segment_snap) const; /// Only look up in the stable vs. Handle getSplitPointFast(DMContext & dm_context, const StableValueSpacePtr & stable_snap) const; - SplitInfo prepareSplitLogical(DMContext & dm_context, SegmentSnapshot & segment_snap, Handle split_point, WriteBatches & wbs) const; - SplitInfo prepareSplitPhysical(DMContext & dm_context, SegmentSnapshot & segment_snap, WriteBatches & wbs) const; + SplitInfo prepareSplitLogical(DMContext & dm_context, // + const SegmentSnapshotPtr & segment_snap, + Handle split_point, + WriteBatches & wbs) const; + SplitInfo prepareSplitPhysical(DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, WriteBatches & wbs) const; /// Make sure that all delta packs have been placed. diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h index c4c2e437b04..ac26cad9337 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -11,18 +11,18 @@ namespace DM struct SegmentReadTask { - SegmentPtr segment; - SegmentSnapshot read_snapshot; - HandleRanges ranges; + SegmentPtr segment; + SegmentSnapshotPtr read_snapshot; + HandleRanges ranges; - explicit SegmentReadTask(const SegmentPtr & segment_, const SegmentSnapshot & read_snapshot_) + explicit SegmentReadTask(const SegmentPtr & segment_, const SegmentSnapshotPtr & read_snapshot_) : segment(segment_), read_snapshot(read_snapshot_) { } - SegmentReadTask(const SegmentPtr & segment_, // - const SegmentSnapshot & read_snapshot_, - const HandleRanges & ranges_) + SegmentReadTask(const SegmentPtr & segment_, // + const SegmentSnapshotPtr & read_snapshot_, + const HandleRanges & ranges_) : segment(segment_), read_snapshot(read_snapshot_), ranges(ranges_) { } diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 29e98fea9b8..324fe82f5a3 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -100,15 +100,16 @@ SkippableBlockInputStreamPtr StableValueSpace::getInputStream(const DMContext & SkippableBlockInputStreams streams; for (auto & file : files) { - streams.push_back(std::make_shared(context.db_context, // - max_data_version, - enable_clean_read, - context.hash_salt, - file, - read_columns, - handle_range, - filter, - IdSetPtr{})); + streams.push_back(std::make_shared( // + context.db_context, + max_data_version, + enable_clean_read, + context.hash_salt, + file, + read_columns, + handle_range, + filter, + IdSetPtr{})); } return std::make_shared(streams); } diff --git a/dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.cpp b/dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.cpp new file mode 100644 index 00000000000..362e4aa20c7 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.cpp @@ -0,0 +1,360 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace DM +{ + +namespace +{ + +/// some helper functions for casting column data type + +template +void insertRangeFromWithNumericTypeCast(const ColumnPtr & from_col, // + const ColumnPtr & null_map, + const ColumnDefine & read_define, + MutableColumnPtr & to_col, + size_t rows_offset, + size_t rows_limit) +{ + // Caller should ensure that both from_col / to_col + // * is numeric + // * no nullable wrapper + // * both signed or unsigned + static_assert(std::is_integral_v); + static_assert(std::is_integral_v); + constexpr bool is_both_signed_or_unsigned = !(std::is_unsigned_v ^ std::is_unsigned_v); + static_assert(is_both_signed_or_unsigned); + assert(from_col != nullptr); + assert(to_col != nullptr); + assert(from_col->isNumeric()); + assert(to_col->isNumeric()); + assert(!from_col->isColumnNullable()); + assert(!to_col->isColumnNullable()); + assert(!from_col->isColumnConst()); + assert(!to_col->isColumnConst()); + + // Something like `insertRangeFrom(from_col, rows_offset, rows_limit)` with static_cast + const PaddedPODArray & from_array = toColumnVectorData(from_col); + PaddedPODArray * to_array_ptr = toMutableColumnVectorDataPtr(to_col); + to_array_ptr->reserve(rows_limit); + for (size_t i = 0; i < rows_limit; ++i) + { + (*to_array_ptr).emplace_back(static_cast(from_array[rows_offset + i])); + } + + if (unlikely(null_map)) + { + /// We are applying cast from nullable to not null, scan to fill "NULL" with default value + + TypeTo default_value = 0; // if read_define.default_value is empty, fill with 0 + if (read_define.default_value.isNull()) + { + // Do nothing + } + else if (read_define.default_value.getType() == Field::Types::Int64) + { + default_value = read_define.default_value.safeGet(); + } + else if (read_define.default_value.getType() == Field::Types::UInt64) + { + default_value = read_define.default_value.safeGet(); + } + else + { + throw Exception("Invalid column value type", ErrorCodes::BAD_ARGUMENTS); + } + + const size_t to_offset_before_inserted = to_array_ptr->size() - rows_limit; + + for (size_t i = 0; i < rows_limit; ++i) + { + const size_t to_offset = to_offset_before_inserted + i; + if (null_map->getInt(rows_offset + i) != 0) + { + // `from_col[rows_offset + i]` is "NULL", fill `to_col[x]` with default value + (*to_array_ptr)[to_offset] = static_cast(default_value); + } + } + } +} + + +bool castNonNullNumericColumn(const DataTypePtr & disk_type_not_null_, + const ColumnPtr & disk_col_not_null, + const ColumnDefine & read_define, + const ColumnPtr & null_map, + MutableColumnPtr & memory_col_not_null, + size_t rows_offset, + size_t rows_limit) +{ + /// Caller should ensure that type is not nullable + assert(disk_type_not_null_ != nullptr); + assert(disk_col_not_null != nullptr); + assert(read_define.type != nullptr); + assert(memory_col_not_null != nullptr); + + const IDataType * disk_type_not_null = disk_type_not_null_.get(); + const IDataType * read_type_not_null = read_define.type.get(); + + /// Caller should ensure nullable is unwrapped + assert(!disk_type_not_null->isNullable()); + assert(!read_type_not_null->isNullable()); + + /// Caller should ensure that dist_type != read_type + assert(!disk_type_not_null->equals(*read_type_not_null)); + + if (checkDataType(disk_type_not_null)) + { + using FromType = UInt32; + if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + } + else if (checkDataType(disk_type_not_null)) + { + using FromType = Int32; + if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + } + else if (checkDataType(disk_type_not_null)) + { + using FromType = UInt16; + if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + else if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + } + else if (checkDataType(disk_type_not_null)) + { + using FromType = Int16; + if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + else if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + } + else if (checkDataType(disk_type_not_null)) + { + using FromType = UInt8; + if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + else if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + else if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + } + else if (checkDataType(disk_type_not_null)) + { + using FromType = Int8; + if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + else if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + else if (checkDataType(read_type_not_null)) + { + insertRangeFromWithNumericTypeCast( + disk_col_not_null, null_map, read_define, memory_col_not_null, rows_offset, rows_limit); + return true; + } + } + + // else is not support + return false; +} + +} // namespace + +void convertColumnByColumnDefine(const DataTypePtr & disk_type, + const ColumnPtr & disk_col, + const ColumnDefine & read_define, + MutableColumnPtr memory_col, + size_t rows_offset, + size_t rows_limit) +{ + const DataTypePtr & read_type = read_define.type; + + // Unwrap nullable(what) + ColumnPtr disk_col_not_null; + MutableColumnPtr memory_col_not_null; + ColumnPtr null_map; + DataTypePtr disk_type_not_null = disk_type; + DataTypePtr read_type_not_null = read_type; + if (disk_type->isNullable() && read_type->isNullable()) + { + // nullable -> nullable, copy null map + const auto & disk_nullable_col = typeid_cast(*disk_col); + const auto & disk_null_map = disk_nullable_col.getNullMapData(); + auto & memory_nullable_col = typeid_cast(*memory_col); + auto & memory_null_map = memory_nullable_col.getNullMapData(); + memory_null_map.insert(disk_null_map.begin(), disk_null_map.end()); + + disk_col_not_null = disk_nullable_col.getNestedColumnPtr(); + memory_col_not_null = memory_nullable_col.getNestedColumn().getPtr(); + + const auto * type_nullable = typeid_cast(disk_type.get()); + disk_type_not_null = type_nullable->getNestedType(); + type_nullable = typeid_cast(read_type.get()); + read_type_not_null = type_nullable->getNestedType(); + } + else if (!disk_type->isNullable() && read_type->isNullable()) + { + // not null -> nullable, set null map to all not null + auto & memory_nullable_col = typeid_cast(*memory_col); + auto & nullmap_data = memory_nullable_col.getNullMapData(); + nullmap_data.resize_fill(rows_offset + rows_limit, 0); + + disk_col_not_null = disk_col; + memory_col_not_null = memory_nullable_col.getNestedColumn().getPtr(); + + const auto * type_nullable = typeid_cast(read_type.get()); + read_type_not_null = type_nullable->getNestedType(); + } + else if (disk_type->isNullable() && !read_type->isNullable()) + { + // nullable -> not null, fill "NULL" values with default value later + const auto & disk_nullable_col = typeid_cast(*disk_col); + null_map = disk_nullable_col.getNullMapColumnPtr(); + disk_col_not_null = disk_nullable_col.getNestedColumnPtr(); + memory_col_not_null = std::move(memory_col); + + const auto * type_nullable = typeid_cast(disk_type.get()); + disk_type_not_null = type_nullable->getNestedType(); + } + else + { + // not null -> not null + disk_col_not_null = disk_col; + memory_col_not_null = std::move(memory_col); + } + + assert(memory_col_not_null != nullptr); + assert(disk_col_not_null != nullptr); + assert(read_type_not_null != nullptr); + assert(disk_type_not_null != nullptr); + + ColumnDefine read_define_not_null(read_define); + read_define_not_null.type = read_type_not_null; + if (disk_type_not_null->equals(*read_type_not_null)) + { + // just change from nullable -> not null / not null -> nullable + memory_col_not_null->insertRangeFrom(*disk_col_not_null, rows_offset, rows_limit); + + if (null_map) + { + /// We are applying cast from nullable to not null, scan to fill "NULL" with default value + + for (size_t i = 0; i < rows_limit; ++i) + { + if (unlikely(null_map->getInt(i) != 0)) + { + // `from_col[i]` is "NULL", fill `to_col[rows_offset + i]` with default value + // TiDB/MySQL don't support this, should not call here. + throw Exception("Reading mismatch data type pack. Cast from " + disk_type->getName() + " to " + read_type->getName() + + " with \"NULL\" value is NOT supported!", + ErrorCodes::NOT_IMPLEMENTED); + } + } + } + } + else if (!castNonNullNumericColumn( + disk_type_not_null, disk_col_not_null, read_define_not_null, null_map, memory_col_not_null, rows_offset, rows_limit)) + { + throw Exception("Reading mismatch data type pack. Cast and assign from " + disk_type->getName() + " to " + read_type->getName() + + " is NOT supported!", + ErrorCodes::NOT_IMPLEMENTED); + } +} + +ColumnPtr convertColumnByColumnDefineIfNeed(const DataTypePtr & from_type, ColumnPtr && from_col, const ColumnDefine & to_column_define) +{ + // No need to convert + if (likely(from_type->equals(*to_column_define.type))) + return std::move(from_col); + + // Check if support + if (unlikely(!isSupportedDataTypeCast(from_type, to_column_define.type))) + { + throw Exception("Reading mismatch data type pack. Cast from " + from_type->getName() + " to " + to_column_define.type->getName() + + " is NOT supported!", + ErrorCodes::NOT_IMPLEMENTED); + } + + // Cast column's data from DataType in disk to what we need now + auto to_col = to_column_define.type->createColumn(); + to_col->reserve(from_col->size()); + convertColumnByColumnDefine(from_type, from_col, to_column_define, to_col->getPtr(), 0, from_col->size()); + return to_col; +} + +ColumnPtr createColumnWithDefaultValue(const ColumnDefine & column_define, size_t num_rows) +{ + ColumnPtr column; + // Read default value from `column_define.default_value` + if (column_define.default_value.isNull()) + { + column = column_define.type->createColumnConstWithDefaultValue(num_rows); + } + else + { + column = column_define.type->createColumnConst(num_rows, column_define.default_value); + } + column = column->convertToFullColumnIfConst(); + return column; +} + +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.h b/dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.h new file mode 100644 index 00000000000..f1a914851cb --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.h @@ -0,0 +1,22 @@ +#include +#include +#include + +namespace DB +{ +namespace DM +{ +//========================================================================================== +// Functions for casting column data when disk data type mismatch with read data type. +//========================================================================================== + +// If `from_type` is the same as `to_column_define.type`, simply return `from_col`. +// If `from_type` is different from `to_column_define.type`, check if we can apply +// cast on read, if not, throw exception. +ColumnPtr convertColumnByColumnDefineIfNeed(const DataTypePtr & from_type, ColumnPtr && from_col, const ColumnDefine & to_column_define); + +// Create a column with `num_rows`, fill with column_define.default_value or column's default. +ColumnPtr createColumnWithDefaultValue(const ColumnDefine & column_define, size_t num_rows); + +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp new file mode 100644 index 00000000000..24e9c4cfaaf --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp @@ -0,0 +1,246 @@ +#include +#include +#include +#include + +#include "dm_basic_include.h" + +namespace DB +{ +namespace DM +{ +namespace tests +{ + +TEST(ConvertColumnType_test, CastNumeric) +{ + { + const Strings to_types = {"UInt8", "UInt16", "UInt32", "UInt64"}; + + DataTypePtr disk_data_type = typeFromString("UInt8"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(UInt64(15))); + disk_col->insert(Field(UInt64(255))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + UInt64 val1 = memory_column->getUInt(0); + ASSERT_EQ(val1, 15UL); + UInt64 val2 = memory_column->getUInt(1); + ASSERT_EQ(val2, 255UL); + } + } + + { + const Strings to_types = {"UInt16", "UInt32", "UInt64"}; + + DataTypePtr disk_data_type = typeFromString("UInt16"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(UInt64(15))); + disk_col->insert(Field(UInt64(255))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + UInt64 val1 = memory_column->getUInt(0); + ASSERT_EQ(val1, 15UL); + UInt64 val2 = memory_column->getUInt(1); + ASSERT_EQ(val2, 255UL); + } + } + + { + const Strings to_types = {"UInt32", "UInt64"}; + + DataTypePtr disk_data_type = typeFromString("UInt32"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(UInt64(15))); + disk_col->insert(Field(UInt64(255))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + UInt64 val1 = memory_column->getUInt(0); + ASSERT_EQ(val1, 15UL); + UInt64 val2 = memory_column->getUInt(1); + ASSERT_EQ(val2, 255UL); + } + } + + { + const Strings to_types = {"Int8", "Int16", "Int32", "Int64"}; + + DataTypePtr disk_data_type = typeFromString("Int8"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + Int64 val1 = memory_column->getInt(0); + ASSERT_EQ(val1, 127L); + Int64 val2 = memory_column->getInt(1); + ASSERT_EQ(val2, -1L); + } + } + + { + const Strings to_types = {"Int16", "Int32", "Int64"}; + + DataTypePtr disk_data_type = typeFromString("Int16"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + Int64 val1 = memory_column->getInt(0); + ASSERT_EQ(val1, 127L); + Int64 val2 = memory_column->getInt(1); + ASSERT_EQ(val2, -1L); + } + } + + { + const Strings to_types = {"Int32", "Int64"}; + + DataTypePtr disk_data_type = typeFromString("Int32"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + Int64 val1 = memory_column->getInt(0); + ASSERT_EQ(val1, 127L); + Int64 val2 = memory_column->getInt(1); + ASSERT_EQ(val2, -1L); + } + } +} + +TEST(ConvertColumnType_test, CastNullableToNotNull) +{ + const Strings to_types = {"Int16", "Int32", "Int64"}; + + DataTypePtr disk_data_type = typeFromString("Nullable(Int8)"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field()); // a "NULL" value + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + Int64 val1 = memory_column->getInt(0); + ASSERT_EQ(val1, 0); // "NULL" value is cast to 0 + Int64 val2 = memory_column->getInt(1); + ASSERT_EQ(val2, 127L); + Int64 val3 = memory_column->getUInt(2); + ASSERT_EQ(val3, -1L); + } +} + +TEST(ConvertColumnType_test, CastNullableToNotNullWithNonZeroDefaultValue) +{ + const Strings to_types = {"Int16", "Int32", "Int64"}; + + DataTypePtr disk_data_type = typeFromString("Nullable(Int8)"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field()); // a "NULL" value + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + read_define.default_value = Field(Int64(5)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + Int64 val1 = memory_column->getInt(0); + ASSERT_EQ(val1, 5); // "NULL" value is cast to default value (5) + Int64 val2 = memory_column->getInt(1); + ASSERT_EQ(val2, 127L); + Int64 val3 = memory_column->getUInt(2); + ASSERT_EQ(val3, -1L); + } +} + +TEST(ConvertColumnType_test, CastNullableToNullable) +{ + const Strings to_types = {"Nullable(Int8)", "Nullable(Int16)", "Nullable(Int32)", "Nullable(Int64)"}; + + DataTypePtr disk_data_type = typeFromString("Nullable(Int8)"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field()); // a "NULL" value + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + ASSERT_TRUE(memory_column->isNullAt(0)); + Field f = (*memory_column)[0]; + ASSERT_TRUE(f.isNull()); + + ASSERT_FALSE(memory_column->isNullAt(1)); + f = (*memory_column)[1]; + ASSERT_EQ(f.getType(), Field::Types::Int64); + ASSERT_EQ(f.get(), 127L); + + ASSERT_FALSE(memory_column->isNullAt(2)); + f = (*memory_column)[2]; + ASSERT_EQ(f.getType(), Field::Types::Int64); + ASSERT_EQ(f.get(), -1L); + } +} + +TEST(ConvertColumnType_test, CastNotNullToNullable) +{ + const Strings to_types = {"Nullable(Int8)", "Nullable(Int16)", "Nullable(Int32)", "Nullable(Int64)"}; + + DataTypePtr disk_data_type = typeFromString("Int8"); + MutableColumnPtr disk_col = disk_data_type->createColumn(); + disk_col->insert(Field(Int64(127))); + disk_col->insert(Field(Int64(-1))); + + for (const String & to_type : to_types) + { + ColumnDefine read_define(0, "c", typeFromString(to_type)); + auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define); + + ASSERT_FALSE(memory_column->isNullAt(0)); + Field f = (*memory_column)[0]; + ASSERT_EQ(f.getType(), Field::Types::Int64); + ASSERT_EQ(f.get(), 127L); + + ASSERT_FALSE(memory_column->isNullAt(1)); + f = (*memory_column)[1]; + ASSERT_EQ(f.getType(), Field::Types::Int64); + ASSERT_EQ(f.get(), -1L); + } +} + +} // namespace tests +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp index af39b138943..022f5b32791 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp @@ -25,9 +25,12 @@ class DMFile_Test : public ::testing::Test void SetUp() override { dropFiles(); - storage_pool = std::make_unique("test.t1", path, DB::Settings()); - dm_file = DMFile::create(0, path); - db_context = std::make_unique(DMTestEnv::getContext(DB::Settings())); + + auto settings = DB::Settings(); + storage_pool = std::make_unique("test.t1", path, settings); + dm_file = DMFile::create(0, path); + db_context = std::make_unique(DMTestEnv::getContext(settings)); + table_columns_ = std::make_shared(); reload(); } @@ -41,17 +44,21 @@ class DMFile_Test : public ::testing::Test } } + // Update dm_context. void reload(const ColumnDefinesPtr & cols = DMTestEnv::getDefaultColumns()) { - dm_context = std::make_unique(*db_context, - path, - db_context->getExtraPaths(), - *storage_pool, - 0, - cols, - 0, - settings.not_compress_columns, - db_context->getSettingsRef()); + *table_columns_ = *cols; + + dm_context = std::make_unique( // + *db_context, + path, + db_context->getExtraPaths(), + *storage_pool, + /*hash_salt*/ 0, + table_columns_, + 0, + settings.not_compress_columns, + db_context->getSettingsRef()); } @@ -63,10 +70,10 @@ class DMFile_Test : public ::testing::Test String path; std::unique_ptr db_context; std::unique_ptr dm_context; - + /// all these var live as ref in dm_context std::unique_ptr storage_pool; - - DeltaMergeStore::Settings settings; + ColumnDefinesPtr table_columns_; + DeltaMergeStore::Settings settings; protected: DMFilePtr dm_file; @@ -76,7 +83,8 @@ class DMFile_Test : public ::testing::Test TEST_F(DMFile_Test, WriteRead) try { - auto cols = DMTestEnv::getDefaultColumns(); + auto cols = DMTestEnv::getDefaultColumns(); + const size_t num_rows_write = 128; { @@ -92,15 +100,16 @@ try { // Test read - auto stream = std::make_shared(dbContext(), // - std::numeric_limits::max(), - false, - dmContext().hash_salt, - dm_file, - *cols, - HandleRange::newAll(), - RSOperatorPtr{}, - IdSetPtr{}); + auto stream = std::make_shared( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); size_t num_rows_read = 0; stream->readPrefix(); @@ -122,7 +131,7 @@ try stream->readSuffix(); ASSERT_EQ(num_rows_read, num_rows_write); } -} // namespace tests +} CATCH TEST_F(DMFile_Test, NumberTypes) @@ -130,26 +139,27 @@ try { auto cols = DMTestEnv::getDefaultColumns(); // Prepare columns - ColumnDefine i64_col(2, "i64", DataTypeFactory::instance().get("Int64")); - ColumnDefine f64_col(3, "f64", DataTypeFactory::instance().get("Float64")); + ColumnDefine i64_col(2, "i64", typeFromString("Int64")); + ColumnDefine f64_col(3, "f64", typeFromString("Float64")); cols->push_back(i64_col); cols->push_back(f64_col); reload(cols); + const size_t num_rows_write = 128; { // Prepare write - Block block = DMTestEnv::prepareSimpleWriteBlock(0, 128, false); + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); auto col = i64_col.type->createColumn(); - for (int i = 0; i < 128; i++) + for (size_t i = 0; i < num_rows_write; i++) { col->insert(toField(Int64(i))); } ColumnWithTypeAndName i64(std::move(col), i64_col.type, i64_col.name, i64_col.id); col = f64_col.type->createColumn(); - for (int i = 0; i < 128; i++) + for (size_t i = 0; i < num_rows_write; i++) { col->insert(toField(Float64(0.125))); } @@ -166,15 +176,16 @@ try { // Test Read - auto stream = std::make_unique(dbContext(), // - std::numeric_limits::max(), - false, - dmContext().hash_salt, - dm_file, - *cols, - HandleRange::newAll(), - RSOperatorPtr{}, - IdSetPtr{}); + auto stream = std::make_unique( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); size_t num_rows_read = 0; stream->readPrefix(); @@ -204,7 +215,7 @@ try num_rows_read += in.rows(); } stream->readSuffix(); - ASSERT_EQ(num_rows_read, 128UL); + ASSERT_EQ(num_rows_read, num_rows_write); } } CATCH @@ -213,17 +224,18 @@ TEST_F(DMFile_Test, StringType) { auto cols = DMTestEnv::getDefaultColumns(); // Prepare columns - ColumnDefine fixed_str_col(2, "str", DataTypeFactory::instance().get("FixedString(5)")); + ColumnDefine fixed_str_col(2, "str", typeFromString("FixedString(5)")); cols->push_back(fixed_str_col); reload(cols); + const size_t num_rows_write = 128; { // Prepare write - Block block = DMTestEnv::prepareSimpleWriteBlock(0, 128, false); + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); auto col = fixed_str_col.type->createColumn(); - for (int i = 0; i < 128; i++) + for (size_t i = 0; i < num_rows_write; i++) { col->insert(toField(String("hello"))); } @@ -239,15 +251,16 @@ TEST_F(DMFile_Test, StringType) { // Test Read - auto stream = std::make_unique(dbContext(), // - std::numeric_limits::max(), - false, - dmContext().hash_salt, - dm_file, - *cols, - HandleRange::newAll(), - RSOperatorPtr{}, - IdSetPtr{}); + auto stream = std::make_unique( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); size_t num_rows_read = 0; stream->readPrefix(); @@ -269,7 +282,7 @@ TEST_F(DMFile_Test, StringType) num_rows_read += in.rows(); } stream->readSuffix(); - ASSERT_EQ(num_rows_read, 128UL); + ASSERT_EQ(num_rows_read, num_rows_write); } } @@ -279,23 +292,24 @@ try auto cols = DMTestEnv::getDefaultColumns(); { // Prepare columns - ColumnDefine nullable_col(2, "i32_null", DataTypeFactory::instance().get("Nullable(Int32)")); + ColumnDefine nullable_col(2, "i32_null", typeFromString("Nullable(Int32)")); cols->emplace_back(nullable_col); } reload(cols); + const size_t num_rows_write = 128; { // Prepare write - Block block = DMTestEnv::prepareSimpleWriteBlock(0, 128, false); + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); - ColumnWithTypeAndName nullable_col({}, DataTypeFactory::instance().get("Nullable(Int32)"), "i32_null", 2); + ColumnWithTypeAndName nullable_col({}, typeFromString("Nullable(Int32)"), "i32_null", 2); auto col = nullable_col.type->createColumn(); - for (int i = 0; i < 64; i++) + for (size_t i = 0; i < 64; i++) { - col->insert(toField(i)); + col->insert(toField(Int64(i))); } - for (int i = 64; i < 128; i++) + for (size_t i = 64; i < num_rows_write; i++) { col->insertDefault(); } @@ -310,15 +324,17 @@ try { // Test read - auto stream = std::make_shared(dbContext(), // - std::numeric_limits::max(), - false, - dmContext().hash_salt, - dm_file, - *cols, - HandleRange::newAll(), - RSOperatorPtr{}, - IdSetPtr{}); + auto stream = std::make_shared( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); + size_t num_rows_read = 0; stream->readPrefix(); while (Block in = stream->read()) @@ -353,8 +369,373 @@ try } num_rows_read += in.rows(); } - ASSERT_EQ(num_rows_read, 128UL); + ASSERT_EQ(num_rows_read, num_rows_write); + stream->readSuffix(); + } +} +CATCH + +/// DDL test cases +class DMFile_DDL_Test : public DMFile_Test +{ +public: + /// Write some data into DMFile. + /// return rows write, schema + std::pair prepareSomeDataToDMFile(bool i8_is_nullable = false) + { + size_t num_rows_write = 128; + auto cols_before_ddl = DMTestEnv::getDefaultColumns(); + + ColumnDefine i8_col(2, "i8", i8_is_nullable ? typeFromString("Nullable(Int8)") : typeFromString("Int8")); + ColumnDefine f64_col(3, "f64", typeFromString("Float64")); + cols_before_ddl->push_back(i8_col); + cols_before_ddl->push_back(f64_col); + + reload(cols_before_ddl); + + { + // Prepare write + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); + + auto col = i8_col.type->createColumn(); + for (size_t i = 0; i < num_rows_write; i++) + { + Field field; // Null by default + if (!i8_is_nullable || (i8_is_nullable && i < num_rows_write / 2)) + field = toField(Int64(i) * (-1 * (i % 2))); + col->insert(field); + } + ColumnWithTypeAndName i64(std::move(col), i8_col.type, i8_col.name, i8_col.id); + + col = f64_col.type->createColumn(); + for (size_t i = 0; i < num_rows_write; i++) + { + col->insert(toField(Float64(0.125))); + } + ColumnWithTypeAndName f64(std::move(col), f64_col.type, f64_col.name, f64_col.id); + + block.insert(i64); + block.insert(f64); + + auto stream = std::make_unique(dbContext(), dm_file, *cols_before_ddl); + stream->writePrefix(); + stream->write(block, 0); + stream->writeSuffix(); + + return {num_rows_write, *cols_before_ddl}; + } + } +}; + +TEST_F(DMFile_DDL_Test, AddColumn) +try +{ + // Prepare some data before ddl + const auto [num_rows_write, cols_before_ddl] = prepareSomeDataToDMFile(); + + // Mock that we add new column after ddl + auto cols_after_ddl = std::make_shared(); + *cols_after_ddl = cols_before_ddl; + // A new string column + ColumnDefine new_s_col(100, "s", typeFromString("String")); + cols_after_ddl->emplace_back(new_s_col); + // A new int64 column with default value 5 + ColumnDefine new_i_col_with_default(101, "i", typeFromString("Int64")); + new_i_col_with_default.default_value = Field(Int64(5)); + cols_after_ddl->emplace_back(new_i_col_with_default); + + { + // Test read with new columns after ddl + auto stream = std::make_unique( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols_after_ddl, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); + + size_t num_rows_read = 0; + stream->readPrefix(); + while (Block in = stream->read()) + { + ASSERT_TRUE(in.has("i8")); + ASSERT_TRUE(in.has("f64")); + ASSERT_TRUE(in.has(new_s_col.name)); + ASSERT_TRUE(in.has(new_i_col_with_default.name)); + for (auto itr : in) + { + auto c = itr.column; + if (itr.name == new_s_col.name) + { + EXPECT_EQ(itr.column_id, new_s_col.id); + EXPECT_TRUE(itr.type->equals(*new_s_col.type)); + for (size_t i = 0; i < c->size(); i++) + { + Field value = (*c)[i]; + ASSERT_EQ(value.getType(), Field::Types::String); + // Empty default value + ASSERT_EQ(value, new_s_col.type->getDefault()); + } + } + else if (itr.name == new_i_col_with_default.name) + { + EXPECT_EQ(itr.column_id, new_i_col_with_default.id); + EXPECT_TRUE(itr.type->equals(*new_i_col_with_default.type)); + for (size_t i = 0; i < c->size(); i++) + { + auto value = c->getInt(i); + ASSERT_EQ(value, 5); // Should fill with default value + } + } + // Check old columns before ddl + else if (itr.name == "i8") + { + EXPECT_EQ(itr.column_id, 2L); + EXPECT_TRUE(itr.type->equals(*typeFromString("Int8"))); + for (size_t i = 0; i < c->size(); i++) + { + EXPECT_EQ(c->getInt(i), Int64(i * (-1 * (i % 2)))); + } + } + else if (itr.name == "f64") + { + EXPECT_EQ(itr.column_id, 3L); + EXPECT_TRUE(itr.type->equals(*typeFromString("Float64"))); + for (size_t i = 0; i < c->size(); i++) + { + Field value = (*c)[i]; + Float64 v = value.get(); + EXPECT_EQ(v, 0.125); + } + } + } + num_rows_read += in.rows(); + } + stream->readSuffix(); + ASSERT_EQ(num_rows_read, num_rows_write); + } +} +CATCH + +TEST_F(DMFile_DDL_Test, UpcastColumnType) +try +{ + // Prepare some data before ddl + const auto [num_rows_write, cols_before_ddl] = prepareSomeDataToDMFile(); + + // Mock that we achange a column type from int8 -> int32, and its name to "i8_new" after ddl + auto cols_after_ddl = std::make_shared(); + *cols_after_ddl = cols_before_ddl; + const ColumnDefine old_col = cols_before_ddl[3]; + ASSERT_TRUE(old_col.type->equals(*typeFromString("Int8"))); + ColumnDefine new_col = old_col; + new_col.type = typeFromString("Int32"); + new_col.name = "i32_new"; + (*cols_after_ddl)[3] = new_col; + + { + // Test read with new columns after ddl + auto stream = std::make_unique( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols_after_ddl, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); + + size_t num_rows_read = 0; + stream->readPrefix(); + while (Block in = stream->read()) + { + ASSERT_TRUE(in.has(new_col.name)); + ASSERT_TRUE(!in.has("i8")); + ASSERT_TRUE(in.has("f64")); + for (auto itr : in) + { + auto c = itr.column; + if (itr.name == new_col.name) + { + EXPECT_EQ(itr.column_id, new_col.id); + EXPECT_TRUE(itr.type->equals(*new_col.type)); + for (size_t i = 0; i < c->size(); i++) + { + auto value = c->getInt(Int64(i)); + ASSERT_EQ(value, (Int64)(i * (-1 * (i % 2)))); + } + } + // Check old columns before ddl + else if (itr.name == "f64") + { + EXPECT_EQ(itr.column_id, 3L); + EXPECT_TRUE(itr.type->equals(*typeFromString("Float64"))); + for (size_t i = 0; i < c->size(); i++) + { + Field value = (*c)[i]; + Float64 v = value.get(); + EXPECT_EQ(v, 0.125); + } + } + } + num_rows_read += in.rows(); + } + stream->readSuffix(); + ASSERT_EQ(num_rows_read, num_rows_write); + } +} +CATCH + +TEST_F(DMFile_DDL_Test, NotNullToNull) +try +{ + // Prepare some data before ddl + const auto [num_rows_write, cols_before_ddl] = prepareSomeDataToDMFile(); + + // Mock that we achange a column type from int8 -> Nullable(int32), and its name to "i8_new" after ddl + auto cols_after_ddl = std::make_shared(); + *cols_after_ddl = cols_before_ddl; + const ColumnDefine old_col = cols_before_ddl[3]; + ASSERT_TRUE(old_col.type->equals(*typeFromString("Int8"))); + ColumnDefine new_col = old_col; + new_col.type = typeFromString("Nullable(Int32)"); + new_col.name = "i32_nullable"; + (*cols_after_ddl)[3] = new_col; + + { + // Test read with new columns after ddl + auto stream = std::make_unique( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols_after_ddl, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); + + size_t num_rows_read = 0; + stream->readPrefix(); + while (Block in = stream->read()) + { + ASSERT_TRUE(in.has(new_col.name)); + ASSERT_TRUE(!in.has("i8")); + ASSERT_TRUE(in.has("f64")); + for (auto itr : in) + { + auto c = itr.column; + if (itr.name == new_col.name) + { + EXPECT_EQ(itr.column_id, new_col.id); + EXPECT_TRUE(itr.type->equals(*new_col.type)); + for (size_t i = 0; i < c->size(); i++) + { + auto value = (*c)[i]; + ASSERT_FALSE(value.isNull()); + ASSERT_EQ(value, (Int64)(i * (-1 * (i % 2)))); + } + } + // Check old columns before ddl + else if (itr.name == "f64") + { + EXPECT_EQ(itr.column_id, 3L); + EXPECT_TRUE(itr.type->equals(*typeFromString("Float64"))); + for (size_t i = 0; i < c->size(); i++) + { + Field value = (*c)[i]; + Float64 v = value.get(); + EXPECT_EQ(v, 0.125); + } + } + } + num_rows_read += in.rows(); + } + stream->readSuffix(); + ASSERT_EQ(num_rows_read, num_rows_write); + } +} +CATCH + +TEST_F(DMFile_DDL_Test, NullToNotNull) +try +{ + // Prepare some data before ddl + const auto [num_rows_write, cols_before_ddl] = prepareSomeDataToDMFile(true); + + // Mock that we achange a column type from Nullable(int8) -> int32, and its name to "i32" after ddl + auto cols_after_ddl = std::make_shared(); + *cols_after_ddl = cols_before_ddl; + const ColumnDefine old_col = cols_before_ddl[3]; + ASSERT_TRUE(old_col.type->equals(*typeFromString("Nullable(Int8)"))); + ColumnDefine new_col = old_col; + new_col.type = typeFromString("Int32"); + new_col.name = "i32"; + (*cols_after_ddl)[3] = new_col; + + { + // Test read with new columns after ddl + auto stream = std::make_unique( // + dbContext(), + std::numeric_limits::max(), + false, + dmContext().hash_salt, + dm_file, + *cols_after_ddl, + HandleRange::newAll(), + RSOperatorPtr{}, + IdSetPtr{}); + + size_t num_rows_read = 0; + stream->readPrefix(); + while (Block in = stream->read()) + { + ASSERT_TRUE(in.has(new_col.name)); + ASSERT_TRUE(!in.has("i8")); + ASSERT_TRUE(in.has("f64")); + for (auto itr : in) + { + auto c = itr.column; + if (itr.name == new_col.name) + { + EXPECT_EQ(itr.column_id, new_col.id); + EXPECT_TRUE(itr.type->equals(*new_col.type)); + for (size_t i = 0; i < c->size(); i++) + { + auto value = (*c)[i]; + if (i < num_rows_write / 2) + { + ASSERT_FALSE(value.isNull()) << " at row: " << i; + ASSERT_EQ(value, (Int64)(i * (-1 * (i % 2)))) << " at row: " << i; + } + else + { + ASSERT_FALSE(value.isNull()) << " at row: " << i; + ASSERT_EQ(value, (Int64)0) << " at row: " << i; + } + } + } + // Check old columns before ddl + else if (itr.name == "f64") + { + EXPECT_EQ(itr.column_id, 3L); + EXPECT_TRUE(itr.type->equals(*typeFromString("Float64"))); + for (size_t i = 0; i < c->size(); i++) + { + Field value = (*c)[i]; + Float64 v = value.get(); + EXPECT_EQ(v, 0.125); + } + } + } + num_rows_read += in.rows(); + } stream->readSuffix(); + ASSERT_EQ(num_rows_read, num_rows_write); } } CATCH diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp index cb60b5b793e..726ed340f58 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp @@ -45,8 +45,10 @@ class Segment_test : public ::testing::Test void SetUp() override { - db_context = std::make_unique(DMTestEnv::getContext(DB::Settings())); + db_context = std::make_unique(DMTestEnv::getContext(DB::Settings())); + table_columns_ = std::make_shared(); dropDataInDisk(); + segment = reload(); ASSERT_EQ(segment->segmentId(), DELTA_MERGE_FIRST_SEGMENT_ID); } @@ -67,7 +69,7 @@ class Segment_test : public ::testing::Test // setColumns should update dm_context at the same time void setColumns(const ColumnDefinesPtr & columns) { - table_columns_ = columns; + *table_columns_ = *columns; dm_context_ = std::make_unique(*db_context, path, @@ -834,34 +836,51 @@ try } CATCH + +enum class SegmentWriteType +{ + ToDisk, + ToCache +}; +class Segment_DDL_test : public Segment_test, // + public testing::WithParamInterface> +{ +}; +String paramToString(const ::testing::TestParamInfo & info) +{ + const auto [write_type, flush_before_ddl] = info.param; + + String name = (write_type == SegmentWriteType::ToDisk) ? "ToDisk_" : "ToCache"; + name += (flush_before_ddl ? "_FlushCache" : "_NotFlushCache"); + return name; +} + /// Mock a col from i8 -> i32 -TEST_F(Segment_test, DISABLED_DDLAlterInt8ToInt32) +TEST_P(Segment_DDL_test, AlterInt8ToInt32) try { const String column_name_i8_to_i32 = "i8_to_i32"; const ColumnID column_id_i8_to_i32 = 4; - const ColumnDefine column_i8_before_ddl(column_id_i8_to_i32, column_name_i8_to_i32, DataTypeFactory::instance().get("Int8")); - const ColumnDefine column_i32_after_ddl(column_id_i8_to_i32, column_name_i8_to_i32, DataTypeFactory::instance().get("Int32")); + const ColumnDefine column_i8_before_ddl(column_id_i8_to_i32, column_name_i8_to_i32, typeFromString("Int8")); + const ColumnDefine column_i32_after_ddl(column_id_i8_to_i32, column_name_i8_to_i32, typeFromString("Int32")); + const auto [write_type, flush_before_ddl] = GetParam(); + + // Prepare some data before ddl + const size_t num_rows_write = 100; { + /// set columns before ddl auto columns_before_ddl = DMTestEnv::getDefaultColumns(); columns_before_ddl->emplace_back(column_i8_before_ddl); - // Not cache any rows DB::Settings db_settings; - db_settings.dm_segment_delta_cache_limit_rows = 0; - segment = reload(columns_before_ddl, std::move(db_settings)); - } - const size_t num_rows_write = 100; - { - // write to segment + /// write to segment Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); - // add int8_col and later read it as int32 // (mock ddl change int8 -> int32) const size_t num_rows = block.rows(); - ColumnWithTypeAndName int8_col(column_i8_before_ddl.type, column_i8_before_ddl.name); + ColumnWithTypeAndName int8_col(nullptr, column_i8_before_ddl.type, column_i8_before_ddl.name, column_id_i8_to_i32); { IColumn::MutablePtr m_col = int8_col.type->createColumn(); auto & column_data = typeid_cast &>(*m_col).getData(); @@ -873,29 +892,31 @@ try int8_col.column = std::move(m_col); } block.insert(int8_col); - - segment->write(dmContext(), std::move(block)); + switch (write_type) + { + case SegmentWriteType::ToDisk: + segment->write(dmContext(), std::move(block)); + break; + case SegmentWriteType::ToCache: + segment->writeToCache(dmContext(), block, 0, num_rows_write); + break; + } } + ColumnDefinesPtr columns_to_read = std::make_shared(); { - ColumnDefines columns_to_read{ - column_i32_after_ddl, - }; - - BlockInputStreamPtr in; - try + *columns_to_read = *DMTestEnv::getDefaultColumns(); + columns_to_read->emplace_back(column_i32_after_ddl); + if (flush_before_ddl) { - // read written data - in = segment->getInputStream(dmContext(), *tableColumns()); + segment->flushCache(dmContext()); } - catch (const Exception & e) - { - const auto text = e.displayText(); - std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl; - std::cerr << "Stack trace:" << std::endl << e.getStackTrace().toString(); + setColumns(columns_to_read); + } - throw; - } + { + // read written data + BlockInputStreamPtr in = segment->getInputStream(dmContext(), *columns_to_read); // check that we can read correct values size_t num_rows_read = 0; @@ -903,25 +924,89 @@ try while (Block block = in->read()) { num_rows_read += block.rows(); + ASSERT_TRUE(block.has(column_name_i8_to_i32)); const ColumnWithTypeAndName & col = block.getByName(column_name_i8_to_i32); - ASSERT_TRUE(col.type->equals(*column_i32_after_ddl.type)) - << "col.type: " + col.type->getName() + " expect type: " + column_i32_after_ddl.type->getName(); + ASSERT_DATATYPE_EQ(col.type, column_i32_after_ddl.type); ASSERT_EQ(col.name, column_i32_after_ddl.name); ASSERT_EQ(col.column_id, column_i32_after_ddl.id); for (size_t i = 0; i < block.rows(); ++i) { auto value = col.column->getInt(i); const auto expected = static_cast(-1 * (i % 2 ? 1 : -1) * i); - ASSERT_EQ(value, expected); + ASSERT_EQ(value, expected) << "at row: " << i; } } in->readSuffix(); ASSERT_EQ(num_rows_read, num_rows_write); } + + + /// Write some data after ddl, replacing som origin rows + { + /// write to segment, replacing some origin rows + Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write / 2, num_rows_write * 2, false, /* tso= */ 3); + + const size_t num_rows = block.rows(); + ColumnWithTypeAndName int32_col(nullptr, column_i32_after_ddl.type, column_i32_after_ddl.name, column_id_i8_to_i32); + { + IColumn::MutablePtr m_col = int32_col.type->createColumn(); + auto & column_data = typeid_cast &>(*m_col).getData(); + column_data.resize(num_rows); + for (size_t i = 0; i < num_rows; ++i) + { + column_data[i] = static_cast(-1 * (i % 2 ? 1 : -1) * i); + } + int32_col.column = std::move(m_col); + } + block.insert(int32_col); + switch (write_type) + { + case SegmentWriteType::ToDisk: + segment->write(dmContext(), std::move(block)); + break; + case SegmentWriteType::ToCache: + segment->writeToCache(dmContext(), block, 0, num_rows); + break; + } + } + + { + // read written data + BlockInputStreamPtr in = segment->getInputStream(dmContext(), *columns_to_read); + + // check that we can read correct values + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + { + num_rows_read += block.rows(); + ASSERT_TRUE(block.has(column_name_i8_to_i32)); + const ColumnWithTypeAndName & col = block.getByName(column_name_i8_to_i32); + ASSERT_DATATYPE_EQ(col.type, column_i32_after_ddl.type); + ASSERT_EQ(col.name, column_i32_after_ddl.name); + ASSERT_EQ(col.column_id, column_i32_after_ddl.id); + for (size_t i = 0; i < block.rows(); ++i) + { + auto value = col.column->getInt(i); + auto expected = 0; + if (i < num_rows_write / 2) + expected = static_cast(-1 * (i % 2 ? 1 : -1) * i); + else + { + auto r = i - num_rows_write / 2; + expected = static_cast(-1 * (r % 2 ? 1 : -1) * r); + } + // std::cerr << " row: " << i << " "<< value << std::endl; + ASSERT_EQ(value, expected) << "at row: " << i; + } + } + in->readSuffix(); + ASSERT_EQ(num_rows_read, (size_t)(num_rows_write * 2)); + } } CATCH -TEST_F(Segment_test, DISABLED_DDLAddColumnWithDefaultValue) +TEST_P(Segment_DDL_test, AddColumn) try { const String new_column_name = "i8"; @@ -930,12 +1015,12 @@ try const Int8 new_column_default_value_int = 16; new_column_define.default_value = toField(new_column_default_value_int); + const auto [write_type, flush_before_ddl] = GetParam(); + { auto columns_before_ddl = DMTestEnv::getDefaultColumns(); // Not cache any rows DB::Settings db_settings; - db_settings.dm_segment_delta_cache_limit_rows = 0; - segment = reload(columns_before_ddl, std::move(db_settings)); } @@ -943,23 +1028,32 @@ try { // write to segment Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); - segment->write(dmContext(), std::move(block)); + switch (write_type) + { + case SegmentWriteType::ToDisk: + segment->write(dmContext(), std::move(block)); + break; + case SegmentWriteType::ToCache: + segment->writeToCache(dmContext(), block, 0, num_rows_write); + break; + } } + auto columns_after_ddl = DMTestEnv::getDefaultColumns(); { // DDL add new column with default value - auto columns_after_ddl = DMTestEnv::getDefaultColumns(); columns_after_ddl->emplace_back(new_column_define); + if (flush_before_ddl) + { + // If write to cache, before apply ddl changes (change column data type), segment->flushCache must be called. + segment->flushCache(dmContext()); + } setColumns(columns_after_ddl); } { - ColumnDefines columns_to_read{ - new_column_define, - }; - // read written data - auto in = segment->getInputStream(dmContext(), *tableColumns()); + auto in = segment->getInputStream(dmContext(), *columns_after_ddl); // check that we can read correct values size_t num_rows_read = 0; @@ -980,9 +1074,80 @@ try in->readSuffix(); ASSERT_EQ(num_rows_read, num_rows_write); } + + + /// Write some data after ddl, replacing som origin rows + { + /// write to segment, replacing some origin rows + Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write / 2, num_rows_write * 2, false, /* tso= */ 3); + + const size_t num_rows = block.rows(); + ColumnWithTypeAndName int8_col(nullptr, new_column_define.type, new_column_define.name, new_column_id); + { + IColumn::MutablePtr m_col = int8_col.type->createColumn(); + auto & column_data = typeid_cast &>(*m_col).getData(); + column_data.resize(num_rows); + for (size_t i = 0; i < num_rows; ++i) + { + column_data[i] = static_cast(-1 * (i % 2 ? 1 : -1) * i); + } + int8_col.column = std::move(m_col); + } + block.insert(int8_col); + switch (write_type) + { + case SegmentWriteType::ToDisk: + segment->write(dmContext(), std::move(block)); + break; + case SegmentWriteType::ToCache: + segment->writeToCache(dmContext(), block, 0, num_rows); + break; + } + } + + { + // read written data + BlockInputStreamPtr in = segment->getInputStream(dmContext(), *columns_after_ddl); + + // check that we can read correct values + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + { + num_rows_read += block.rows(); + ASSERT_TRUE(block.has(new_column_name)); + const ColumnWithTypeAndName & col = block.getByName(new_column_name); + ASSERT_DATATYPE_EQ(col.type, new_column_define.type); + ASSERT_EQ(col.name, new_column_define.name); + ASSERT_EQ(col.column_id, new_column_define.id); + for (size_t i = 0; i < block.rows(); ++i) + { + int8_t value = col.column->getInt(i); + int8_t expected = 0; + if (i < num_rows_write / 2) + expected = new_column_default_value_int; + else + { + auto r = i - num_rows_write / 2; + expected = static_cast(-1 * (r % 2 ? 1 : -1) * r); + } + // std::cerr << " row: " << i << " "<< value << std::endl; + ASSERT_EQ(value, expected) << "at row: " << i; + } + } + in->readSuffix(); + ASSERT_EQ(num_rows_read, (size_t)(num_rows_write * 2)); + } } CATCH +INSTANTIATE_TEST_CASE_P(SegmentWriteType, + Segment_DDL_test, + ::testing::Combine( // + ::testing::Values(SegmentWriteType::ToDisk, SegmentWriteType::ToCache), + ::testing::Bool()), + paramToString); + } // namespace tests } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index 1285cf636ed..56980d9c0be 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -53,6 +53,8 @@ class IManageableStorage : public IStorage virtual void deleteRows(const Context &, size_t /*rows*/) { throw Exception("Unsupported"); } + virtual BlockInputStreamPtr listSegments(const Context &) { throw Exception("Unsupported"); } + virtual ::TiDB::StorageEngine engineType() const = 0; virtual String getDatabaseName() const = 0; @@ -66,6 +68,10 @@ class IManageableStorage : public IStorage const AlterCommands & commands, const TiDB::TableInfo & table_info, const String & database_name, const Context & context) = 0; + + /// Remove this storage from TMTContext. Should be called after its metadata and data have been removed from disk. + virtual void removeFromTMTContext() = 0; + PKType getPKType() const { static const DataTypeInt64 & dataTypeInt64 = {}; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 97f74d530d8..b1dc0b832c0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -65,6 +65,7 @@ struct numeric_limits<__uint128_t> #include #include #include + #include namespace ProfileEvents @@ -463,7 +464,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t /// Blocking learner read. Note that learner read must be performed ahead of data read, /// otherwise the desired index will be blocked by the lock of data read. auto read_index_result = region->learnerRead(); - GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_read_index_duration_seconds).Observe(read_index_watch.elapsedSeconds()); + GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_read_index_duration_seconds) + .Observe(read_index_watch.elapsedSeconds()); if (read_index_result.region_unavailable) { @@ -479,8 +481,13 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t else { Stopwatch wait_index_watch; - region->waitIndex(read_index_result.read_index); - GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_wait_index_duration_seconds).Observe(wait_index_watch.elapsedSeconds()); + if (region->waitIndex(read_index_result.read_index, tmt.getTerminated())) + { + region_status = RegionException::RegionReadStatus::NOT_FOUND; + continue; + } + GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_wait_index_duration_seconds) + .Observe(wait_index_watch.elapsedSeconds()); } auto [block, status] = RegionTable::readBlockByRegion(*data.table_info, data.getColumns(), tmt_column_names_to_read, @@ -671,7 +678,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(const Names & column_names_t if (minmax_idx_condition && !minmax_idx_condition->mayBeTrueInRange(data.minmax_idx_columns.size(), &part->minmax_idx.min_values[0], - &part->minmax_idx.max_values[0], data.minmax_idx_column_types)) + &part->minmax_idx.max_values[0], data.minmax_idx_column_types)) continue; if (max_block_number_to_read && part->info.max_block > max_block_number_to_read) diff --git a/dbms/src/Storages/MutableSupport.cpp b/dbms/src/Storages/MutableSupport.cpp index 25947cf7579..f0fd0212f2a 100644 --- a/dbms/src/Storages/MutableSupport.cpp +++ b/dbms/src/Storages/MutableSupport.cpp @@ -6,6 +6,7 @@ namespace DB const String MutableSupport::mmt_storage_name = "MutableMergeTree"; const String MutableSupport::txn_storage_name = "TxnMergeTree"; +const String MutableSupport::delta_tree_storage_name = "DeltaMerge"; const String MutableSupport::tidb_pk_column_name = "_tidb_rowid"; const String MutableSupport::version_column_name = "_INTERNAL_VERSION"; @@ -15,4 +16,4 @@ const DataTypePtr MutableSupport::tidb_pk_column_type = DataTypeFactory::instanc const DataTypePtr MutableSupport::version_column_type = DataTypeFactory::instance().get("UInt64"); const DataTypePtr MutableSupport::delmark_column_type = DataTypeFactory::instance().get("UInt8"); -} +} // namespace DB diff --git a/dbms/src/Storages/MutableSupport.h b/dbms/src/Storages/MutableSupport.h index 293ba87a9f6..cd4a1fb65fc 100644 --- a/dbms/src/Storages/MutableSupport.h +++ b/dbms/src/Storages/MutableSupport.h @@ -26,7 +26,7 @@ class MutableSupport : public ext::singleton const OrderedNameSet & hiddenColumns(const String& table_type_name) { - if (mmt_storage_name == table_type_name || txn_storage_name == table_type_name) + if (mmt_storage_name == table_type_name || txn_storage_name == table_type_name || delta_tree_storage_name == table_type_name) return mutable_hidden; return empty; } @@ -51,6 +51,7 @@ class MutableSupport : public ext::singleton static const String mmt_storage_name; static const String txn_storage_name; + static const String delta_tree_storage_name; static const String tidb_pk_column_name; static const String version_column_name; diff --git a/dbms/src/Storages/StorageDebugging.cpp b/dbms/src/Storages/StorageDebugging.cpp index cf3965a5427..1f823016033 100644 --- a/dbms/src/Storages/StorageDebugging.cpp +++ b/dbms/src/Storages/StorageDebugging.cpp @@ -198,7 +198,10 @@ void StorageDebugging::shutdown() return; shutdown_called = true; +} +void StorageDebugging::removeFromTMTContext() +{ // remove this table from TMTContext TMTContext & tmt_context = global_context.getTMTContext(); tmt_context.getStorages().remove(tidb_table_info.id); diff --git a/dbms/src/Storages/StorageDebugging.h b/dbms/src/Storages/StorageDebugging.h index 937a7e07d72..13f0958fb6d 100644 --- a/dbms/src/Storages/StorageDebugging.h +++ b/dbms/src/Storages/StorageDebugging.h @@ -70,6 +70,7 @@ class StorageDebugging : public ext::shared_ptr_helper, public void shutdown() override; + void removeFromTMTContext() override; private: String database_name; diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 22ba2ffb1d3..e67f472b141 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -7,6 +8,7 @@ #include #endif +#include #include #include #include @@ -26,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -256,6 +259,7 @@ class DMBlockOutputStream : public IBlockOutputStream Block getHeader() const override { return header; } void write(const Block & block) override + try { if (db_settings.dm_insert_max_rows == 0) { @@ -282,6 +286,11 @@ class DMBlockOutputStream : public IBlockOutputStream } } } + catch (DB::Exception & e) + { + e.addMessage("(while writing to table `" + store->getDatabaseName() + "`.`" + store->getTableName() + "`)"); + throw; + } private: DeltaMergeStorePtr store; @@ -358,6 +367,7 @@ RegionMap doLearnerRead(const TiDB::TableID table_id, // concurrent_num = std::max(1, std::min(concurrent_num, regions_info.size())); KVStorePtr & kvstore = tmt.getKVStore(); + Context & context = tmt.getContext(); RegionMap kvstore_region; // check region is not null and store region map. for (const auto & info : regions_info) @@ -385,7 +395,7 @@ RegionMap doLearnerRead(const TiDB::TableID table_id, // if (region_status != RegionException::RegionReadStatus::OK) return; - const RegionQueryInfo & region_to_query = regions_info[region_idx]; + RegionQueryInfo & region_to_query = regions_info[region_idx]; const RegionID region_id = region_to_query.region_id; auto region = kvstore_region[region_id]; @@ -401,9 +411,14 @@ RegionMap doLearnerRead(const TiDB::TableID table_id, // return; } + GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_read_index_count).Increment(); + Stopwatch read_index_watch; + /// Blocking learner read. Note that learner read must be performed ahead of data read, /// otherwise the desired index will be blocked by the lock of data read. auto read_index_result = region->learnerRead(); + GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_read_index_duration_seconds) + .Observe(read_index_watch.elapsedSeconds()); if (read_index_result.region_unavailable) { // client-c detect region removed. Set region_status and continue. @@ -416,12 +431,36 @@ RegionMap doLearnerRead(const TiDB::TableID table_id, // continue; } else - region->waitIndex(read_index_result.read_index); - + { + Stopwatch wait_index_watch; + if (region->waitIndex(read_index_result.read_index, tmt.getTerminated())) + { + region_status = RegionException::RegionReadStatus::NOT_FOUND; + continue; + } + GET_METRIC(const_cast(context).getTiFlashMetrics(), tiflash_raft_wait_index_duration_seconds) + .Observe(wait_index_watch.elapsedSeconds()); + } if (resolve_locks) { - auto scanner = region->createCommittedScanner(); - RegionTable::resolveLocks(scanner, start_ts); + status = RegionTable::resolveLocksAndWriteRegion( // + tmt, // + table_id, // + region, // + start_ts, // + region_to_query.version, // + region_to_query.conf_version, // + region_to_query.range_in_table, log); + + if (status != RegionException::RegionReadStatus::OK) + { + LOG_WARNING(log, + "Check memory cache, region " << region_id << ", version " << region_to_query.version << ", handle range [" + << region_to_query.range_in_table.first.toString() << ", " + << region_to_query.range_in_table.second.toString() << ") , status " + << RegionException::RegionReadStatusString(status)); + region_status = status; + } } } }; @@ -494,6 +533,62 @@ RegionMap doLearnerRead(const TiDB::TableID table_id, // } // namespace +std::unordered_set parseSegmentSet(const ASTPtr & ast) +{ + if (!ast) + return {}; + const auto & partition_ast = typeid_cast(*ast); + + if (!partition_ast.value) + return {parse(partition_ast.id)}; + + auto parse_segment_id = [](const ASTLiteral * literal) -> std::pair { + if (!literal) + return {false, 0}; + switch (literal->value.getType()) + { + case Field::Types::String: + return {true, parse(literal->value.get())}; + case Field::Types::UInt64: + return {true, literal->value.get()}; + default: + return {false, 0}; + } + }; + + { + const auto * partition_lit = typeid_cast(partition_ast.value.get()); + auto [suc, id] = parse_segment_id(partition_lit); + if (suc) + return {id}; + } + + const auto * partition_function = typeid_cast(partition_ast.value.get()); + if (partition_function && partition_function->name == "tuple") + { + std::unordered_set ids; + bool ok = true; + for (const auto & item : partition_function->arguments->children) + { + const auto * partition_lit = typeid_cast(item.get()); + auto [suc, id] = parse_segment_id(partition_lit); + if (suc) + { + ids.emplace(id); + } + else + { + ok = false; + break; + } + } + if (ok) + return ids; + } + + throw Exception("Unable to parse segment IDs in literal form: `" + partition_ast.fields_str.toString() + "`"); +} + BlockInputStreams StorageDeltaMerge::read( // const Names & column_names, const SelectQueryInfo & query_info, @@ -530,7 +625,8 @@ BlockInputStreams StorageDeltaMerge::read( // const ASTSelectQuery & select_query = typeid_cast(*query_info.query); if (select_query.raw_for_mutable) - return store->readRaw(context, context.getSettingsRef(), columns_to_read, num_streams); + return store->readRaw( + context, context.getSettingsRef(), columns_to_read, num_streams, parseSegmentSet(select_query.segment_expression_list)); else { if (unlikely(!query_info.mvcc_query_info)) @@ -650,7 +746,7 @@ BlockInputStreams StorageDeltaMerge::read( // auto streams = store->read(context, context.getSettingsRef(), columns_to_read, ranges, num_streams, - /*max_version=*/mvcc_query_info.read_tso, rs_operator, max_block_size); + /*max_version=*/mvcc_query_info.read_tso, rs_operator, max_block_size, parseSegmentSet(select_query.segment_expression_list)); { /// Ensure read_tso and regions' info after read. @@ -848,6 +944,8 @@ void StorageDeltaMerge::alterImpl(const AlterCommands & commands, setColumns(std::move(new_columns)); } +String StorageDeltaMerge::getName() const { return MutableSupport::delta_tree_storage_name; } + void StorageDeltaMerge::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) { const String new_path = new_path_to_db + "/" + new_table_name; @@ -925,8 +1023,8 @@ void updateDeltaMergeTableCreateStatement( // else if (args.children.size() == 2) args.children.back() = literal; else - throw Exception( - "Wrong arguments num:" + DB::toString(args.children.size()) + " in table: " + table_name + " with engine=DeltaMerge", + throw Exception("Wrong arguments num:" + DB::toString(args.children.size()) + " in table: " + table_name + + " with engine=" + MutableSupport::delta_tree_storage_name, ErrorCodes::BAD_ARGUMENTS); }; @@ -1033,11 +1131,15 @@ void StorageDeltaMerge::startup() void StorageDeltaMerge::shutdown() { - if (shutdown_called) + bool v = false; + if (!shutdown_called.compare_exchange_strong(v, true)) return; - shutdown_called = true; + store->shutdown(); +} +void StorageDeltaMerge::removeFromTMTContext() +{ // remove this table from TMTContext TMTContext & tmt_context = global_context.getTMTContext(); tmt_context.getStorages().remove(tidb_table_info.id); diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 25e573a58d6..ac39aaf7acc 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -30,7 +30,7 @@ class StorageDeltaMerge : public ext::shared_ptr_helper, publ bool supportsModification() const override { return true; } - String getName() const override { return "DeltaMerge"; } + String getName() const override; String getTableName() const override { return table_name; } void drop() override; @@ -73,6 +73,8 @@ class StorageDeltaMerge : public ext::shared_ptr_helper, publ void shutdown() override; + void removeFromTMTContext() override; + SortDescription getPrimarySortDescription() const override; const OrderedNameSet & getHiddenColumnsImpl() const override { return hidden_columns; } @@ -82,6 +84,8 @@ class StorageDeltaMerge : public ext::shared_ptr_helper, publ void checkStatus(const Context & context) override; void deleteRows(const Context &, size_t rows) override; + const DM::DeltaMergeStorePtr & getStore() { return store; } + protected: StorageDeltaMerge(const String & path_, const String & db_name_, diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 92e98d577ac..dedc2270ba5 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -33,7 +33,7 @@ class StorageMergeTree : public ext::shared_ptr_helper, public public: void startup() override; void shutdown() override; - void removeFromTMTContext(); + void removeFromTMTContext() override; ~StorageMergeTree() override; std::string getName() const override { return data.merging_params.getModeName() + "MergeTree"; } diff --git a/dbms/src/Storages/System/StorageSystemDTSegment.cpp b/dbms/src/Storages/System/StorageSystemDTSegment.cpp new file mode 100644 index 00000000000..3c73ceda92e --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemDTSegment.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +StorageSystemDTSegments::StorageSystemDTSegments(const std::string & name_) : name(name_) +{ + setColumns(ColumnsDescription({ + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"table_id", std::make_shared()}, + + {"segment_id", std::make_shared()}, + {"range", std::make_shared()}, + + {"rows", std::make_shared()}, + {"size", std::make_shared()}, + {"delete_ranges", std::make_shared()}, + + {"delta_pack_count", std::make_shared()}, + {"stable_pack_count", std::make_shared()}, + + {"avg_delta_pack_rows", std::make_shared()}, + {"avg_stable_pack_rows", std::make_shared()}, + + {"delta_rate", std::make_shared()}, + {"delta_cache_size", std::make_shared()}, + })); +} + +BlockInputStreams StorageSystemDTSegments::read(const Names & column_names, + const SelectQueryInfo &, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + check(column_names); + processed_stage = QueryProcessingStage::FetchColumns; + + MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); + + auto databases = context.getDatabases(); + for (const auto & d : databases) + { + String database_name = d.first; + auto & database = d.second; + auto it = database->getIterator(context); + for (; it->isValid(); it->next()) + { + auto & table_name = it->name(); + auto & storage = it->table(); + if (storage->getName() != MutableSupport::delta_tree_storage_name) + continue; + + auto dm_storage = std::dynamic_pointer_cast(storage); + auto table_id = dm_storage->getTableInfo().id; + auto segment_stats = dm_storage->getStore()->getSegmentStats(); + for (auto & stat : segment_stats) + { + res_columns[0]->insert(database_name); + res_columns[1]->insert(table_name); + res_columns[2]->insert(table_id); + + res_columns[3]->insert(stat.segment_id); + res_columns[4]->insert(stat.range.toString()); + res_columns[5]->insert(stat.rows); + res_columns[6]->insert(stat.size); + res_columns[7]->insert(stat.delete_ranges); + + res_columns[8]->insert(stat.delta_pack_count); + res_columns[9]->insert(stat.stable_pack_count); + + res_columns[10]->insert(stat.avg_delta_pack_rows); + res_columns[11]->insert(stat.avg_stable_pack_rows); + + res_columns[12]->insert(stat.delta_rate); + res_columns[13]->insert(stat.delta_cache_size); + } + } + } + + return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); +} + + +} // namespace DB diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.h b/dbms/src/Storages/System/StorageSystemDTSegments.h new file mode 100644 index 00000000000..b8b46c4c007 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemDTSegments.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemDTSegments : public ext::shared_ptr_helper, public IStorage +{ +public: + std::string getName() const override { return "SystemDTSegments"; } + std::string getTableName() const override { return name; } + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + const std::string name; + +protected: + StorageSystemDTSegments(const std::string & name_); +}; + +} diff --git a/dbms/src/Storages/System/StorageSystemDTTable.cpp b/dbms/src/Storages/System/StorageSystemDTTable.cpp new file mode 100644 index 00000000000..e64fad690e5 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemDTTable.cpp @@ -0,0 +1,178 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +StorageSystemDTTables::StorageSystemDTTables(const std::string & name_) : name(name_) +{ + setColumns(ColumnsDescription({ + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"table_id", std::make_shared()}, + + {"segment_count", std::make_shared()}, + + {"total_rows", std::make_shared()}, + {"total_size", std::make_shared()}, + {"total_delete_ranges", std::make_shared()}, + + {"delta_rate_rows", std::make_shared()}, + {"delta_rate_segments", std::make_shared()}, + + {"delta_placed_rate", std::make_shared()}, + {"delta_cache_size", std::make_shared()}, + {"delta_cache_rate", std::make_shared()}, + {"delta_cache_wasted_rate", std::make_shared()}, + + {"avg_segment_rows", std::make_shared()}, + {"avg_segment_size", std::make_shared()}, + + {"delta_count", std::make_shared()}, + {"total_delta_rows", std::make_shared()}, + {"total_delta_size", std::make_shared()}, + {"avg_delta_rows", std::make_shared()}, + {"avg_delta_size", std::make_shared()}, + {"avg_delta_delete_ranges", std::make_shared()}, + + {"stable_count", std::make_shared()}, + {"total_stable_rows", std::make_shared()}, + {"total_stable_size", std::make_shared()}, + {"avg_stable_rows", std::make_shared()}, + {"avg_stable_size", std::make_shared()}, + + {"total_pack_count_in_delta", std::make_shared()}, + {"avg_pack_count_in_delta", std::make_shared()}, + {"avg_pack_rows_in_delta", std::make_shared()}, + {"avg_pack_size_in_delta", std::make_shared()}, + + {"total_pack_count_in_stable", std::make_shared()}, + {"avg_pack_count_in_stable", std::make_shared()}, + {"avg_pack_rows_in_stable", std::make_shared()}, + {"avg_pack_size_in_stable", std::make_shared()}, + + {"storage_stable_num_snapshots", std::make_shared()}, + {"storage_stable_num_pages", std::make_shared()}, + {"storage_stable_num_normal_pages", std::make_shared()}, + {"storage_stable_max_page_id", std::make_shared()}, + + {"storage_delta_num_snapshots", std::make_shared()}, + {"storage_delta_num_pages", std::make_shared()}, + {"storage_delta_num_normal_pages", std::make_shared()}, + {"storage_delta_max_page_id", std::make_shared()}, + + {"storage_meta_num_snapshots", std::make_shared()}, + {"storage_meta_num_pages", std::make_shared()}, + {"storage_meta_num_normal_pages", std::make_shared()}, + {"storage_meta_max_page_id", std::make_shared()}, + + {"background_tasks_length", std::make_shared()}, + })); +} + + +BlockInputStreams StorageSystemDTTables::read(const Names & column_names, + const SelectQueryInfo &, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + check(column_names); + processed_stage = QueryProcessingStage::FetchColumns; + + MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); + + auto databases = context.getDatabases(); + for (const auto & d : databases) + { + String database_name = d.first; + auto & database = d.second; + auto it = database->getIterator(context); + for (; it->isValid(); it->next()) + { + auto & table_name = it->name(); + auto & storage = it->table(); + if (storage->getName() != MutableSupport::delta_tree_storage_name) + continue; + + auto dm_storage = std::dynamic_pointer_cast(storage); + auto table_id = dm_storage->getTableInfo().id; + auto stat = dm_storage->getStore()->getStat(); + + res_columns[0]->insert(database_name); + res_columns[1]->insert(table_name); + res_columns[2]->insert(table_id); + + res_columns[3]->insert(stat.segment_count); + + res_columns[4]->insert(stat.total_rows); + res_columns[5]->insert(stat.total_size); + res_columns[6]->insert(stat.total_delete_ranges); + + res_columns[7]->insert(stat.delta_rate_rows); + res_columns[8]->insert(stat.delta_rate_segments); + + res_columns[9]->insert(stat.delta_placed_rate); + res_columns[10]->insert(stat.delta_cache_size); + res_columns[11]->insert(stat.delta_cache_rate); + res_columns[12]->insert(stat.delta_cache_wasted_rate); + + res_columns[13]->insert(stat.avg_segment_rows); + res_columns[14]->insert(stat.avg_segment_size); + + res_columns[15]->insert(stat.delta_count); + res_columns[16]->insert(stat.total_delta_rows); + res_columns[17]->insert(stat.total_delta_size); + res_columns[18]->insert(stat.avg_delta_rows); + res_columns[19]->insert(stat.avg_delta_size); + res_columns[20]->insert(stat.avg_delta_delete_ranges); + + res_columns[21]->insert(stat.stable_count); + res_columns[22]->insert(stat.total_stable_rows); + res_columns[23]->insert(stat.total_stable_size); + res_columns[24]->insert(stat.avg_stable_rows); + res_columns[25]->insert(stat.avg_stable_size); + + res_columns[26]->insert(stat.total_pack_count_in_delta); + res_columns[27]->insert(stat.avg_pack_count_in_delta); + res_columns[28]->insert(stat.avg_pack_rows_in_delta); + res_columns[29]->insert(stat.avg_pack_size_in_delta); + + res_columns[30]->insert(stat.total_pack_count_in_stable); + res_columns[31]->insert(stat.avg_pack_count_in_stable); + res_columns[32]->insert(stat.avg_pack_rows_in_stable); + res_columns[33]->insert(stat.avg_pack_size_in_stable); + + res_columns[34]->insert(stat.storage_stable_num_snapshots); + res_columns[35]->insert(stat.storage_stable_num_pages); + res_columns[36]->insert(stat.storage_stable_num_normal_pages); + res_columns[37]->insert(stat.storage_stable_max_page_id); + + res_columns[38]->insert(stat.storage_delta_num_snapshots); + res_columns[39]->insert(stat.storage_delta_num_pages); + res_columns[40]->insert(stat.storage_delta_num_normal_pages); + res_columns[41]->insert(stat.storage_delta_max_page_id); + + res_columns[42]->insert(stat.storage_meta_num_snapshots); + res_columns[43]->insert(stat.storage_meta_num_pages); + res_columns[44]->insert(stat.storage_meta_num_normal_pages); + res_columns[45]->insert(stat.storage_meta_max_page_id); + + res_columns[46]->insert(stat.background_tasks_length); + } + } + + return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); +} + + +} // namespace DB diff --git a/dbms/src/Storages/System/StorageSystemDTTables.h b/dbms/src/Storages/System/StorageSystemDTTables.h new file mode 100644 index 00000000000..215d05c52c5 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemDTTables.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemDTTables : public ext::shared_ptr_helper, public IStorage +{ +public: + std::string getName() const override { return "SystemDTTables"; } + std::string getTableName() const override { return name; } + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + const std::string name; + +protected: + StorageSystemDTTables(const std::string & name_); +}; + +} diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index e84762778f2..21a97387089 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -1,24 +1,27 @@ #include #include -#include -#include +#include #include -#include -#include +#include +#include +#include #include #include #include #include -#include -#include - +#include +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int CANNOT_GET_CREATE_TABLE_QUERY; +extern const int CANNOT_GET_CREATE_TABLE_QUERY; } /// Some virtual columns routines @@ -52,7 +55,8 @@ NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const Strin struct VirtualColumnsProcessor { explicit VirtualColumnsProcessor(const ColumnsWithTypeAndName & all_virtual_columns_) - : all_virtual_columns(all_virtual_columns_), virtual_columns_mask(all_virtual_columns_.size(), 0) {} + : all_virtual_columns(all_virtual_columns_), virtual_columns_mask(all_virtual_columns_.size(), 0) + {} /// Separates real and virtual column names, returns real ones Names process(const Names & column_names, const std::vector & virtual_columns_exists_flag = {}) @@ -108,28 +112,23 @@ struct VirtualColumnsProcessor std::vector virtual_columns_mask; }; -} +} // namespace -StorageSystemTables::StorageSystemTables(const std::string & name_) - : name(name_) +StorageSystemTables::StorageSystemTables(const std::string & name_) : name(name_) { - setColumns(ColumnsDescription( - { + setColumns(ColumnsDescription({ {"database", std::make_shared()}, {"name", std::make_shared()}, {"engine", std::make_shared()}, + {"tidb_table_id", std::make_shared()}, {"is_temporary", std::make_shared()}, {"data_path", std::make_shared()}, {"metadata_path", std::make_shared()}, })); - virtual_columns = - { - {std::make_shared(), "metadata_modification_time"}, - {std::make_shared(), "create_table_query"}, - {std::make_shared(), "engine_full"} - }; + virtual_columns = {{std::make_shared(), "metadata_modification_time"}, + {std::make_shared(), "create_table_query"}, {std::make_shared(), "engine_full"}}; } @@ -139,14 +138,13 @@ static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & cont for (const auto & db : context.getDatabases()) column->insert(db.first); - Block block { ColumnWithTypeAndName( std::move(column), std::make_shared(), "database" ) }; + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; VirtualColumnUtils::filterBlockWithQuery(query, block, context); return block.getByPosition(0).column; } -BlockInputStreams StorageSystemTables::read( - const Names & column_names, +BlockInputStreams StorageSystemTables::read(const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum & processed_stage, @@ -161,7 +159,8 @@ BlockInputStreams StorageSystemTables::read( bool has_engine_full = false; VirtualColumnsProcessor virtual_columns_processor(virtual_columns); - real_column_names = virtual_columns_processor.process(column_names, {&has_metadata_modification_time, &has_create_table_query, &has_engine_full}); + real_column_names + = virtual_columns_processor.process(column_names, {&has_metadata_modification_time, &has_create_table_query, &has_engine_full}); check(real_column_names); Block res_block = getSampleBlock(); @@ -190,7 +189,18 @@ BlockInputStreams StorageSystemTables::read( size_t j = 0; res_columns[j++]->insert(database_name); res_columns[j++]->insert(table_name); - res_columns[j++]->insert(iterator->table()->getName()); + const String engine_name = iterator->table()->getName(); + res_columns[j++]->insert(engine_name); + TableID table_id = -1; + if (engine_name == MutableSupport::txn_storage_name || engine_name == MutableSupport::delta_tree_storage_name) + { + auto managed_storage = std::dynamic_pointer_cast(iterator->table()); + if (managed_storage) + { + table_id = managed_storage->getTableInfo().id; + } + } + res_columns[j++]->insert(Int64(table_id)); res_columns[j++]->insert(UInt64(0)); res_columns[j++]->insert(iterator->table()->getDataPath()); res_columns[j++]->insert(database->getTableMetadataPath(table_name)); @@ -268,4 +278,4 @@ NameAndTypePair StorageSystemTables::getColumn(const String & column_name) const return !virtual_column.name.empty() ? virtual_column : ITableDeclaration::getColumn(column_name); } -} +} // namespace DB diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 93d4809b3c7..adf2b14db12 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -35,6 +37,8 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("numbers", StorageSystemNumbers::create("numbers", false)); system_database.attachTable("numbers_mt", StorageSystemNumbers::create("numbers_mt", true)); system_database.attachTable("databases", StorageSystemDatabases::create("databases")); + system_database.attachTable("dt_tables", StorageSystemDTTables::create("dt_tables")); + system_database.attachTable("dt_segments", StorageSystemDTSegments::create("dt_segments")); system_database.attachTable("tables", StorageSystemTables::create("tables")); system_database.attachTable("columns", StorageSystemColumns::create("columns")); system_database.attachTable("functions", StorageSystemFunctions::create("functions")); diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 9272c66ba0b..332904bd1b1 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -1,7 +1,7 @@ #include #include -#include #include +#include #include #include #include @@ -144,8 +144,8 @@ static const metapb::Peer & findPeer(const metapb::Region & region, UInt64 peer_ throw Exception(std::string(__PRETTY_FUNCTION__) + ": peer " + DB::toString(peer_id) + " not found", ErrorCodes::LOGICAL_ERROR); } -void KVStore::handleApplySnapshot(metapb::Region && region, UInt64 peer_id, const SnapshotDataView & lock_buff, - const SnapshotDataView & write_buff, const SnapshotDataView & default_buff, UInt64 index, UInt64 term, TMTContext & tmt) +void KVStore::handleApplySnapshot( + metapb::Region && region, UInt64 peer_id, const SnapshotViewArray snaps, UInt64 index, UInt64 term, TMTContext & tmt) { auto meta = ({ auto peer = findPeer(region, peer_id); @@ -163,24 +163,23 @@ void KVStore::handleApplySnapshot(metapb::Region && region, UInt64 peer_id, cons LOG_INFO(log, "Try to apply snapshot: " << new_region->toString(true)); { - struct CfData - { - ColumnFamilyType type; - const SnapshotDataView & data; - }; - CfData cf_data[3] - = {{ColumnFamilyType::Lock, (lock_buff)}, {ColumnFamilyType::Default, (default_buff)}, {ColumnFamilyType::Write, (write_buff)}}; - for (auto i = 0; i < 3; ++i) + std::stringstream ss; + + for (UInt64 i = 0; i < snaps.len; ++i) { - for (UInt64 n = 0; n < cf_data[i].data.len; ++n) + auto & snapshot = snaps.views[i]; + for (UInt64 n = 0; n < snapshot.len; ++n) { - auto & k = cf_data[i].data.keys[n]; - auto & v = cf_data[i].data.vals[n]; - auto key = std::string(k.data, k.len); - auto value = std::string(v.data, v.len); - new_region->insert(cf_data[i].type, TiKVKey(std::move(key)), TiKVValue(std::move(value))); + auto & k = snapshot.keys[n]; + auto & v = snapshot.vals[n]; + new_region->insert(snapshot.cf, TiKVKey(k.data, k.len), TiKVValue(v.data, v.len)); } + + ss << "[cf: " << CFToName(snapshot.cf) << ", kv size: " << snapshot.len << "]; "; } + + if (snaps.len) + LOG_INFO(log, "Insert snapshot " << ss.str()); } new_region->tryPreDecodeTiKVValue(tmt); @@ -190,4 +189,38 @@ void KVStore::handleApplySnapshot(metapb::Region && region, UInt64 peer_id, cons LOG_INFO(log, new_region->toString(false) << " apply snapshot " << (status ? "success" : "fail")); } +void KVStore::handleIngestSST(UInt64 region_id, const SnapshotViewArray snaps, UInt64 index, UInt64 term, TMTContext & tmt) +{ + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + + const RegionPtr region = getRegion(region_id); + + if (region == nullptr) + throw Exception(std::string(__PRETTY_FUNCTION__) + ": region " + std::to_string(region_id) + " is not found"); + + const auto func_try_flush = [&]() { + if (!region->writeCFCount()) + return; + try + { + tmt.getRegionTable().tryFlushRegion(region, false); + tryFlushRegionCacheInStorage(tmt, *region, log); + } + catch (Exception & e) + { + // sst of write cf may be ingested first, exception may be raised because there is no matched data in default cf. + // ignore it. + LOG_DEBUG(log, __FUNCTION__ << ": catch but ignore exception: " << e.message()); + } + }; + + // try to flush remain data in memory. + func_try_flush(); + region->handleIngestSST(snaps, index, term); + region->tryPreDecodeTiKVValue(tmt); + func_try_flush(); + + region_persister.persist(*region, region_task_lock); +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index af5810caf11..0256549b94b 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -1,6 +1,6 @@ #include -#include #include +#include #include #include #include @@ -198,11 +198,12 @@ void KVStore::mockRemoveRegion(const DB::RegionID region_id, RegionTable & regio { auto task_lock = genTaskLock(); auto region_lock = region_manager.genRegionTaskLock(region_id); - removeRegion(region_id, region_table, task_lock, region_lock); + // mock remove region should remove data by default + removeRegion(region_id, /* remove_data */ true, region_table, task_lock, region_lock); } -void KVStore::removeRegion( - const RegionID region_id, RegionTable & region_table, const KVStoreTaskLock & task_lock, const RegionTaskLock & region_lock) +void KVStore::removeRegion(const RegionID region_id, bool remove_data, RegionTable & region_table, const KVStoreTaskLock & task_lock, + const RegionTaskLock & region_lock) { LOG_INFO(log, "Start to remove [region " << region_id << "]"); @@ -220,7 +221,7 @@ void KVStore::removeRegion( region_persister.drop(region_id, region_lock); - region_table.removeRegion(region_id); + region_table.removeRegion(region_id, remove_data); LOG_INFO(log, "Remove [region " << region_id << "] done"); } @@ -280,35 +281,17 @@ TiFlashApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 r return TiFlashApplyRes::NotFound; } - /// If isBgFlushDisabled = true, then only set region's applied index and term after region is flushed. - - bool is_bg_flush_disabled = tmt.isBgFlushDisabled(); const auto ori_size = region->dataSize(); - auto res = region->handleWriteRaftCmd(cmds, index, term, /* set_applied */ !is_bg_flush_disabled); + auto res = region->handleWriteRaftCmd(cmds, index, term, tmt); { tmt.getRegionTable().updateRegion(*region); - if (region->dataSize() != ori_size) + if (region->dataSize() != ori_size && !tmt.isBgFlushDisabled()) { - if (is_bg_flush_disabled) - { - // Decode data in region and then flush - region->tryPreDecodeTiKVValue(tmt); - tmt.getRegionTable().tryFlushRegion(region, false); - } - else - { - tmt.getBackgroundService().addRegionToDecode(region); - } + tmt.getBackgroundService().addRegionToDecode(region); } } - if (is_bg_flush_disabled) - { - region->setApplied(index, term); - region->notifyApplied(); - } - return res; } @@ -323,7 +306,7 @@ void KVStore::handleDestroy(UInt64 region_id, TMTContext & tmt) } LOG_INFO(log, "Handle destroy " << region->toString()); region->setPendingRemove(); - removeRegion(region_id, tmt.getRegionTable(), task_lock, region_manager.genRegionTaskLock(region_id)); + removeRegion(region_id, /* remove_data */ true, tmt.getRegionTable(), task_lock, region_manager.genRegionTaskLock(region_id)); } void KVStore::setRegionCompactLogPeriod(Seconds period) { REGION_COMPACT_LOG_PERIOD = period; } @@ -457,7 +440,10 @@ TiFlashApplyRes KVStore::handleAdminRaftCmd(raft_cmdpb::AdminRequest && request, const auto handle_change_peer = [&]() { if (curr_region.isPendingRemove()) - removeRegion(curr_region_id, region_table, task_lock, region_task_lock); + { + // remove `curr_region` from this node, we can remove its data. + removeRegion(curr_region_id, /* remove_data */ true, region_table, task_lock, region_task_lock); + } else persist_and_sync(); }; @@ -470,7 +456,9 @@ TiFlashApplyRes KVStore::handleAdminRaftCmd(raft_cmdpb::AdminRequest && request, { auto source_region = getRegion(source_region_id); source_region->setPendingRemove(); - removeRegion(source_region_id, region_table, task_lock, region_manager.genRegionTaskLock(source_region_id)); + // `source_region` is merged, don't remove its data in storage. + removeRegion( + source_region_id, /* remove_data */ false, region_table, task_lock, region_manager.genRegionTaskLock(source_region_id)); } region_range_index.remove(result.ori_region_range->comparableKeys(), curr_region_id); region_range_index.add(curr_region_ptr); @@ -489,7 +477,9 @@ TiFlashApplyRes KVStore::handleAdminRaftCmd(raft_cmdpb::AdminRequest && request, "Admin cmd " << raft_cmdpb::AdminCmdType_Name(type) << " has been applied, try to remove source " << source_region->toString(false)); source_region->setPendingRemove(); - removeRegion(source_region->id(), region_table, task_lock, region_manager.genRegionTaskLock(source_region->id())); + // `source_region` is merged, don't remove its data in storage. + removeRegion(source_region->id(), /* remove_data */ false, region_table, task_lock, + region_manager.genRegionTaskLock(source_region->id())); } } break; diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index fe592b266ab..b2063681983 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -30,7 +30,7 @@ struct TiKVRangeKey; class TMTContext; -struct SnapshotDataView; +struct SnapshotViewArray; struct WriteCmdsView; enum TiFlashApplyRes : uint32_t; @@ -69,18 +69,26 @@ class KVStore final : private boost::noncopyable TiFlashApplyRes handleWriteRaftCmd( raft_cmdpb::RaftCmdRequest && request, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); TiFlashApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); - void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, const SnapshotDataView & lock_buff, - const SnapshotDataView & write_buff, const SnapshotDataView & default_buff, uint64_t index, uint64_t term, TMTContext & tmt); + void handleApplySnapshot( + metapb::Region && region, uint64_t peer_id, const SnapshotViewArray snaps, uint64_t index, uint64_t term, TMTContext & tmt); bool tryApplySnapshot(RegionPtr new_region, Context & context, bool try_flush_region); void handleDestroy(UInt64 region_id, TMTContext & tmt); void setRegionCompactLogPeriod(Seconds period); + void handleIngestSST(UInt64 region_id, const SnapshotViewArray snaps, UInt64 index, UInt64 term, TMTContext & tmt); private: friend class MockTiDB; friend struct MockTiDBTable; friend void dbgFuncRemoveRegion(Context &, const ASTs &, /*DBGInvoker::Printer*/ std::function); - void removeRegion( - const RegionID region_id, RegionTable & region_table, const KVStoreTaskLock & task_lock, const RegionTaskLock & region_lock); + + // Remove region from this TiFlash node. + // If region is destroy or moved to another node(change peer), + // set `remove_data` true to remove obsolete data from storage. + void removeRegion(const RegionID region_id, + bool remove_data, + RegionTable & region_table, + const KVStoreTaskLock & task_lock, + const RegionTaskLock & region_lock); void mockRemoveRegion(const RegionID region_id, RegionTable & region_table); KVStoreTaskLock genTaskLock() const; diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 50eeafe6bc6..38a5930891e 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -22,38 +22,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } // namespace ErrorCodes -void RegionTable::writeBlockByRegion(Context & context, RegionPtr region, RegionDataReadInfoList & data_list_to_remove, Logger * log) +void writeRegionDataToStorage(Context & context, const RegionPtr & region, RegionDataReadInfoList & data_list_read, Logger * log) { const auto & tmt = context.getTMTContext(); TableID table_id = region->getMappedTableID(); - UInt64 region_read_cost = -1, region_decode_cost = -1, write_part_cost = -1; - - RegionDataReadInfoList data_list_read; - { - auto scanner = region->createCommittedScanner(); - - /// Some sanity checks for region meta. - { - if (region->isPendingRemove()) - return; - } - - /// Read raw KVs from region cache. - { - // Shortcut for empty region. - if (!scanner.hasNext()) - return; - - data_list_read.reserve(scanner.writeMapSize()); - - auto start_time = Clock::now(); - do - { - data_list_read.emplace_back(scanner.next()); - } while (scanner.hasNext()); - region_read_cost = std::chrono::duration_cast(Clock::now() - start_time).count(); - } - } + UInt64 region_decode_cost = -1, write_part_cost = -1; /// Declare lambda of atomic read then write to call multiple times. auto atomicReadWrite = [&](bool force_decode) { @@ -65,7 +38,6 @@ void RegionTable::writeBlockByRegion(Context & context, RegionPtr region, Region return false; // Table must have just been dropped or truncated. // TODO: What if we support delete range? Do we still want to remove KVs from region cache? - data_list_to_remove = std::move(data_list_read); return true; } @@ -123,9 +95,6 @@ void RegionTable::writeBlockByRegion(Context & context, RegionPtr region, Region } write_part_cost = std::chrono::duration_cast(Clock::now() - start_time).count(); - /// Move read data to outer to remove. - data_list_to_remove = std::move(data_list_read); - return true; }; @@ -147,24 +116,19 @@ void RegionTable::writeBlockByRegion(Context & context, RegionPtr region, Region } LOG_TRACE(log, - __FUNCTION__ << ": table " << table_id << ", region " << region->id() << ", cost [region read " << region_read_cost - << ", region decode " << region_decode_cost << ", write part " << write_part_cost << "] ms"); + __FUNCTION__ << ": table " << table_id << ", region " << region->id() << ", cost [region decode " << region_decode_cost + << ", write part " << write_part_cost << "] ms"); } -std::tuple RegionTable::readBlockByRegion(const TiDB::TableInfo & table_info, - const ColumnsDescription & columns, - const Names & column_names_to_read, +std::pair resolveLocksAndReadRegionData(const TiDB::TableID table_id, const RegionPtr & region, + const Timestamp start_ts, RegionVersion region_version, RegionVersion conf_version, - bool resolve_locks, - Timestamp start_ts, DB::HandleRange & handle_range, - RegionScanFilterPtr scan_filter) + bool resolve_locks, + bool need_data_value) { - if (!region) - throw Exception(std::string(__PRETTY_FUNCTION__) + ": region is null", ErrorCodes::LOGICAL_ERROR); - RegionDataReadInfoList data_list_read; { auto scanner = region->createCommittedScanner(); @@ -172,38 +136,112 @@ std::tuple RegionTable::readBlockByReg /// Some sanity checks for region meta. { if (region->isPendingRemove()) - return {Block(), RegionException::PENDING_REMOVE}; + return {{}, RegionException::PENDING_REMOVE}; const auto & [version, conf_ver, key_range] = region->dumpVersionRange(); if (version != region_version || conf_ver != conf_version) - return {Block(), RegionException::VERSION_ERROR}; + return {{}, RegionException::VERSION_ERROR}; - handle_range = key_range->getHandleRangeByTable(table_info.id); + handle_range = key_range->getHandleRangeByTable(table_id); } /// Deal with locks. if (resolve_locks) { - resolveLocks(scanner, start_ts); + /// Check if there are any lock should be resolved, if so, throw LockException. + if (LockInfoPtr lock_info = scanner.getLockInfo(start_ts); lock_info) + { + LockInfos lock_infos; + lock_infos.emplace_back(std::move(lock_info)); + throw LockException(std::move(lock_infos)); + } } /// Read raw KVs from region cache. { // Shortcut for empty region. if (!scanner.hasNext()) - return {Block(), RegionException::OK}; + return {{}, RegionException::OK}; data_list_read.reserve(scanner.writeMapSize()); // Tiny optimization for queries that need only handle, tso, delmark. - bool need_value = column_names_to_read.size() != 3; do { - data_list_read.emplace_back(scanner.next(need_value)); + data_list_read.emplace_back(scanner.next(need_data_value)); + } while (scanner.hasNext()); + } + } + return {std::move(data_list_read), RegionException::OK}; +} + +void RegionTable::writeBlockByRegion( + Context & context, const RegionPtr & region, RegionDataReadInfoList & data_list_to_remove, Logger * log, bool lock_region) +{ + RegionDataReadInfoList data_list_read; + { + auto scanner = region->createCommittedScanner(lock_region); + + /// Some sanity checks for region meta. + { + if (region->isPendingRemove()) + return; + } + + /// Read raw KVs from region cache. + { + // Shortcut for empty region. + if (!scanner.hasNext()) + return; + + data_list_read.reserve(scanner.writeMapSize()); + + do + { + data_list_read.emplace_back(scanner.next()); } while (scanner.hasNext()); } } + writeRegionDataToStorage(context, region, data_list_read, log); + + /// Remove data in region. + { + auto remover = region->createCommittedRemover(lock_region); + for (const auto & [handle, write_type, commit_ts, value] : data_list_read) + { + std::ignore = write_type; + std::ignore = value; + + remover.remove({handle, commit_ts}); + } + } + + /// Save removed data to outer. + data_list_to_remove = std::move(data_list_read); +} + +std::tuple RegionTable::readBlockByRegion(const TiDB::TableInfo & table_info, + const ColumnsDescription & columns, + const Names & column_names_to_read, + const RegionPtr & region, + RegionVersion region_version, + RegionVersion conf_version, + bool resolve_locks, + Timestamp start_ts, + DB::HandleRange & handle_range, + RegionScanFilterPtr scan_filter) +{ + if (!region) + throw Exception(std::string(__PRETTY_FUNCTION__) + ": region is null", ErrorCodes::LOGICAL_ERROR); + + // Tiny optimization for queries that need only handle, tso, delmark. + bool need_value = column_names_to_read.size() != 3; + auto [data_list_read, read_status] = resolveLocksAndReadRegionData( + table_info.id, region, start_ts, region_version, conf_version, handle_range, resolve_locks, need_value); + if (read_status != RegionException::OK) + return {Block(), read_status}; + /// Read region data as block. Block block; { @@ -219,14 +257,36 @@ std::tuple RegionTable::readBlockByReg return {std::move(block), RegionException::OK}; } -void RegionTable::resolveLocks(Region::CommittedScanner & scanner, const Timestamp start_ts) +RegionException::RegionReadStatus RegionTable::resolveLocksAndWriteRegion(TMTContext & tmt, + const TiDB::TableID table_id, + const RegionPtr & region, + const Timestamp start_ts, + RegionVersion region_version, + RegionVersion conf_version, + DB::HandleRange & handle_range, + Logger * log) { - if (LockInfoPtr lock_info = scanner.getLockInfo(start_ts); lock_info) + auto [data_list_read, read_status] = resolveLocksAndReadRegionData( + table_id, region, start_ts, region_version, conf_version, handle_range, /* resolve_locks */ true, /* need_data_value */ true); + if (read_status != RegionException::OK) + return read_status; + + auto & context = tmt.getContext(); + writeRegionDataToStorage(context, region, data_list_read, log); + + /// Remove committed data { - LockInfos lock_infos; - lock_infos.emplace_back(std::move(lock_info)); - throw LockException(std::move(lock_infos)); + auto remover = region->createCommittedRemover(); + for (const auto & [handle, write_type, commit_ts, value] : data_list_read) + { + std::ignore = write_type; + std::ignore = value; + + remover.remove({handle, commit_ts}); + } } + + return RegionException::OK; } } // namespace DB diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 6cb0ab3576f..93d9269c22a 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -79,15 +79,15 @@ TiFlashApplyRes HandleAdminRaftCmd(const TiFlashServer * server, BaseBuffView re } } -void HandleApplySnapshot(const TiFlashServer * server, BaseBuffView region_buff, uint64_t peer_id, SnapshotDataView lock_cf_view, - SnapshotDataView write_cf_view, SnapshotDataView default_cf_view, uint64_t index, uint64_t term) +void HandleApplySnapshot( + const TiFlashServer * server, BaseBuffView region_buff, uint64_t peer_id, SnapshotViewArray snaps, uint64_t index, uint64_t term) { try { metapb::Region region; region.ParseFromArray(region_buff.data, (int)region_buff.len); auto & kvstore = server->tmt.getKVStore(); - kvstore->handleApplySnapshot(std::move(region), peer_id, lock_cf_view, write_cf_view, default_cf_view, index, term, server->tmt); + kvstore->handleApplySnapshot(std::move(region), peer_id, snaps, index, term, server->tmt); } catch (...) { @@ -125,4 +125,20 @@ void HandleDestroy(TiFlashServer * server, RegionId region_id) } } +void HandleIngestSST(TiFlashServer * server, SnapshotViewArray snaps, RaftCmdHeader header) +{ + try + { + auto & kvstore = server->tmt.getKVStore(); + kvstore->handleIngestSST(header.region_id, snaps, header.index, header.term, server->tmt); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + exit(-1); + } +} + +uint8_t HandleCheckTerminated(TiFlashServer * server) { return server->tmt.getTerminated().load(std::memory_order_relaxed) ? 1 : 0; } + } // namespace DB diff --git a/dbms/src/Storages/Transaction/ProxyFFIType.h b/dbms/src/Storages/Transaction/ProxyFFIType.h index 24c6b849934..cd533d73bea 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIType.h +++ b/dbms/src/Storages/Transaction/ProxyFFIType.h @@ -47,10 +47,17 @@ struct BaseBuffView const uint64_t len; }; -struct SnapshotDataView +struct SnapshotView { const BaseBuffView * keys; const BaseBuffView * vals; + const ColumnFamilyType cf; + const uint64_t len = 0; +}; + +struct SnapshotViewArray +{ + const SnapshotView * views; const uint64_t len = 0; }; @@ -76,10 +83,12 @@ struct TiFlashServerHelper void (*fn_gc_buff)(BaseBuff *); TiFlashApplyRes (*fn_handle_write_raft_cmd)(const TiFlashServer *, WriteCmdsView, RaftCmdHeader); TiFlashApplyRes (*fn_handle_admin_raft_cmd)(const TiFlashServer *, BaseBuffView, BaseBuffView, RaftCmdHeader); - void (*fn_handle_apply_snapshot)( - const TiFlashServer *, BaseBuffView, uint64_t, SnapshotDataView, SnapshotDataView, SnapshotDataView, uint64_t, uint64_t); + void (*fn_handle_apply_snapshot)(const TiFlashServer *, BaseBuffView, uint64_t, SnapshotViewArray, uint64_t, uint64_t); void (*fn_atomic_update_proxy)(TiFlashServer *, TiFlashRaftProxy *); void (*fn_handle_destroy)(TiFlashServer *, RegionId); + void (*fn_handle_ingest_sst)(TiFlashServer *, SnapshotViewArray, RaftCmdHeader); + uint8_t (*fn_handle_check_terminated)(TiFlashServer *); + // uint32_t magic_number; // use a very special number to check whether this struct is legal uint32_t version; // version of function interface @@ -96,9 +105,11 @@ struct TiFlashServer void GcBuff(BaseBuff * buff); TiFlashApplyRes HandleAdminRaftCmd(const TiFlashServer * server, BaseBuffView req_buff, BaseBuffView resp_buff, RaftCmdHeader header); -void HandleApplySnapshot(const TiFlashServer * server, BaseBuffView region_buff, uint64_t peer_id, SnapshotDataView lock_buff, - SnapshotDataView write_buff, SnapshotDataView default_buff, uint64_t index, uint64_t term); +void HandleApplySnapshot( + const TiFlashServer * server, BaseBuffView region_buff, uint64_t peer_id, SnapshotViewArray snaps, uint64_t index, uint64_t term); TiFlashApplyRes HandleWriteRaftCmd(const TiFlashServer * server, WriteCmdsView req_buff, RaftCmdHeader header); void AtomicUpdateProxy(TiFlashServer * server, TiFlashRaftProxy * proxy); void HandleDestroy(TiFlashServer * server, RegionId region_id); +void HandleIngestSST(TiFlashServer * server, SnapshotViewArray snaps, RaftCmdHeader header); +uint8_t HandleCheckTerminated(TiFlashServer * server); } // namespace DB diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 334e8457fed..5fd315e9614 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -403,9 +404,15 @@ void Region::markCompactLog() const { last_compact_log_time = Clock::now(); } Timepoint Region::lastCompactLogTime() const { return last_compact_log_time; } -Region::CommittedScanner Region::createCommittedScanner() { return Region::CommittedScanner(this->shared_from_this()); } +Region::CommittedScanner Region::createCommittedScanner(bool use_lock) +{ + return Region::CommittedScanner(this->shared_from_this(), use_lock); +} -Region::CommittedRemover Region::createCommittedRemover() { return Region::CommittedRemover(this->shared_from_this()); } +Region::CommittedRemover Region::createCommittedRemover(bool use_lock) +{ + return Region::CommittedRemover(this->shared_from_this(), use_lock); +} std::string Region::toString(bool dump_status) const { return meta.toString(dump_status); } @@ -418,7 +425,7 @@ ReadIndexResult Region::learnerRead() return {}; } -void Region::waitIndex(UInt64 index) +TerminateWaitIndex Region::waitIndex(UInt64 index, const std::atomic_bool & terminated) { if (index_reader != nullptr) { @@ -426,10 +433,12 @@ void Region::waitIndex(UInt64 index) if (index != 1 + RAFT_INIT_LOG_INDEX && !meta.checkIndex(index)) { LOG_DEBUG(log, toString() << " need to wait learner index: " << index); - meta.waitIndex(index); + if (meta.waitIndex(index, terminated)) + return true; LOG_DEBUG(log, toString(false) << " wait learner index " << index << " done"); } } + return false; } UInt64 Region::version() const { return meta.version(); } @@ -516,7 +525,7 @@ void Region::compareAndCompleteSnapshot(HandleMap & handle_map, const Timestamp LOG_INFO(log, __FUNCTION__ << ": add deleted gc: " << deleted_gc_cnt); } -TiFlashApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, bool set_applied) +TiFlashApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt) { if (index == 1 + RAFT_INIT_LOG_INDEX) { @@ -602,16 +611,53 @@ TiFlashApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 in handle_write_cmd_func(); - if (set_applied) - meta.setApplied(index, term); + if (tmt.isBgFlushDisabled()) + { + /// Flush data right after they are committed. + RegionDataReadInfoList data_list_to_remove; + RegionTable::writeBlockByRegion(tmt.getContext(), shared_from_this(), data_list_to_remove, log, false); + + /// Do not need to run predecode. + data.writeCF().getCFDataPreDecode().popAll(); + data.defaultCF().getCFDataPreDecode().popAll(); + } + + meta.setApplied(index, term); } - if (set_applied) - meta.notifyAll(); + meta.notifyAll(); return TiFlashApplyRes::None; } +void Region::handleIngestSST(const SnapshotViewArray snaps, UInt64 index, UInt64 term) +{ + if (index <= appliedIndex()) + return; + + { + std::unique_lock lock(mutex); + std::lock_guard predecode_lock(predecode_mutex); + + for (UInt64 i = 0; i < snaps.len; ++i) + { + auto & snapshot = snaps.views[i]; + + LOG_INFO(log, + __FUNCTION__ << ": " << toString(false) << " begin to ingest sst of cf " << CFToName(snapshot.cf) << " at [term: " << term + << ", index: " << index << "], kv count " << snapshot.len); + for (UInt64 n = 0; n < snapshot.len; ++n) + { + auto & k = snapshot.keys[n]; + auto & v = snapshot.vals[n]; + doInsert(snapshot.cf, TiKVKey(k.data, k.len), TiKVValue(v.data, v.len)); + } + } + meta.setApplied(index, term); + } + meta.notifyAll(); +} + RegionRaftCommandDelegate & Region::makeRaftCommandDelegate(const KVStoreTaskLock & lock) { static_assert(sizeof(RegionRaftCommandDelegate) == sizeof(Region)); diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index fdd00dcb626..cd99505c195 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -24,9 +24,9 @@ class Context; class TMTContext; struct WriteCmdsView; enum TiFlashApplyRes : uint32_t; +struct SnapshotViewArray; /// Store all kv data of one region. Including 'write', 'data' and 'lock' column families. -/// TODO: currently the synchronize mechanism is broken and need to fix. class Region : public std::enable_shared_from_this { public: @@ -40,8 +40,11 @@ class Region : public std::enable_shared_from_this class CommittedScanner : private boost::noncopyable { public: - CommittedScanner(const RegionPtr & store_) : store(store_), lock(store_->mutex) + CommittedScanner(const RegionPtr & store_, bool use_lock = true) : store(store_) { + if (use_lock) + lock = std::shared_lock(store_->mutex); + const auto & data = store->data.writeCF().getData(); write_map_size = data.size(); @@ -69,7 +72,11 @@ class Region : public std::enable_shared_from_this class CommittedRemover : private boost::noncopyable { public: - CommittedRemover(const RegionPtr & store_) : store(store_), lock(store_->mutex) {} + CommittedRemover(const RegionPtr & store_, bool use_lock = true) : store(store_) + { + if (use_lock) + lock = std::unique_lock(store_->mutex); + } void remove(const RegionWriteCFData::Key & key) { @@ -91,8 +98,8 @@ class Region : public std::enable_shared_from_this void insert(ColumnFamilyType cf, TiKVKey && key, TiKVValue && value); void remove(const std::string & cf, const TiKVKey & key); - CommittedScanner createCommittedScanner(); - CommittedRemover createCommittedRemover(); + CommittedScanner createCommittedScanner(bool use_lock = true); + CommittedRemover createCommittedRemover(bool use_lock = true); std::tuple serialize(WriteBuffer & buf) const; static RegionPtr deserialize(ReadBuffer & buf, const IndexReaderCreateFunc * index_reader_create = nullptr); @@ -127,11 +134,11 @@ class Region : public std::enable_shared_from_this ReadIndexResult learnerRead(); - void waitIndex(UInt64 index); + /// If server is terminating, return true (read logic should throw NOT_FOUND exception and let upper layer retry other store). + TerminateWaitIndex waitIndex(UInt64 index, const std::atomic_bool & terminated); UInt64 appliedIndex() const; - void setApplied(UInt64 index, UInt64 term) { meta.setApplied(index, term); } void notifyApplied() { meta.notifyAll(); } RegionVersion version() const; @@ -159,7 +166,8 @@ class Region : public std::enable_shared_from_this void tryPreDecodeTiKVValue(TMTContext & tmt); TableID getMappedTableID() const; - TiFlashApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, bool set_applied = true); + TiFlashApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt); + void handleIngestSST(const SnapshotViewArray snaps, UInt64 index, UInt64 term); private: Region() = delete; diff --git a/dbms/src/Storages/Transaction/RegionCFDataBase.cpp b/dbms/src/Storages/Transaction/RegionCFDataBase.cpp index ce6c8446744..76e8e21b505 100644 --- a/dbms/src/Storages/Transaction/RegionCFDataBase.cpp +++ b/dbms/src/Storages/Transaction/RegionCFDataBase.cpp @@ -58,6 +58,7 @@ void RegionCFDataBase::finishInsert(typename Map::iterat auto & [handle, ts] = write_it->first; auto & [write_type, prewrite_ts, short_value] = decoded_val; + std::ignore = key; std::ignore = value; std::ignore = ts; @@ -73,12 +74,8 @@ void RegionCFDataBase::finishInsert(typename Map::iterat if (auto data_it = default_cf_map.find({handle, prewrite_ts}); data_it != default_cf_map.end()) { - short_value = std::get<1>(data_it->second); + short_value = RegionDefaultCFDataTrait::getTiKVValue(data_it); } - else - throw Exception("Handle: " + std::to_string(handle) + ", Prewrite ts: " + std::to_string(prewrite_ts) - + " can not found in default cf for key: " + key->toHex(), - ErrorCodes::LOGICAL_ERROR); } } else diff --git a/dbms/src/Storages/Transaction/RegionCFDataTrait.h b/dbms/src/Storages/Transaction/RegionCFDataTrait.h index 727d10a56cb..7acf482d10e 100644 --- a/dbms/src/Storages/Transaction/RegionCFDataTrait.h +++ b/dbms/src/Storages/Transaction/RegionCFDataTrait.h @@ -53,6 +53,8 @@ struct RegionDefaultCFDataTrait return {Key{handle_id, ts}, Value{std::make_shared(std::move(key)), std::make_shared(std::move(value))}}; } + + static std::shared_ptr getTiKVValue(const Map::const_iterator & it) { return std::get<1>(it->second); } }; struct RegionLockCFDataTrait diff --git a/dbms/src/Storages/Transaction/RegionData.cpp b/dbms/src/Storages/Transaction/RegionData.cpp index 1e0540c91ac..66ac6edbe79 100644 --- a/dbms/src/Storages/Transaction/RegionData.cpp +++ b/dbms/src/Storages/Transaction/RegionData.cpp @@ -97,7 +97,15 @@ RegionDataReadInfo RegionData::readDataByWriteIt(const ConstWriteCFIter & write_ return std::make_tuple(handle, write_type, ts, nullptr); if (!short_value) - throw Exception(" key [" + key->toString() + "] not found in default cf", ErrorCodes::LOGICAL_ERROR); + { + const auto & map = default_cf.getData(); + if (auto data_it = map.find({handle, prewrite_ts}); data_it != map.end()) + return std::make_tuple(handle, write_type, ts, RegionDefaultCFDataTrait::getTiKVValue(data_it)); + else + throw Exception("Handle: " + std::to_string(handle) + ", Prewrite ts: " + std::to_string(prewrite_ts) + + " can not found in default cf for key: " + key->toHex(), + ErrorCodes::LOGICAL_ERROR); + } return std::make_tuple(handle, write_type, ts, short_value); } diff --git a/dbms/src/Storages/Transaction/RegionMeta.cpp b/dbms/src/Storages/Transaction/RegionMeta.cpp index f756f03c11f..650c5fa7441 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.cpp +++ b/dbms/src/Storages/Transaction/RegionMeta.cpp @@ -144,10 +144,19 @@ void RegionMeta::setPeerState(const raft_serverpb::PeerState peer_state_) region_state.setState(peer_state_); } -void RegionMeta::waitIndex(UInt64 index) const +TerminateWaitIndex RegionMeta::waitIndex(UInt64 index, const std::atomic_bool & terminated) const { std::unique_lock lock(mutex); - cv.wait(lock, [this, index] { return doCheckIndex(index); }); + TerminateWaitIndex res = false; + cv.wait(lock, [&] { + if (terminated) + { + res = true; + return true; + } + return doCheckIndex(index); + }); + return res; } bool RegionMeta::checkIndex(UInt64 index) const diff --git a/dbms/src/Storages/Transaction/RegionMeta.h b/dbms/src/Storages/Transaction/RegionMeta.h index 6c7baebb0b1..c22d1cb2978 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.h +++ b/dbms/src/Storages/Transaction/RegionMeta.h @@ -16,6 +16,7 @@ struct RegionMergeResult; class Region; class MetaRaftCommandDelegate; class RegionRaftCommandDelegate; +using TerminateWaitIndex = bool; class RegionMeta { @@ -61,7 +62,7 @@ class RegionMeta friend bool operator==(const RegionMeta & meta1, const RegionMeta & meta2); - void waitIndex(UInt64 index) const; + TerminateWaitIndex waitIndex(UInt64 index, const std::atomic_bool & terminated) const; bool checkIndex(UInt64 index) const; bool isPeerRemoved() const; diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index ec628aef59e..f167f00e47d 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -121,19 +121,7 @@ RegionDataReadInfoList RegionTable::flushRegion(const RegionPtr & region, bool t writeBlockByRegion(*context, region, data_list_to_remove, log); } - /// Remove data in region. { - { - auto remover = region->createCommittedRemover(); - for (const auto & [handle, write_type, commit_ts, value] : data_list_to_remove) - { - std::ignore = write_type; - std::ignore = value; - - remover.remove({handle, commit_ts}); - } - } - size_t cache_size = region->dataSize(); if (cache_size == 0) @@ -230,7 +218,7 @@ TableID RegionTable::popOneTableToOptimize() return res; } -void RegionTable::removeRegion(const RegionID region_id) +void RegionTable::removeRegion(const RegionID region_id, bool remove_data) { std::lock_guard lock(mutex); @@ -244,7 +232,13 @@ void RegionTable::removeRegion(const RegionID region_id) TableID table_id = it->second; auto & table = tables.find(table_id)->second; + do { + // Sometime we don't need to remove data. e.g. remove region after region merge. + if (!remove_data) + break; + /// Some region of this table is removed, if it is a DeltaTree, write deleteRange. + /// Now we assume that StorageDeltaMerge::deleteRange do not block for long time and do it in sync mode. /// If this block for long time, consider to do this in background threads. TMTContext & tmt = context->getTMTContext(); @@ -253,22 +247,28 @@ void RegionTable::removeRegion(const RegionID region_id) { // acquire lock so that no other threads can change storage's structure auto storage_lock = storage->lockStructure(true, __PRETTY_FUNCTION__); + // Check if it has been dropped by other thread + if (storage->is_dropped) + break; // continue to remove region on `regions`, `table.regions` + auto dm_storage = std::dynamic_pointer_cast(storage); auto region_it = table.regions.find(region_id); - if (region_it == table.regions.end()) - return; - HandleRange handle_range = region_it->second.range_in_table; + if (dm_storage == nullptr || region_it == table.regions.end()) + break; + HandleRange handle_range = region_it->second.range_in_table; auto dm_handle_range = toDMHandleRange(handle_range); dm_storage->deleteRange(dm_handle_range, context->getSettingsRef()); - dm_storage->flushCache(*context, dm_handle_range); + dm_storage->flushCache(*context, dm_handle_range); // flush to disk } - } + } while (0); regions.erase(it); table.regions.erase(region_id); if (table.regions.empty()) { + /// All regions of this table is removed, the storage maybe drop or pd + /// move it to another node, we can optimize outdated data. table_to_optimize.insert(table_id); tables.erase(table_id); } diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 4ce22d85b5d..b8f2f0eb081 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -109,7 +109,7 @@ class RegionTable : private boost::noncopyable /// This functional only shrink the table range of this region_id void shrinkRegionRange(const Region & region); - void removeRegion(const RegionID region_id); + void removeRegion(const RegionID region_id, bool remove_data); TableID popOneTableToOptimize(); @@ -126,7 +126,8 @@ class RegionTable : private boost::noncopyable /// Will trigger schema sync on read error for only once, /// assuming that newer schema can always apply to older data by setting force_decode to true in readRegionBlock. /// Note that table schema must be keep unchanged throughout the process of read then write, we take good care of the lock. - static void writeBlockByRegion(Context & context, RegionPtr region, RegionDataReadInfoList & data_list_to_remove, Logger * log); + static void writeBlockByRegion( + Context & context, const RegionPtr & region, RegionDataReadInfoList & data_list_to_remove, Logger * log, bool lock_region = true); /// Read the data of the given region into block, take good care of learner read and locks. /// Assuming that the schema has been properly synced by outer, i.e. being new enough to decode data before start_ts, @@ -142,8 +143,16 @@ class RegionTable : private boost::noncopyable DB::HandleRange & handle_range, RegionScanFilterPtr scan_filter = nullptr); - /// Check if there are any lock should be resolved, if so, throw LockException. - static void resolveLocks(Region::CommittedScanner & scanner, const Timestamp start_ts); + /// Check transaction locks in region, and write committed data in it into storage engine if check passed. Otherwise throw an LockException. + /// The write logic is the same as #writeBlockByRegion, with some extra checks about region version and conf_version. + static RegionException::RegionReadStatus resolveLocksAndWriteRegion(TMTContext & tmt, + const TiDB::TableID table_id, + const RegionPtr & region, + const Timestamp start_ts, + RegionVersion region_version, + RegionVersion conf_version, + DB::HandleRange & handle_range, + Logger * log); void checkTableOptimize(); void checkTableOptimize(TableID, const double); @@ -165,9 +174,9 @@ class RegionTable : private boost::noncopyable InternalRegion & insertRegion(Table & table, const RegionRangeKeys & region_range_keys, const RegionID region_id); InternalRegion & doGetInternalRegion(TableID table_id, RegionID region_id); + RegionDataReadInfoList flushRegion(const RegionPtr & region, bool try_persist) const; bool shouldFlush(const InternalRegion & region) const; RegionID pickRegionToFlush(); - RegionDataReadInfoList flushRegion(const RegionPtr & region, bool try_persist) const; void incrDirtyFlag(RegionID region_id); void clearDirtyFlag(RegionID region_id); diff --git a/dbms/src/Storages/Transaction/SchemaBuilder.cpp b/dbms/src/Storages/Transaction/SchemaBuilder.cpp index 6e9275e82e8..fdf387fab98 100644 --- a/dbms/src/Storages/Transaction/SchemaBuilder.cpp +++ b/dbms/src/Storages/Transaction/SchemaBuilder.cpp @@ -36,7 +36,12 @@ inline void setAlterCommandColumn(Logger * log, AlterCommand & command, const Co if (!column_info.origin_default_value.isEmpty()) { LOG_DEBUG(log, "add default value for column: " << column_info.name); - auto arg0 = std::make_shared(column_info.defaultValueToField()); + ASTPtr arg0; + // If it's date time types, we should use string literal to generate default value. + if (column_info.tp == TypeDatetime || column_info.tp == TypeTimestamp || column_info.tp == TypeDate) + arg0 = std::make_shared(Field(column_info.origin_default_value.convert())); + else + arg0 = std::make_shared(column_info.defaultValueToField()); auto arg1 = std::make_shared(command.data_type->getName()); auto args = std::make_shared(); args->children.emplace_back(arg0); @@ -347,7 +352,8 @@ void SchemaBuilder::applyAlterPartition(TiDB::DBInfoPtr db_info, TableID auto storage = tmt_context.getStorages().get(table_id).get(); if (storage == nullptr) { - throw Exception("miss table in Flash " + table_info->name, ErrorCodes::DDL_ERROR); + throw Exception("miss table in Flash `" + table_info->db_name + "`.`" + table_info->name + "`, id: " + DB::toString(table_id), + ErrorCodes::DDL_ERROR); } const String & db_name = storage->getDatabaseName(); const auto & orig_table_info = storage->getTableInfo(); @@ -878,7 +884,7 @@ void SchemaBuilder::syncAllSchema() std::vector tables = getter.listTables(db->id); for (const auto & table : tables) { - LOG_DEBUG(log, "collect table: " << table->name << " with id "<< table->id); + LOG_DEBUG(log, "collect table: " << table->name << " with id " << table->id); all_tables.emplace_back(table, db); if (table->isLogicalPartitionTable()) { diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index dc2aaaff69e..474ebff6ef5 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -92,4 +92,13 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) getKVStore()->setRegionCompactLogPeriod(Seconds{config.getUInt64(COMPACT_LOG_MIN_PERIOD, 200)}); } +const std::atomic_bool & TMTContext::getTerminated() const { return terminated; } + +void TMTContext::setTerminated() +{ + terminated = true; + // notify all region to stop learner read. + kvstore->traverseRegions([](const RegionID, const RegionPtr & region) { region->notifyApplied(); }); +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index 4f9b8a1238b..0a62647f4ce 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -66,6 +66,9 @@ class TMTContext : private boost::noncopyable void reloadConfig(const Poco::Util::AbstractConfiguration & config); + const std::atomic_bool & getTerminated() const; + void setTerminated(); + private: Context & context; KVStorePtr kvstore; @@ -85,6 +88,8 @@ class TMTContext : private boost::noncopyable ::TiDB::StorageEngine engine; bool disable_bg_flush; + + std::atomic_bool terminated{false}; }; } // namespace DB diff --git a/dbms/src/Storages/Transaction/TMTStorages.cpp b/dbms/src/Storages/Transaction/TMTStorages.cpp index 1ba27269106..42f140c7ab7 100644 --- a/dbms/src/Storages/Transaction/TMTStorages.cpp +++ b/dbms/src/Storages/Transaction/TMTStorages.cpp @@ -19,10 +19,9 @@ ManageableStoragePtr ManagedStorages::get(TableID table_id) const { std::lock_guard lock(mutex); - auto it = storages.find(table_id); - if (it == storages.end()) - return nullptr; - return it->second; + if (auto it = storages.find(table_id); it != storages.end()) + return it->second; + return nullptr; } std::unordered_map ManagedStorages::getAllStorage() const @@ -36,7 +35,7 @@ ManageableStoragePtr ManagedStorages::getByName(const std::string & db, const st std::lock_guard lock(mutex); auto it = std::find_if(storages.begin(), storages.end(), [&](const std::pair & pair) { - auto &storage = pair.second; + auto & storage = pair.second; return storage->getDatabaseName() == db && storage->getTableName() == table; }); if (it == storages.end()) diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 2cfd79e1e65..fd4b109d9b9 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -150,14 +150,12 @@ UInt64 ColumnInfo::getSetValue(const String & set_str) const if (marked.empty()) return value; - throw DB::Exception( - std::string(__PRETTY_FUNCTION__) + ": can't parse set type value."); + throw DB::Exception(std::string(__PRETTY_FUNCTION__) + ": can't parse set type value."); } Int64 ColumnInfo::getTimeValue(const String & time_str) const { - const static long fractional_seconds_multiplier[] = {1000000000, 100000000, 10000000, 1000000, - 100000, 10000, 1000, 100, 10, 1}; + const static long fractional_seconds_multiplier[] = {1000000000, 100000000, 10000000, 1000000, 100000, 10000, 1000, 100, 10, 1}; bool negative = time_str[0] == '-'; Poco::StringTokenizer second_and_fsp(time_str, "."); Poco::StringTokenizer string_tokens(second_and_fsp[0], ":"); @@ -171,8 +169,7 @@ Int64 ColumnInfo::getTimeValue(const String & time_str) const fs_length = second_and_fsp[1].length(); fs_value = std::stol(second_and_fsp[1]); } - ret = ret * fractional_seconds_multiplier[0] - + fs_value * fractional_seconds_multiplier[fs_length]; + ret = ret * fractional_seconds_multiplier[0] + fs_value * fractional_seconds_multiplier[fs_length]; return negative ? -ret : ret; } @@ -186,7 +183,8 @@ Int64 ColumnInfo::getYearValue(Int64 val) const return val; } -Poco::JSON::Object::Ptr ColumnInfo::getJSONObject() const try +Poco::JSON::Object::Ptr ColumnInfo::getJSONObject() const +try { Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); @@ -229,7 +227,8 @@ catch (const Poco::Exception & e) std::string(__PRETTY_FUNCTION__) + ": Serialize TiDB schema JSON failed (ColumnInfo): " + e.displayText(), DB::Exception(e)); } -void ColumnInfo::deserialize(Poco::JSON::Object::Ptr json) try +void ColumnInfo::deserialize(Poco::JSON::Object::Ptr json) +try { id = json->getValue("id"); name = json->getObject("name")->getValue("L"); @@ -263,7 +262,8 @@ catch (const Poco::Exception & e) PartitionDefinition::PartitionDefinition(Poco::JSON::Object::Ptr json) { deserialize(json); } -Poco::JSON::Object::Ptr PartitionDefinition::getJSONObject() const try +Poco::JSON::Object::Ptr PartitionDefinition::getJSONObject() const +try { Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); json->set("id", id); @@ -284,7 +284,8 @@ catch (const Poco::Exception & e) std::string(__PRETTY_FUNCTION__) + ": Serialize TiDB schema JSON failed (PartitionDef): " + e.displayText(), DB::Exception(e)); } -void PartitionDefinition::deserialize(Poco::JSON::Object::Ptr json) try +void PartitionDefinition::deserialize(Poco::JSON::Object::Ptr json) +try { id = json->getValue("id"); name = json->getObject("name")->getValue("L"); @@ -299,7 +300,8 @@ catch (const Poco::Exception & e) PartitionInfo::PartitionInfo(Poco::JSON::Object::Ptr json) { deserialize(json); } -Poco::JSON::Object::Ptr PartitionInfo::getJSONObject() const try +Poco::JSON::Object::Ptr PartitionInfo::getJSONObject() const +try { Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); @@ -328,7 +330,8 @@ catch (const Poco::Exception & e) std::string(__PRETTY_FUNCTION__) + ": Serialize TiDB schema JSON failed (PartitionInfo): " + e.displayText(), DB::Exception(e)); } -void PartitionInfo::deserialize(Poco::JSON::Object::Ptr json) try +void PartitionInfo::deserialize(Poco::JSON::Object::Ptr json) +try { type = static_cast(json->getValue("type")); expr = json->getValue("expr"); @@ -352,7 +355,8 @@ catch (const Poco::Exception & e) TableInfo::TableInfo(const String & table_info_json) { deserialize(table_info_json); } -String TableInfo::serialize() const try +String TableInfo::serialize() const +try { std::stringstream buf; @@ -401,7 +405,8 @@ catch (const Poco::Exception & e) std::string(__PRETTY_FUNCTION__) + ": Serialize TiDB schema JSON failed (TableInfo): " + e.displayText(), DB::Exception(e)); } -void DBInfo::deserialize(const String & json_str) try +void DBInfo::deserialize(const String & json_str) +try { Poco::JSON::Parser parser; Poco::Dynamic::Var result = parser.parse(json_str); @@ -419,7 +424,8 @@ catch (const Poco::Exception & e) DB::Exception(e)); } -void TableInfo::deserialize(const String & json_str) try +void TableInfo::deserialize(const String & json_str) +try { if (json_str.empty()) { @@ -542,9 +548,9 @@ String TableInfo::getColumnName(const ColumnID id) const DB::ErrorCodes::LOGICAL_ERROR); } -ColumnInfo TableInfo::getColumnInfoByID(const ColumnID id) const +const ColumnInfo & TableInfo::getColumnInfo(const ColumnID id) const { - for (auto & col : columns) + for (const auto & col : columns) { if (id == col.id) { @@ -553,8 +559,8 @@ ColumnInfo TableInfo::getColumnInfoByID(const ColumnID id) const } throw DB::Exception( - std::string(__PRETTY_FUNCTION__) + ": Invalidate column id " + std::to_string(id) + " for table " + db_name + "." + name, - DB::ErrorCodes::LOGICAL_ERROR); + std::string(__PRETTY_FUNCTION__) + ": Invalidate column id " + std::to_string(id) + " for table " + db_name + "." + name, + DB::ErrorCodes::LOGICAL_ERROR); } std::optional> TableInfo::getPKHandleColumn() const diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index 955af266f64..05bf979bca1 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -1,13 +1,13 @@ #pragma once -#include - #include #include #include #include -#include #include +#include + +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" @@ -287,7 +287,8 @@ struct TableInfo ColumnID getColumnID(const String & name) const; String getColumnName(const ColumnID id) const; - ColumnInfo getColumnInfoByID(const ColumnID id) const; + + const ColumnInfo & getColumnInfo(const ColumnID id) const; std::optional> getPKHandleColumn() const; diff --git a/dbms/src/test_utils/TiflashTestBasic.h b/dbms/src/test_utils/TiflashTestBasic.h index db654fa9cc2..10eed176958 100644 --- a/dbms/src/test_utils/TiflashTestBasic.h +++ b/dbms/src/test_utils/TiflashTestBasic.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -24,6 +25,21 @@ namespace tests throw; \ } +/// helper functions for comparing DataType +inline ::testing::AssertionResult DataTypeCompare( // + const char * lhs_expr, + const char * rhs_expr, + const DataTypePtr & lhs, + const DataTypePtr & rhs) +{ + if (lhs->equals(*rhs)) + return ::testing::AssertionSuccess(); + else + return ::testing::internal::EqFailure(lhs_expr, rhs_expr, lhs->getName(), rhs->getName(), false); +} +#define ASSERT_DATATYPE_EQ(val1, val2) ASSERT_PRED_FORMAT2(::DB::tests::DataTypeCompare, val1, val2) +#define EXPECT_DATATYPE_EQ(val1, val2) EXPECT_PRED_FORMAT2(::DB::tests::DataTypeCompare, val1, val2) + // A simple helper for getting DataType from type name inline DataTypePtr typeFromString(const String & str) { diff --git a/release-centos7/build/build-tiflash-ci.sh b/release-centos7/build/build-tiflash-ci.sh index e557fd2c472..cf6baa430c4 100755 --- a/release-centos7/build/build-tiflash-ci.sh +++ b/release-centos7/build/build-tiflash-ci.sh @@ -41,7 +41,7 @@ mkdir -p ${SRCPATH}/libs/libtiflash-proxy # upload: curl -F builds/pingcap/tiflash-proxy/{commit-hash}/libtiflash_proxy.so=@libtiflash_proxy.so http://fileserver.pingcap.net/upload curl -o "${SRCPATH}/libs/libtiflash-proxy/libtiflash_proxy.so" \ -http://fileserver.pingcap.net/download/builds/pingcap/tiflash-proxy/5885ec91490120b66c86fcb7d6aa7c1d4259c735/libtiflash_proxy.so +http://fileserver.pingcap.net/download/builds/pingcap/tiflash-proxy/ad4addbdcab1170f972d88723fe2840068851914/libtiflash_proxy.so chmod 0731 "${SRCPATH}/libs/libtiflash-proxy/libtiflash_proxy.so" build_dir="$SRCPATH/release-centos7/build-release" diff --git a/tests/_env.sh b/tests/_env.sh index bd70de7c84e..c8f2efb2ef3 100644 --- a/tests/_env.sh +++ b/tests/_env.sh @@ -10,9 +10,6 @@ fi export storage_bin="$build_dir/dbms/src/Server/tiflash" -# Serve config for launching -export storage_server_config="../../running/config/config.xml" - # Server address for connecting export storage_server="127.0.0.1" diff --git a/tests/delta-merge-test/ddl/alter.test b/tests/delta-merge-test/ddl/alter.test index 7c6f49a9a29..b5ef46d0718 100644 --- a/tests/delta-merge-test/ddl/alter.test +++ b/tests/delta-merge-test/ddl/alter.test @@ -1,6 +1,3 @@ -#TODO: enable ddl tests for DeltaMerge -#RETURN - >> drop table if exists dm_test ## Prepare table @@ -19,22 +16,14 @@ >> alter table dm_test drop column a Received exception from server (version {#WORD}): Code: 36. DB::Exception: Received from {#WORD} DB::Exception: Storage engine DeltaMerge doesn't support drop primary key / hidden column: a. -#>> show create table dm_test -#┌─statement───────────────────────────────────────────────────────────────┐ -#│ CREATE TABLE default.dm_test ( a Int8, b Int32) ENGINE = DeltaMerge(a) │ -#└─────────────────────────────────────────────────────────────────────────┘ >> alter table dm_test drop column b -#>> show create table dm_test -#┌─statement─────────────────────────────────────────────────────┐ -#│ CREATE TABLE default.dm_test ( a Int8) ENGINE = DeltaMerge(a) │ -#└───────────────────────────────────────────────────────────────┘ >> select * from dm_test ┌─a─┐ │ 1 │ └───┘ -# add a column which name is the same as before +# add a column which name is the same as before, but column b value should be default value(0) >> alter table dm_test add column b Int32 >> select * from dm_test ┌─a─┬─b─┐ diff --git a/tests/delta-merge-test/ddl/alter_default_value.test b/tests/delta-merge-test/ddl/alter_default_value.test deleted file mode 100644 index d3df83dc84b..00000000000 --- a/tests/delta-merge-test/ddl/alter_default_value.test +++ /dev/null @@ -1,50 +0,0 @@ -#TODO: enable ddl tests for DeltaMerge -#RETURN - -# modify column default value - ->> drop table if exists dm_test -## Prepare table ->> create table dm_test ( - pk Int8 - ) engine = DeltaMerge(pk) - ->> insert into table dm_test values(1) - -## create new column without default value. the row which pk==1, will filled with 'zero' value ->> alter table dm_test add column i0 Int32 ->> alter table dm_test add column f32_0 Float32 ->> alter table dm_test add column f64_0 Float64 ->> alter table dm_test add column dec0 Decimal(10,4) ->> alter table dm_test add column s0 String ->> alter table dm_test add column fs0 FixedString(4) ->> alter table dm_test add column dt0 DateTime - -## create new column with default value. the row which pk==1, will filled with those default value -## See FLASH-453 -#>> alter table dm_test add column s2 String default 'non-empty' - -## These alter command will throw exception now -## See FLASH-453 ->> alter table dm_test add column i1 Int32 default 999 ->> alter table dm_test add column f32_1 Float32 default 1.234 ->> alter table dm_test add column f64_1 Float64 default 1.234 ->> alter table dm_test add column dec1 Decimal(10,4) default 3.1415 ->> alter table dm_test add column fs1 FixedString(4) default 'aaa' ->> alter table dm_test add column dt1 DateTime default '1999-09-09 12:34:56' - ->> select * from dm_test where pk = 1 -┌─pk─┬─i0─┬─f32_0─┬─f64_0─┬─dec0───┬─s0─┬─fs0──────┬─────────────────dt0─┬──i1─┬─f32_1─┬─f64_1─┬─dec1───┬─fs1───┬─────────────────dt1─┐ -│ 1 │ 0 │ 0 │ 0 │ 0.0000 │ │ \0\0\0\0 │ 0000-00-00 00:00:00 │ 999 │ 1.234 │ 1.234 │ 3.1415 │ aaa\0 │ 1999-09-09 12:34:56 │ -└────┴────┴───────┴───────┴────────┴────┴──────────┴─────────────────────┴─────┴───────┴───────┴────────┴───────┴─────────────────────┘ - - -## insert a row, missing fields will be filled with default value ->> insert into table dm_test(pk) values(3) ->> select * from dm_test where pk = 3 -┌─pk─┬─i0─┬─f32_0─┬─f64_0─┬─dec0───┬─s0─┬─fs0──────┬─────────────────dt0─┬──i1─┬─f32_1─┬─f64_1─┬─dec1───┬─fs1───┬─────────────────dt1─┐ -│ 3 │ 0 │ 0 │ 0 │ 0.0000 │ │ \0\0\0\0 │ 0000-00-00 00:00:00 │ 999 │ 1.234 │ 1.234 │ 3.1415 │ aaa\0 │ 1999-09-09 12:34:56 │ -└────┴────┴───────┴───────┴────────┴────┴──────────┴─────────────────────┴─────┴───────┴───────┴────────┴───────┴─────────────────────┘ - -## clean up ->> drop table if exists dm_test diff --git a/tests/delta-merge-test/ddl/alter_joint_primary_key.test b/tests/delta-merge-test/ddl/alter_joint_primary_key.test index 73c8ee4800d..c5857235612 100644 --- a/tests/delta-merge-test/ddl/alter_joint_primary_key.test +++ b/tests/delta-merge-test/ddl/alter_joint_primary_key.test @@ -1,5 +1,3 @@ -#TODO: enable ddl tests for DeltaMerge -#RETURN >> drop table if exists dm_test @@ -26,17 +24,23 @@ Code: 36. DB::Exception: Received from {#WORD} DB::Exception: Storage engine Del ┌─a─┬─b─┬─c─────────────┬─d─────────────────────────────┐ │ 1 │ 2 │ hello TiFlash │ hello world\0\0\0\0\0\0\0\0\0 │ └───┴───┴───────────────┴───────────────────────────────┘ ->> show create table dm_test -┌─statement─────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ CREATE TABLE default.dm_test ( a Int32, b Int32, c String, d FixedString(20)) ENGINE = DeltaMerge((a, b)) │ -└───────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - +>> desc dm_test +┌─name──────────────┬─type────────────┬─default_type─┬─default_expression─┐ +│ a │ Int32 │ │ │ +│ b │ Int32 │ │ │ +│ c │ String │ │ │ +│ d │ FixedString(20) │ │ │ +│ _tidb_rowid │ Int64 │ │ │ +└───────────────────┴─────────────────┴──────────────┴────────────────────┘ >> alter table dm_test drop column c -#>> show create table dm_test -#┌─statement──────────────────────────────────────────────────────────────────────────────────────────┐ -#│ CREATE TABLE default.dm_test ( a Int32, b Int32, d FixedString(200)) ENGINE = DeltaMerge((a, b)) │ -#└────────────────────────────────────────────────────────────────────────────────────────────────────┘ +>> desc dm_test +┌─name──────────────┬─type────────────┬─default_type─┬─default_expression─┐ +│ a │ Int32 │ │ │ +│ b │ Int32 │ │ │ +│ d │ FixedString(20) │ │ │ +│ _tidb_rowid │ Int64 │ │ │ +└───────────────────┴─────────────────┴──────────────┴────────────────────┘ >> select * from dm_test ┌─a─┬─b─┬─d─────────────────────────────┐ │ 1 │ 2 │ hello world\0\0\0\0\0\0\0\0\0 │ @@ -48,10 +52,12 @@ Code: 36. DB::Exception: Received from {#WORD} DB::Exception: Storage engine Del ┌─a─┬─b─┐ │ 1 │ 2 │ └───┴───┘ -#>> show create table dm_test -#┌─statement─────────────────────────────────────────────────────────────────────┐ -#│ CREATE TABLE default.dm_test ( a Int32, b Int32) ENGINE = DeltaMerge((a, b)) │ -#└───────────────────────────────────────────────────────────────────────────────┘ +>> desc dm_test +┌─name──────────────┬─type───┬─default_type─┬─default_expression─┐ +│ a │ Int32 │ │ │ +│ b │ Int32 │ │ │ +│ _tidb_rowid │ Int64 │ │ │ +└───────────────────┴────────┴──────────────┴────────────────────┘ ## Clean up >> drop table if exists dm_test diff --git a/tests/delta-merge-test/ddl/alter_nullable.test b/tests/delta-merge-test/ddl/alter_nullable.test index 3a4cc8108be..c75e06bfd02 100644 --- a/tests/delta-merge-test/ddl/alter_nullable.test +++ b/tests/delta-merge-test/ddl/alter_nullable.test @@ -1,5 +1,3 @@ -#TODO: enable ddl tests for DeltaMerge -#RETURN >> drop table if exists dm_test diff --git a/tests/delta-merge-test/raft/bugs/FLASH-484.test b/tests/delta-merge-test/raft/bugs/FLASH-484.test index 9e2e306f7ad..abecde66cdc 100644 --- a/tests/delta-merge-test/raft/bugs/FLASH-484.test +++ b/tests/delta-merge-test/raft/bugs/FLASH-484.test @@ -19,7 +19,6 @@ ## insert data and get exception, then rowid==51 is not flush to Storage => DBGInvoke __raft_insert_row(default, test, 4, 51, 'test51') -=> DBGInvoke __try_flush_region(4) Received exception from server (version {#WORD}): Code: 48. DB::Exception: Received from {#WORD} DB::Exception: Reject first write for test, engine: BuggyMemory-RejectFirstWrite. => select count(*) from default.test @@ -31,7 +30,6 @@ Code: 48. DB::Exception: Received from {#WORD} DB::Exception: Reject first write ## insert more data, and this time we get all 3 rows(including rowid==51) flush to Storage => DBGInvoke __raft_insert_row(default, test, 4, 52, 'test52') => DBGInvoke __raft_insert_row(default, test, 4, 19, 'test19') -=> DBGInvoke __try_flush_region(4) => select * from default.test order by _tidb_rowid ┌─col_1──┬─_tidb_rowid─┐ │ test19 │ 19 │ diff --git a/tests/delta-merge-test/raft/region_merge.test b/tests/delta-merge-test/raft/region_merge.test new file mode 100644 index 00000000000..f117f364222 --- /dev/null +++ b/tests/delta-merge-test/raft/region_merge.test @@ -0,0 +1,107 @@ +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') + +=> DBGInvoke __region_snapshot(4, 0, 100, default, test) + +# raft_insert_row_full(database_name, table_name, region_id, handle_id, tso, del, val1, val2, ...) +=> DBGInvoke __raft_insert_row_full(default, test, 4, 2, 1, 0, 10) +=> DBGInvoke __raft_insert_row_full(default, test, 4, 1, 1, 0, 12) +=> DBGInvoke __raft_insert_row_full(default, test, 4, 3, 1, 0, 14) +=> DBGInvoke __raft_insert_row_full(default, test, 4, 4, 4, 1, 0) +=> DBGInvoke __raft_insert_row_full(default, test, 4, 4, 3, 0, 15) +=> DBGInvoke __raft_insert_row_full(default, test, 4, 5, 1, 0, 16) +=> select (*) from default.test " --schema_version "10000000 +┌─col_1─┬─_tidb_rowid─┐ +│ 12 │ 1 │ +│ 10 │ 2 │ +│ 14 │ 3 │ +│ 16 │ 5 │ +└───────┴─────────────┘ +=> selraw (*),_INTERNAL_VERSION, _INTERNAL_DELMARK from default.test order by _tidb_rowid, _INTERNAL_VERSION, _INTERNAL_DELMARK +┌─col_1─┬─_tidb_rowid─┬─_INTERNAL_VERSION─┬─_INTERNAL_DELMARK─┐ +│ 12 │ 1 │ 1 │ 0 │ +│ 10 │ 2 │ 1 │ 0 │ +│ 14 │ 3 │ 1 │ 0 │ +│ 15 │ 4 │ 3 │ 0 │ +│ 0 │ 4 │ 4 │ 1 │ +│ 16 │ 5 │ 1 │ 0 │ +└───────┴─────────────┴───────────────────┴───────────────────┘ +=> DBGInvoke dump_all_mock_region(default, test) +┌─dump_all_mock_region(default, test)────────┐ +│ [region 4] ranges: [0, 100), state: Normal │ +│ total size: 1 │ +└────────────────────────────────────────────┘ +=> DBGInvoke region_split(4, default, test, 0, 4, 4, 100, 5) +┌─region_split(4, default, test, 0, 4, 4, 100, 5)─┐ +│ execute batch split, region 4 into (4,5) │ +└─────────────────────────────────────────────────┘ +=> DBGInvoke __raft_insert_row_full(default, test, 5, 10, 1, 0, 17) +=> DBGInvoke __raft_insert_row_full(default, test, 5, 11, 1, 0, 18) +=> DBGInvoke __raft_insert_row_full(default, test, 5, 10, 3, 0, 19) +=> select (*) from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ 12 │ 1 │ +│ 10 │ 2 │ +│ 14 │ 3 │ +│ 16 │ 5 │ +│ 19 │ 10 │ +│ 18 │ 11 │ +└───────┴─────────────┘ +=> selraw (*), _INTERNAL_VERSION, _INTERNAL_DELMARK from default.test order by _tidb_rowid, _INTERNAL_VERSION, _INTERNAL_DELMARK +┌─col_1─┬─_tidb_rowid─┬─_INTERNAL_VERSION─┬─_INTERNAL_DELMARK─┐ +│ 12 │ 1 │ 1 │ 0 │ +│ 10 │ 2 │ 1 │ 0 │ +│ 14 │ 3 │ 1 │ 0 │ +│ 15 │ 4 │ 3 │ 0 │ +│ 0 │ 4 │ 4 │ 1 │ +│ 16 │ 5 │ 1 │ 0 │ +│ 17 │ 10 │ 1 │ 0 │ +│ 19 │ 10 │ 3 │ 0 │ +│ 18 │ 11 │ 1 │ 0 │ +└───────┴─────────────┴───────────────────┴───────────────────┘ +=> DBGInvoke dump_all_mock_region(default, test) +┌─dump_all_mock_region(default, test)────────┐ +│ [region 5] ranges: [4, 100), state: Normal │ +│ [region 4] ranges: [0, 4), state: Normal │ +│ total size: 2 │ +└────────────────────────────────────────────┘ +=> DBGInvoke region_prepare_merge(4, 5); +┌─region_prepare_merge(4, 5)───────────────┐ +│ execute prepare merge, source 4 target 5 │ +└──────────────────────────────────────────┘ +=> DBGInvoke dump_all_mock_region(default, test) +┌─dump_all_mock_region(default, test)────────┐ +│ [region 5] ranges: [4, 100), state: Normal │ +│ [region 4] ranges: [0, 4), state: Merging │ +│ total size: 2 │ +└────────────────────────────────────────────┘ +=> DBGInvoke region_commit_merge(4, 5); +┌─region_commit_merge(4, 5)────────────────┐ +│ execute commit merge, source 4 current 5 │ +└──────────────────────────────────────────┘ +=> DBGInvoke dump_all_mock_region(default, test) +┌─dump_all_mock_region(default, test)────────┐ +│ [region 5] ranges: [0, 100), state: Normal │ +│ total size: 1 │ +└────────────────────────────────────────────┘ +=> DBGInvoke __try_flush_region(5) +=> DBGInvoke dump_all_mock_region(default, test) +┌─dump_all_mock_region(default, test)────────┐ +│ [region 5] ranges: [0, 100), state: Normal │ +│ total size: 1 │ +└────────────────────────────────────────────┘ +=> select _tidb_rowid, col_1 from default.test order by _tidb_rowid +┌─_tidb_rowid─┬─col_1─┐ +│ 1 │ 12 │ +│ 2 │ 10 │ +│ 3 │ 14 │ +│ 5 │ 16 │ +│ 10 │ 19 │ +│ 11 │ 18 │ +└─────────────┴───────┘ +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/delta-merge-test/raft/schema/alter_for_nullable.test b/tests/delta-merge-test/raft/schema/alter_for_nullable.test new file mode 100644 index 00000000000..d61cc50b39d --- /dev/null +++ b/tests/delta-merge-test/raft/schema/alter_for_nullable.test @@ -0,0 +1,70 @@ + +# Preparation. +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Sync add column by reading. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8, col_3 Int32', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1, 3) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2, 4) +=> DBGInvoke __try_flush_region(4) + +# test add nullable flag and change type at the same time. +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int32)') +# test trigger by background worker. +=> DBGInvoke __refresh_schemas() +=> select col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +└───────┘ + +# test only add nullable. +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_3 Nullable(Int32)') + +=> DBGInvoke __put_region(5, 100, 150, default, test) +=> DBGInvoke __raft_insert_row(default, test, 5, 100, 'test', 1, NULL) +=> DBGInvoke __raft_insert_row(default, test, 5, 101, 'test', 2, NULL) +# test trigger by flush worker. +=> DBGInvoke __try_flush_region(5) + +=> select col_3 from default.test +┌─col_2─┐ +│ 3 │ +│ 4 │ +│ \N │ +│ \N │ +└───────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +# Test convert nullable type to not-null type. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)', '', 'dt') +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) +=> DBGInvoke __try_flush_region(4) +=> select col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +└───────┘ +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Int16') +=> DBGInvoke __refresh_schemas() +=> select col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +└───────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() diff --git a/tests/delta-merge-test/raft/schema/alter_on_read.test b/tests/delta-merge-test/raft/schema/alter_on_read.test new file mode 100644 index 00000000000..46c3cb4daf5 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/alter_on_read.test @@ -0,0 +1,215 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Sync add column by reading. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 1) +# Data in region cache with tso greater than read tso will be ignored. +=> select * from default.test " --read_tso "1500000000000000 +# Data in region cache with tso less than read tso will be force decoded (extra column will be discarded), even under an out-of-date schema. +=> select col_1 from default.test +┌─col_1─┐ +│ test1 │ +└───────┘ + +# For engine DeltaTree, each write will trigger schema sync. So schema is fresh here. +## Verify this schema is truely out-of-date. +#=> select col_2 from default.test +#Received exception from server (version {#WORD}): +#Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_2. +# Read with specified bigger schema version will trigger schema sync. +=> select col_2 from default.test " --schema_version "10000000 +┌─col_2─┐ +│ 1 │ +└───────┘ + +# Sync drop column by reading. +=> DBGInvoke __try_flush_region(4) +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 2) + +# For engine DeltaTree, each write will trigger schema sync. So schema is fresh here. +## Data in region cache with tso less than read tso will be force decoded (missing column will be filled with default value or null), even under an out-of-date schema. +#=> select col_1 from default.test +#┌─col_1─┐ +#│ test1 │ +#│ │ +#└───────┘ +# Read with specified bigger schema version will trigger schema sync. +=> select col_1 from default.test " --schema_version "10000000 +Received exception from server (version {#WORD}): +Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_1. + +# Sync type change by checking sign overflow in CH when flushing. +=> DBGInvoke __try_flush_region(4) +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int16)') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, -128) +=> select col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +│ -128 │ +└───────┘ +=> desc default.test +┌─name────────┬─type───────────┬─default_type─┬─default_expression─┐ +│ _tidb_rowid │ Int64 │ │ │ +│ col_2 │ Nullable(Int8) │ │ │ +└─────────────┴────────────────┴──────────────┴────────────────────┘ +=> DBGInvoke __raft_insert_row(default, test, 4, 53, 128) + +# For engine DeltaTree, each write will trigger schema sync. So schema is fresh here. +## 128 will overflow when decoding using out-of-date schema (Int8). +#=> select col_2 from default.test +#Received exception from server (version {#WORD}): +#Code: 49. DB::Exception: Received from {#WORD} DB::Exception: Detected overflow when decoding data 128 of column col_2 with type Nullable(Int8). +# Read with specified bigger schema version will trigger schema sync. +=> select col_2 from default.test " --schema_version "10000000 +┌─col_2─┐ +│ 1 │ +│ 2 │ +│ -128 │ +│ 128 │ +└───────┘ + +# Sync type change by checking value overflow in CH when flushing. +=> DBGInvoke __try_flush_region(4) +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int64)') +=> DBGInvoke __raft_insert_row(default, test, 4, 54, 65536) + +# For engine DeltaTree, each write will trigger schema sync. So schema is fresh here. +## 65536 will overflow when decoding using out-of-date schema (Int16). +#=> select col_2 from default.test +#Received exception from server (version {#WORD}): +#Code: 49. DB::Exception: Received from {#WORD} DB::Exception: Detected overflow when decoding data 65536 of column col_2 with type Nullable(Int16). +# Read with specified bigger schema version will trigger schema sync. +=> select col_2 from default.test " --schema_version "10000000 +┌─col_2─┐ +│ 1 │ +│ 2 │ +│ -128 │ +│ 128 │ +│ 65536 │ +└───────┘ + +# Sync add column and type change together by checking value overflow in CH when flushing. +=> DBGInvoke __try_flush_region(4) +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 UInt8') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_3 UInt64') +=> DBGInvoke __raft_insert_row(default, test, 4, 55, 0, 256) +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_4 Nullable(UInt8)') + +# For engine DeltaTree, each write will trigger schema sync. So schema is fresh here. +## 256 will overflow when decoding using out-of-date schema (UInt8). +#=> select col_3 from default.test +#Received exception from server (version {#WORD}): +#Code: 49. DB::Exception: Received from {#WORD} DB::Exception: Detected overflow when decoding data 256 of column col_3 with type UInt8. +# Read with specified bigger schema version will trigger schema sync. +=> select col_3, col_4 from default.test " --schema_version "10000000 +┌─col_3─┬─col_4─┐ +│ 0 │ \N │ +│ 0 │ \N │ +│ 0 │ \N │ +│ 0 │ \N │ +│ 0 │ \N │ +│ 256 │ \N │ +└───────┴───────┘ + +# Edge values not overflowing. +=> DBGInvoke __try_flush_region(4) +=> DBGInvoke __raft_insert_row(default, test, 4, 56, -9223372036854775807, 18446744073709551615, 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 57, 9223372036854775807, 18446744073709551615, 1) +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_3) +# Here engine DeltaTree don't trigger schema sync too +=> select col_2, col_3, col_4 from default.test +┌─col_2─┬─col_3─┬─col_4─┐ +│ 1 │ 0 │ \N │ +│ 2 │ 0 │ \N │ +└───────┴───────┴───────┘ +┌─col_2─┬─col_3─┬─col_4─┐ +│ -128 │ 0 │ \N │ +│ 128 │ 0 │ \N │ +│ 65536 │ 0 │ \N │ +└───────┴───────┴───────┘ +┌────────────────col_2─┬────────────────col_3─┬─col_4─┐ +│ 0 │ 256 │ \N │ +│ -9223372036854775807 │ 18446744073709551615 │ 1 │ +└──────────────────────┴──────────────────────┴───────┘ +┌───────────────col_2─┬────────────────col_3─┬─col_4─┐ +│ 9223372036854775807 │ 18446744073709551615 │ 1 │ +└─────────────────────┴──────────────────────┴───────┘ +=> select col_3, col_4 from default.test " --schema_version "100000000 +Received exception from server (version {#WORD}): +Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_3. + +# Sync drop column and type change together by checking value overflow in CH when flushing. +=> DBGInvoke __try_flush_region(4) +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_4 Nullable(UInt64)') +=> DBGInvoke __raft_insert_row(default, test, 4, 58, 0, 256) +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_2) +# For engine DeltaTree, each write will trigger schema sync. So schema is fresh here. +## -256 will overflow when decoding using out-of-date schema (UInt8). +#=> select col_4 from default.test +#Received exception from server (version {#WORD}): +#Code: 49. DB::Exception: Received from {#WORD} DB::Exception: Detected overflow when decoding data 256 of column col_4 with type Nullable(UInt8). +# Read with specified bigger schema version will trigger schema sync. +=> select col_4 from default.test " --schema_version "100000000 +┌─col_4─┐ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ 1 │ +│ 1 │ +│ 256 │ +└───────┘ +=> select col_2 from default.test +Received exception from server (version {#WORD}): +Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_2. +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') +=> DBGInvoke __refresh_schemas() +=> select col_2 from default.test +┌─col_2─┐ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +│ \N │ +└───────┘ +=> DBGInvoke __reset_schemas() +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_2) + +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Int8') + +=> DBGInvoke __refresh_schemas() + +=> select col_2 from default.test +┌─col_2─┐ +│ 0 │ +│ 0 │ +│ 0 │ +│ 0 │ +│ 0 │ +│ 0 │ +│ 0 │ +│ 0 │ +│ 0 │ +└───────┘ +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/alter_on_write.test b/tests/delta-merge-test/raft/schema/alter_on_write.test new file mode 100644 index 00000000000..02ccb8be935 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/alter_on_write.test @@ -0,0 +1,132 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Sync add column by checking missing column in CH when flushing. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> select col_1 from default.test +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 1) +=> DBGInvoke __try_flush_region(4) +=> selraw nokvstore col_2 from default.test +┌─col_2─┐ +│ 1 │ +└───────┘ + +# Sync drop column by checking redundant column in CH when flushing. +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 2) +=> DBGInvoke __try_flush_region(4) +=> selraw nokvstore col_2 from default.test +┌─col_1─┐ +│ 1 │ +│ 2 │ +└───────┘ +=> selraw nokvstore col_1 from default.test +Received exception from server (version {#WORD}): +Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_1. + +# Sync type change by checking sign overflow in CH when flushing. +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int16)') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, -128) +=> DBGInvoke __try_flush_region(4) +=> selraw nokvstore col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +│ -128 │ +└───────┘ +=> desc default.test +┌─name────────┬─type───────────┬─default_type─┬─default_expression─┐ +│ _tidb_rowid │ Int64 │ │ │ +│ col_2 │ Nullable(Int8) │ │ │ +└─────────────┴────────────────┴──────────────┴────────────────────┘ +=> DBGInvoke __raft_insert_row(default, test, 4, 53, 128) +=> DBGInvoke __try_flush_region(4) +=> selraw nokvstore col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +│ -128 │ +│ 128 │ +└───────┘ + +# Sync type change by checking value overflow in CH when flushing. +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int64)') +=> DBGInvoke __raft_insert_row(default, test, 4, 54, 65536) +=> DBGInvoke __try_flush_region(4) +=> selraw nokvstore col_2 from default.test +┌─col_2─┐ +│ 1 │ +│ 2 │ +│ -128 │ +│ 128 │ +│ 65536 │ +└───────┘ + +# Sync add column and type change together by checking value overflow in CH when flushing. +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 UInt8') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_3 UInt64') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_4 Nullable(UInt8)') +# For DeltaTree, each write will trigger schema sync. +=> DBGInvoke __raft_insert_row(default, test, 4, 55, 0, 256, 0) +=> selraw nokvstore col_3, col_4 from default.test +┌─col_3─┬─col_4─┐ +│ 0 │ \N │ +│ 0 │ \N │ +│ 0 │ \N │ +│ 0 │ \N │ +│ 0 │ \N │ +│ 256 │ 0 │ +└───────┴───────┘ + +# Not sync drop column for edge values not overflowing. +=> DBGInvoke __raft_insert_row(default, test, 4, 56, -9223372036854775807, 18446744073709551615, 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 57, 9223372036854775807, 18446744073709551615, 1) +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_3) +=> DBGInvoke __try_flush_region(4) +=> selraw nokvstore col_2, col_3, col_4 from default.test +┌─col_2─┬─col_3─┬─col_4─┐ +│ 1 │ 0 │ \N │ +│ 2 │ 0 │ \N │ +│ -128 │ 0 │ \N │ +│ 128 │ 0 │ \N │ +│ 65536 │ 0 │ \N │ +└───────┴───────┴───────┘ +┌─col_2─┬─col_3─┬─col_4─┐ +│ 0 │ 256 │ 0 │ +└───────┴───────┴───────┘ +┌────────────────col_2─┬────────────────col_3─┬─col_4─┐ +│ -9223372036854775807 │ 18446744073709551615 │ 1 │ +│ 9223372036854775807 │ 18446744073709551615 │ 1 │ +└──────────────────────┴──────────────────────┴───────┘ +=> DBGInvoke __refresh_schemas() +=> selraw nokvstore col_3 from default.test +Received exception from server (version {#WORD}): +Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_3. + +# Sync drop column and type change together by checking value overflow in CH when flushing. +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_4 Nullable(UInt64)') +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_2) +# For DeltaTree, each write will trigger schema sync. +=> DBGInvoke __raft_insert_row(default, test, 4, 58, 256) +=> desc default.test +┌─name────────┬─type─────────────┬─default_type─┬─default_expression─┐ +│ _tidb_rowid │ Int64 │ │ │ +│ col_4 │ Nullable(UInt64) │ │ │ +└─────────────┴──────────────────┴──────────────┴────────────────────┘ +=> selraw nokvstore col_2 from default.test +Received exception from server (version {#WORD}): +Code: 47. DB::Exception: Received from {#WORD} DB::Exception: Unknown identifier: col_2. + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/default_value.test b/tests/delta-merge-test/raft/schema/default_value.test new file mode 100644 index 00000000000..ab097ff5f69 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/default_value.test @@ -0,0 +1,42 @@ +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') + +=> DBGInvoke __put_region(4, 0, 100, default, test) + +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, 'test2') +=> DBGInvoke __raft_insert_row(default, test, 4, 53, 'test2') +=> DBGInvoke __try_flush_region(4) + +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int64) DEFAULT 123') +# test default value of time type +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 DEFAULT \'asTiDBType|Nullable(Time)|11:11:11\'') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_4 DEFAULT \'asTiDBType|Nullable(Time(3))|-11:11:11.111\'') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_5 DEFAULT \'asTiDBType|Nullable(Time(3))|111:11:11.111\'') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_6 DEFAULT \'asTiDBType|Nullable(Time)|-111:11:11\'') +# test default value of year type +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_7 DEFAULT \'asTiDBType|Nullable(year)|0\'') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_8 DEFAULT \'asTiDBType|Nullable(year)|70\'') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_9 DEFAULT \'asTiDBType|Nullable(year)|1986\'') +# test default value of set type +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_10 DEFAULT \'asTiDBType|Nullable(set(\\\\\'1\\\\\',\\\\\'4\\\\\',\\\\\'10\\\\\',\\\\\'20\\\\\'))|1,10\'') + +=> DBGInvoke __put_region(5, 100, 200, default, test) +=> DBGInvoke __raft_insert_row(default, test, 5, 154, 'test2', 321, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) +=> DBGInvoke __raft_insert_row(default, test, 5, 155, 'test2', 321, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) +=> DBGInvoke __try_flush_region(5) + +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┬─col_2─┬──────────col_3─┬───────────col_4─┬───────────col_5─┬────────────col_6─┬─col_7─┬─col_8─┬─col_9─┬─col_10─┐ +│ test2 │ 154 │ 321 │ \N │ \N │ \N │ \N │ \N │ \N │ \N │ \N │ +│ test2 │ 155 │ 321 │ \N │ \N │ \N │ \N │ \N │ \N │ \N │ \N │ +│ test2 │ 51 │ 123 │ 40271000000000 │ -40271111000000 │ 400271111000000 │ -400271000000000 │ 2000 │ 1970 │ 1986 │ 5 │ +│ test2 │ 52 │ 123 │ 40271000000000 │ -40271111000000 │ 400271111000000 │ -400271000000000 │ 2000 │ 1970 │ 1986 │ 5 │ +│ test2 │ 53 │ 123 │ 40271000000000 │ -40271111000000 │ 400271111000000 │ -400271000000000 │ 2000 │ 1970 │ 1986 │ 5 │ +└───────┴─────────────┴───────┴────────────────┴─────────────────┴─────────────────┴──────────────────┴───────┴───────┴───────┴────────┘ +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/drop_on_read.test b/tests/delta-merge-test/raft/schema/drop_on_read.test new file mode 100644 index 00000000000..2cb42140300 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/drop_on_read.test @@ -0,0 +1,27 @@ +=> DBGInvoke __clean_up_region() +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __drop_tidb_table(default, test, 'false') +=> select * from default.test +=> select * from default.test " --schema_version "10000000 +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. +=> DBGInvoke __clean_up_region() + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)', '', 'dt') +=> select * from default.test +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. +=> select * from default.test " --schema_version "10000000 + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/drop_on_restart.test b/tests/delta-merge-test/raft/schema/drop_on_restart.test new file mode 100644 index 00000000000..fe61a736a6b --- /dev/null +++ b/tests/delta-merge-test/raft/schema/drop_on_restart.test @@ -0,0 +1,54 @@ +=> DBGInvoke __clean_up_region() +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> DBGInvoke __drop_tidb_db(db3) +=> DBGInvoke __drop_tidb_db(db4) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_db(db3) +=> DBGInvoke __mock_tidb_db(db4) +=> DBGInvoke __mock_tidb_table(db3, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(db4, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> show databases +┌─name────┐ +│ db3 │ +│ db4 │ +│ default │ +│ system │ +└─────────┘ +=> DBGInvoke __reset_schemas() +=> DBGInvoke __drop_tidb_db(db3); +=> DBGInvoke __drop_tidb_table(db4, test); +=> DBGInvoke __refresh_schemas() +=> show databases +┌─name────┐ +│ db4 │ +│ default │ +│ system │ +└─────────┘ +=> select * from db4.test +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table db4.test doesn't exist.. +=> DBGInvoke __mock_tidb_db(db3) +=> DBGInvoke __refresh_schemas() +=> show databases +┌─name────┐ +│ db3 │ +│ db4 │ +│ default │ +│ system │ +└─────────┘ +=> DBGInvoke __reset_schemas() +=> DBGInvoke __drop_tidb_db(db3); +=> DBGInvoke __drop_tidb_db(db4); +=> DBGInvoke __refresh_schemas() +=> show databases +┌─name────┐ +│ default │ +│ system │ +└─────────┘ diff --git a/tests/delta-merge-test/raft/schema/drop_on_write.test b/tests/delta-merge-test/raft/schema/drop_on_write.test new file mode 100644 index 00000000000..fa8ff9585a4 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/drop_on_write.test @@ -0,0 +1,67 @@ +#TODO: For DeltaTree, each write will trigger schema sync. So we can not mock this situation, this test is useless +#RETURN + +=> DBGInvoke __clean_up_region() +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> select col_1 from default.test +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 1) +=> DBGInvoke __drop_tidb_table(default, test, 'false') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. + +=> DBGInvoke __clean_up_region() +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)', '', 'dt') +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __refresh_schemas() +=> select col_1, col_2 from default.test +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_2) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2') +=> DBGInvoke __drop_tidb_table(default, test, 'false') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +┌─col_1─┬─col_2─┬─_tidb_rowid─┐ +│ test2 │ \N │ 51 │ +└───────┴───────┴─────────────┘ + +=> DBGInvoke __clean_up_region() +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __refresh_schemas() +=> select col_1, col_2 from default.test +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_2) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2') +=> DBGInvoke __drop_tidb_table(default, test, 'false') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. + +=> DBGInvoke __clean_up_region() +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)', '', 'dt') +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __refresh_schemas() +=> select col_1, col_2 from default.test +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int64)') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, 'test2', 256) +=> DBGInvoke __drop_tidb_table(default, test, 'false') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') +=> DBGInvoke __clean_up_region() diff --git a/tests/delta-merge-test/raft/schema/mydate.test b/tests/delta-merge-test/raft/schema/mydate.test new file mode 100644 index 00000000000..b8fd45b8237 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/mydate.test @@ -0,0 +1,73 @@ + +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDateTime(1)', '', 'dt') + +=> DBGInvoke __put_region(4, 0, 100, default, test) + +=> DBGInvoke __raft_insert_row(default, test, 4, 51, '1991-11-12 11:12:13.234') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, '1991-00-14 11:00:01') +=> DBGInvoke __raft_insert_row(default, test, 4, 53, '2001-12-13 11:11:11') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test + +┌───────────────col_1───┬─_tidb_rowid─┐ +│ 1991-11-12 11:12:13.2 │ 51 │ +│ 1991-00-14 11:00:01.0 │ 52 │ +│ 2001-12-13 11:11:11.0 │ 53 │ +└───────────────────────┴─────────────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDate', '', 'dt') + +=> DBGInvoke __put_region(4, 0, 100, default, test) + +=> DBGInvoke __raft_insert_row(default, test, 4, 51, '1991-11-12') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, '1991-00-14') +=> DBGInvoke __raft_insert_row(default, test, 4, 53, '2001-12-13') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test + +┌──────col_1─┬─_tidb_rowid─┐ +│ 1991-11-12 │ 51 │ +│ 1991-00-14 │ 52 │ +│ 2001-12-13 │ 53 │ +└────────────┴─────────────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDateTime(1)', '', 'dt') + +=> DBGInvoke __put_region(4, 0, 100, default, test) + +=> DBGInvoke __raft_insert_row(default, test, 4, 51, '1991-11-12 11:12:13.234') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, '1991-01-14 11:00:01') +=> DBGInvoke __raft_insert_row(default, test, 4, 53, '2001-12-13 11:11:11') +=> DBGInvoke __raft_insert_row(default, test, 4, 54, '1991-09-05 11:00:01') +=> DBGInvoke __try_flush_region(4) +=> select * from default.test + +┌───────────────col_1───┬─_tidb_rowid─┐ +│ 1991-11-12 11:12:13.2 │ 51 │ +│ 1991-01-14 11:00:01.0 │ 52 │ +│ 2001-12-13 11:11:11.0 │ 53 │ +│ 1991-09-05 11:00:01.0 │ 54 │ +└───────────────────────┴─────────────┘ + +=> select ConvertTimeZoneFromUTC(col_1, 'Asia/Shanghai') as col_1 , _tidb_rowid from default.test +┌───────────────col_1───┬─_tidb_rowid─┐ +│ 1991-11-12 19:12:13.2 │ 51 │ +│ 1991-01-14 19:00:01.0 │ 52 │ +│ 2001-12-13 19:11:11.0 │ 53 │ +│ 1991-09-05 20:00:01.0 │ 54 │ +└───────────────────────┴─────────────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/partition_table_restart.test b/tests/delta-merge-test/raft/schema/partition_table_restart.test new file mode 100644 index 00000000000..26be87b70e8 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/partition_table_restart.test @@ -0,0 +1,43 @@ +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> DBGInvoke __drop_tidb_table(default, test1) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_partition(default, test, 9999) +=> DBGInvoke __mock_tidb_partition(default, test, 9998) +=> DBGInvoke __mock_tidb_partition(default, test, 9997) +=> DBGInvoke __refresh_schemas() +=> drop table test_9997 +=> DBGInvoke __reset_schemas() + +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 Nullable(Int8)') +=> DBGInvoke __rename_tidb_table(default, test, test1) +=> DBGInvoke __refresh_schemas() + +=> show tables +┌─name───────┐ +│ test1 │ +│ test1_9997 │ +│ test1_9998 │ +│ test1_9999 │ +└────────────┘ +=> select col_2 from default.test1_9997 +=> select * from default.test_9997 +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test_9997 doesn't exist.. +=> select * from default.test_9998 +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test_9998 doesn't exist.. + +=> DBGInvoke __reset_schemas() +=> DBGInvoke __drop_tidb_partition(default, test1, 9997) +=> DBGInvoke __refresh_schemas() +=> select * from default.test1_9997 +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test1_9997 doesn't exist.. +=> select * from default.test1_9999 +=> DBGInvoke __drop_tidb_table(default, test1) +=> DBGInvoke __refresh_schemas() + +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/rename_column copy.test b/tests/delta-merge-test/raft/schema/rename_column copy.test new file mode 100644 index 00000000000..961245ee6c4 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/rename_column copy.test @@ -0,0 +1,216 @@ + +# Preparation. +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# create table and insert some rows +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +┌─col_1─┬─col_2─┬─_tidb_rowid─┐ +│ test │ 1 │ 50 │ +│ test │ 2 │ 51 │ +└───────┴───────┴─────────────┘ + +# test cyclic rename +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_1, col_3) +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_2, col_1) +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_3, col_2) +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_1 Nullable(Int32)') +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(String)') +=> DBGInvoke __refresh_schemas() +# test doesn't check column name, so we select columns one by one. +=> select col_2 from default.test +┌─col_2─┐ +│ test │ +│ test │ +└───────┘ +=> select col_1 from default.test +┌─col_1─┐ +│ 1 │ +│ 2 │ +└───────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +┌─col_1─┬─col_2─┬─_tidb_rowid─┐ +│ test │ 1 │ 50 │ +│ test │ 2 │ 51 │ +└───────┴───────┴─────────────┘ + +# test change nullable firstly, then rename cyclic. + +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_1 Nullable(String)') +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_2 Nullable(Int32)') +=> DBGInvoke __refresh_schemas() +=> select * from default.test +┌─col_1─┬─col_2─┬─_tidb_rowid─┐ +│ test │ 1 │ 50 │ +│ test │ 2 │ 51 │ +└───────┴───────┴─────────────┘ + +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_1, col_3) +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_2, col_1) +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_3, col_2) +=> DBGInvoke __refresh_schemas() +=> select col_2 from default.test +┌─col_2─┐ +│ test │ +│ test │ +└───────┘ +=> select col_1 from default.test +┌─col_1─┐ +│ 1 │ +│ 2 │ +└───────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +┌─col_1─┬─col_2─┬─_tidb_rowid─┐ +│ test │ 1 │ 50 │ +│ test │ 2 │ 51 │ +└───────┴───────┴─────────────┘ + +# test drop and then rename and then create + +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_1) +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_2, col_1) +=> DBGInvoke __refresh_schemas() +=> select col_1 from default.test +┌─col_1─┐ +│ 1 │ +│ 2 │ +└───────┘ +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8) default 0') +=> DBGInvoke __refresh_schemas() +=> select col_2 from default.test +┌─col_2─┐ +│ 0 │ +│ 0 │ +└───────┘ +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + +# do a lot of change and sync at once. + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) +=> DBGInvoke __try_flush_region(4) +=> select * from default.test +┌─col_1─┬─col_2─┬─_tidb_rowid─┐ +│ test │ 1 │ 50 │ +│ test │ 2 │ 51 │ +└───────┴───────┴─────────────┘ + +# test drop and then rename and then create + +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_1) +=> DBGInvoke __rename_column_in_tidb_table(default, test, col_2, col_1) +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8) default 0') +=> DBGInvoke __modify_column_in_tidb_table(default, test, 'col_1 Nullable(Int32)') +=> DBGInvoke __reset_schemas() +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(5, 101, 200, default, test) +=> DBGInvoke __raft_insert_row(default, test, 5, 150, NULL, NULL) +=> DBGInvoke __raft_insert_row(default, test, 5, 151, NULL, NULL) +=> DBGInvoke __try_flush_region(5) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┬─col_2─┐ +│ 1 │ 50 │ 0 │ +│ 2 │ 51 │ 0 │ +│ \N │ 150 │ \N │ +│ \N │ 151 │ \N │ +└───────┴─────────────┴───────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() + +## test for partial-linked rename +=> DBGInvoke __mock_tidb_table(default, test, 'a String, b Int8', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) +=> DBGInvoke __try_flush_region(4) +=> select a, b from default.test order by _tidb_rowid +┌─a────┬─b─┐ +│ test │ 1 │ +│ test │ 2 │ +└──────┴───┘ + +# rename a -> c, and b -> a +=> DBGInvoke __rename_column_in_tidb_table(default, test, a, c) +=> DBGInvoke __rename_column_in_tidb_table(default, test, b, a) +=> DBGInvoke __refresh_schemas() +=> select a, c from default.test order by _tidb_rowid +┌─a─┬─c────┐ +│ 1 │ test │ +│ 2 │ test │ +└───┴──────┘ + +# rename after add column +=> DBGInvoke __add_column_to_tidb_table(default, test, 'k Nullable(Int8) default 0') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __rename_column_in_tidb_table(default, test, k, g) +=> DBGInvoke __refresh_schemas() +=> select a, g from default.test order by _tidb_rowid +┌─a─┬─g─┐ +│ 1 │ 0 │ +│ 2 │ 0 │ +└───┴───┘ + +=> DBGInvoke __rename_column_in_tidb_table(default, test, a, d) +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __rename_column_in_tidb_table(default, test, g, k) +=> DBGInvoke __refresh_schemas() +=> select d, k from default.test order by _tidb_rowid +┌─d─┬─k─┐ +│ 1 │ 0 │ +│ 2 │ 0 │ +└───┴───┘ + +=> DBGInvoke __rename_tidb_table(default, test, test1) +=> DBGInvoke __add_column_to_tidb_table(default, test1, 'z Nullable(Int8) default 0') +=> DBGInvoke __refresh_schemas() +=> select d, z from default.test1 order by _tidb_rowid +┌─d─┬─z─┐ +│ 1 │ 0 │ +│ 2 │ 0 │ +└───┴───┘ + +=> DBGInvoke __drop_tidb_table(default, test1) +=> drop table if exists default.test +=> DBGInvoke __refresh_schemas() diff --git a/tests/delta-merge-test/raft/schema/rename_column.test b/tests/delta-merge-test/raft/schema/rename_column.test index 0e4029e27fe..28b01b5ba7f 100644 --- a/tests/delta-merge-test/raft/schema/rename_column.test +++ b/tests/delta-merge-test/raft/schema/rename_column.test @@ -1,6 +1,3 @@ -#TODO: enable ddl tests for DeltaMerge -#RETURN - # Preparation. => DBGInvoke __enable_schema_sync_service('false') diff --git a/tests/delta-merge-test/raft/schema/rename_on_read.test b/tests/delta-merge-test/raft/schema/rename_on_read.test new file mode 100644 index 00000000000..7d077bf6d3b --- /dev/null +++ b/tests/delta-merge-test/raft/schema/rename_on_read.test @@ -0,0 +1,24 @@ +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __drop_tidb_table(default, test1) +=> drop table if exists default.test1 + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> select * from default.test +=> DBGInvoke __rename_tidb_table(default, test, test1) +=> select * from default.test +=> select * from default.test " --schema_version "1000000 +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. +=> select * from default.test1 +=> select * from default.test1 " --schema_version "1000000 + +=> DBGInvoke __drop_tidb_table(default, test1) +=> drop table if exists default.test1 +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/rename_on_write.test b/tests/delta-merge-test/raft/schema/rename_on_write.test new file mode 100644 index 00000000000..522e9ad3702 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/rename_on_write.test @@ -0,0 +1,32 @@ +#TODO: We can not mock this situation, ignore for now +#RETURN + +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __drop_tidb_table(default, test1) +=> drop table if exists default.test1 + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> select col_1 from default.test +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') +=> DBGInvoke __rename_tidb_table(default, test, test1) +#For DeltaTree, each write will trigger schema sync. +=> DBGInvoke __raft_insert_row(default, test1, 4, 50, 'test1', 1) +=> select * from default.test +Received exception from server (version {#WORD}): +Code: 60. DB::Exception: Received from {#WORD} DB::Exception: Table default.test doesn't exist.. +=> select * from default.test1 +┌─col_1─┬─_tidb_rowid─┬─col_2─┐ +│ test1 │ 50 │ 1 │ +└───────┴─────────────┴───────┘ + +=> DBGInvoke __drop_tidb_table(default, test1) +=> drop table if exists default.test1 +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/delta-merge-test/raft/schema/truncate_on_read.test b/tests/delta-merge-test/raft/schema/truncate_on_read.test new file mode 100644 index 00000000000..09c71658fad --- /dev/null +++ b/tests/delta-merge-test/raft/schema/truncate_on_read.test @@ -0,0 +1,27 @@ +=> DBGInvoke __clean_up_region() +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1') +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ test1 │ 50 │ +└───────┴─────────────┘ +=> DBGInvoke __truncate_tidb_table(default, test) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ test1 │ 50 │ +└───────┴─────────────┘ +=> select * from default.test " --schema_version "10000000 + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') +=> DBGInvoke __clean_up_region() diff --git a/tests/delta-merge-test/raft/schema/truncate_on_write.test b/tests/delta-merge-test/raft/schema/truncate_on_write.test new file mode 100644 index 00000000000..c6dc4e1e2c1 --- /dev/null +++ b/tests/delta-merge-test/raft/schema/truncate_on_write.test @@ -0,0 +1,29 @@ +#TODO: We can not mock this situation, ignore for now +#RETURN + +=> DBGInvoke __clean_up_region() +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1') +=> select col_1 from default.test +┌─col_1─┐ +│ test1 │ +└───────┘ +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test1', 1) +=> DBGInvoke __truncate_tidb_table(default, test) +=> DBGInvoke __try_flush_region(4) +=> select * from default.test + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') +=> DBGInvoke __clean_up_region() diff --git a/tests/docker/cluster.yaml b/tests/docker/cluster.yaml new file mode 100644 index 00000000000..1876eddc893 --- /dev/null +++ b/tests/docker/cluster.yaml @@ -0,0 +1,56 @@ +version: '2.3' + +services: + pd0: + image: hub.pingcap.net/qa/pd:${BRANCH:-master} + ports: + - "2379:2379" + volumes: + - ./config/pd.toml:/pd.toml:ro + - ./data/pd0:/data + - ./log/pd0:/log + command: + - --name=pd0 + - --client-urls=http://0.0.0.0:2379 + - --peer-urls=http://0.0.0.0:2380 + - --advertise-client-urls=http://pd0:2379 + - --advertise-peer-urls=http://pd0:2380 + - --initial-cluster=pd0=http://pd0:2380 + - --config=/pd.toml + - --data-dir=/data + - --log-file=/log/pd.log + restart: on-failure + tikv0: + image: hub.pingcap.net/qa/tikv:${BRANCH:-master} + ports: + - "20160:20160" + volumes: + - ./config/tikv.toml:/tikv.toml:ro + - ./data/tikv0:/data + - ./log/tikv0:/log + command: + - --addr=0.0.0.0:20160 + - --advertise-addr=tikv0:20160 + - --pd=pd0:2379 + - --config=/tikv.toml + - --data-dir=/data + - --log-file=/log/tikv.log + depends_on: + - "pd0" + restart: on-failure + tidb0: + image: hub.pingcap.net/qa/tidb:${BRANCH:-master} + ports: + - "4000:4000" + - "10080:10080" + volumes: + - ./config/tidb.toml:/tidb.toml:ro + - ./log/tidb0:/log + command: + - --store=tikv + - --path=pd0:2379 + - --config=/tidb.toml + - --log-file=/log/tidb.log + depends_on: + - "tikv0" + restart: on-failure diff --git a/tests/docker/config/tics_dt.toml b/tests/docker/config/tics_dt.toml new file mode 100644 index 00000000000..ebe665ea166 --- /dev/null +++ b/tests/docker/config/tics_dt.toml @@ -0,0 +1,23 @@ +tmp_path = "/tmp/tiflash/data/tmp" +display_name = "TiFlash" +default_profile = "default" +users_config = "users.toml" +# specify paths used for store data, multiple path should be seperated by comma +path = "/tmp/tiflash/data/db" +mark_cache_size = 5368709120 +minmax_index_cache_size = 5368709120 +listen_host = "0.0.0.0" +tcp_port = 9000 +http_port = 8123 +interserver_http_port = 9009 +[logger] + count = 10 + errorlog = "/tmp/tiflash/log/error.log" + size = "1000M" + log = "/tmp/tiflash/log/server.log" + level = "trace" +[application] + runAsDaemon = true +[raft] + # specify which storage engine we use. tmt or dt + storage_engine = "dt" diff --git a/tests/docker/config/config.toml b/tests/docker/config/tics_tmt.toml similarity index 88% rename from tests/docker/config/config.toml rename to tests/docker/config/tics_tmt.toml index fcbd28a90ca..13a7109b52d 100644 --- a/tests/docker/config/config.toml +++ b/tests/docker/config/tics_tmt.toml @@ -19,6 +19,5 @@ interserver_http_port = 9009 [application] runAsDaemon = true [raft] - # specify which storage engine we use. tmt or dm + # specify which storage engine we use. tmt or dt storage_engine = "tmt" - disable_bg_flush = false diff --git a/tests/docker/config/tiflash_dt.toml b/tests/docker/config/tiflash_dt.toml new file mode 100644 index 00000000000..ff609f6e7b9 --- /dev/null +++ b/tests/docker/config/tiflash_dt.toml @@ -0,0 +1,39 @@ +tmp_path = "/tmp/tiflash/data/tmp" +display_name = "TiFlash" +default_profile = "default" +users_config = "users.toml" +# specify paths used for store data, multiple path should be seperated by comma +path = "/tmp/tiflash/data/db" +mark_cache_size = 5368709120 +minmax_index_cache_size = 5368709120 +listen_host = "0.0.0.0" +tcp_port = 9000 +http_port = 8123 +interserver_http_port = 9009 +[flash] + tidb_status_addr = "tidb0:10080" + service_addr = "tiflash0:3930" + [flash.flash_cluster] + refresh_interval = 20 + update_rule_interval = 5 + master_ttl = 60 + cluster_manager_path = "/tiflash/flash_cluster_manager" + [flash.proxy] + addr = "0.0.0.0:20170" + advertise-addr = "tiflash0:20170" + data-dir = "/data" + config = "/tikv.toml" + log-file = "/log/tikv.log" +[logger] + count = 10 + errorlog = "/tmp/tiflash/log/error.log" + size = "1000M" + log = "/tmp/tiflash/log/server.log" + level = "trace" +[application] + runAsDaemon = true + +[raft] + pd_addr = "pd0:2379" + # specify which storage engine we use. tmt or dt + storage_engine = "dt" diff --git a/tests/docker/config/tiflash.toml b/tests/docker/config/tiflash_tmt.toml similarity index 92% rename from tests/docker/config/tiflash.toml rename to tests/docker/config/tiflash_tmt.toml index 56e999cf709..83e0672af07 100644 --- a/tests/docker/config/tiflash.toml +++ b/tests/docker/config/tiflash_tmt.toml @@ -35,6 +35,5 @@ interserver_http_port = 9009 [raft] pd_addr = "pd0:2379" - # specify which storage engine we use. tmt or dm + # specify which storage engine we use. tmt or dt storage_engine = "tmt" - disable_bg_flush = false diff --git a/tests/docker/config/tikv-learner.toml b/tests/docker/config/tikv-learner.toml index 5cb39d2100f..65ff0c07a8e 100644 --- a/tests/docker/config/tikv-learner.toml +++ b/tests/docker/config/tikv-learner.toml @@ -26,6 +26,7 @@ engine-addr = "tiflash0:3930" [raftstore] raftdb-path = "" sync-log = true +capacity = 107374182400 max-leader-missing-duration = "22s" abnormal-leader-missing-duration = "21s" peer-stale-state-check-interval = "20s" diff --git a/tests/docker/config/tikv.toml b/tests/docker/config/tikv.toml index c36ca8eb31e..42e9aba78c4 100644 --- a/tests/docker/config/tikv.toml +++ b/tests/docker/config/tikv.toml @@ -26,6 +26,7 @@ labels = { } [raftstore] raftdb-path = "" sync-log = true +capacity = 107374182400 consistency-check-interval = "10s" raft-log-gc-count-limit = 256 diff --git a/tests/docker/docker-compose.yaml b/tests/docker/docker-compose.yaml deleted file mode 100644 index 91ff093a5f5..00000000000 --- a/tests/docker/docker-compose.yaml +++ /dev/null @@ -1,108 +0,0 @@ -version: '2.3' - -services: - # tics0 container is for tests under mutable-test directory - tics0: - image: hub.pingcap.net/tiflash/tics:${TAG:-master} - ports: - - "3930:3930" - - "9000:9000" - - "8123:8123" - - "9009:9009" - volumes: - - ./config/config.toml:/config.toml:ro - - ./config/users.toml:/users.toml:ro - - ./data/tics0:/tmp/tiflash/data - - ./log/tics0:/tmp/tiflash/log - - ..:/tests - - ./_env.sh:/tests/_env.sh - command: - - --config-file - - /config.toml - # tics-gtest container is for gtest cases - tics-gtest: - image: hub.pingcap.net/tiflash/tics:${TAG:-master} - volumes: - - ./config/config.toml:/config.toml:ro - - ./config/users.toml:/users.toml:ro - - ./log/tics-gtest:/tmp/tiflash/log - - ..:/tests - - ./_env.sh:/tests/_env.sh - entrypoint: sleep infinity # just wait - pd0: - image: hub.pingcap.net/qa/pd:${BRANCH:-master} - ports: - - "2379:2379" - volumes: - - ./config/pd.toml:/pd.toml:ro - - ./data/pd0:/data - - ./log/pd0:/log - command: - - --name=pd0 - - --client-urls=http://0.0.0.0:2379 - - --peer-urls=http://0.0.0.0:2380 - - --advertise-client-urls=http://pd0:2379 - - --advertise-peer-urls=http://pd0:2380 - - --initial-cluster=pd0=http://pd0:2380 - - --config=/pd.toml - - --data-dir=/data - - --log-file=/log/pd.log - restart: on-failure - tikv0: - image: hub.pingcap.net/qa/tikv:${BRANCH:-master} - ports: - - "20160:20160" - volumes: - - ./config/tikv.toml:/tikv.toml:ro - - ./data/tikv0:/data - - ./log/tikv0:/log - command: - - --addr=0.0.0.0:20160 - - --advertise-addr=tikv0:20160 - - --pd=pd0:2379 - - --config=/tikv.toml - - --data-dir=/data - - --log-file=/log/tikv.log - depends_on: - - "pd0" - restart: on-failure - tidb0: - image: hub.pingcap.net/qa/tidb:${BRANCH:-master} - ports: - - "4000:4000" - - "10080:10080" - volumes: - - ./config/tidb.toml:/tidb.toml:ro - - ./log/tidb0:/log - command: - - --store=tikv - - --path=pd0:2379 - - --config=/tidb.toml - - --log-file=/log/tidb.log - depends_on: - - "tikv0" - restart: on-failure - # tiflash0 container is for tests under fullstack-test directory - tiflash0: - image: hub.pingcap.net/tiflash/tics:${TAG:-master} - ports: - - "3930:3930" - - "9000:9000" - - "8123:8123" - - "9009:9009" - - "20170:20170" - volumes: - - ./config/tiflash.toml:/config.toml:ro - - ./config/tiflash-users.toml:/users.toml:ro - - ./data/tiflash0:/tmp/tiflash/data - - ./log/tiflash0:/tmp/tiflash/log - - ..:/tests - - ./_env.sh:/tests/_env.sh - - ./log/tiflash-cluster-manager:/tmp/tiflash/data/tmp - - ./config/tikv-learner.toml:/tikv.toml:ro - - ./data/tikv-learner0:/data - - ./log/tikv-learner0:/log - command: - - --config-file - - /config.toml - restart: on-failure diff --git a/tests/docker/gtest.yaml b/tests/docker/gtest.yaml new file mode 100644 index 00000000000..36f28a356c8 --- /dev/null +++ b/tests/docker/gtest.yaml @@ -0,0 +1,13 @@ +version: '2.3' + +services: + # tics-gtest container is for gtest cases + tics-gtest: + image: hub.pingcap.net/tiflash/tics:${TAG:-master} + volumes: + - ./config/config.toml:/config.toml:ro + - ./config/users.toml:/users.toml:ro + - ./log/tics-gtest:/tmp/tiflash/log + - ..:/tests + - ./_env.sh:/tests/_env.sh + entrypoint: sleep infinity # just wait diff --git a/tests/docker/mock-test-dt.yaml b/tests/docker/mock-test-dt.yaml new file mode 100644 index 00000000000..22ef3e82337 --- /dev/null +++ b/tests/docker/mock-test-dt.yaml @@ -0,0 +1,21 @@ +version: '2.3' + +services: + # tics0 container is for tests under mutable-test && delta-merge-test directory + tics0: + image: hub.pingcap.net/tiflash/tics:${TAG:-master} + ports: + - "3930:3930" + - "9000:9000" + - "8123:8123" + - "9009:9009" + volumes: + - ./config/tics_dt.toml:/config.toml:ro + - ./config/users.toml:/users.toml:ro + - ./data/tics_dt:/tmp/tiflash/data + - ./log/tics_dt:/tmp/tiflash/log + - ..:/tests + - ./_env.sh:/tests/_env.sh + command: + - --config-file + - /config.toml diff --git a/tests/docker/mock-test-tmt.yaml b/tests/docker/mock-test-tmt.yaml new file mode 100644 index 00000000000..cf443c9887e --- /dev/null +++ b/tests/docker/mock-test-tmt.yaml @@ -0,0 +1,21 @@ +version: '2.3' + +services: + # tics0 container is for tests under mutable-test && delta-merge-test directory + tics0: + image: hub.pingcap.net/tiflash/tics:${TAG:-master} + ports: + - "3930:3930" + - "9000:9000" + - "8123:8123" + - "9009:9009" + volumes: + - ./config/tics_tmt.toml:/config.toml:ro + - ./config/users.toml:/users.toml:ro + - ./data/tics_tmt:/tmp/tiflash/data + - ./log/tics_tmt:/tmp/tiflash/log + - ..:/tests + - ./_env.sh:/tests/_env.sh + command: + - --config-file + - /config.toml diff --git a/tests/docker/run.sh b/tests/docker/run.sh index 9f801170c7f..a1be3fea672 100644 --- a/tests/docker/run.sh +++ b/tests/docker/run.sh @@ -2,24 +2,51 @@ set -xe -docker-compose down +# Stop all docker instances if exist. +# tiflash-dt && tiflash-tmt share the same name "tiflash0", we just need one here +docker-compose -f gtest.yaml -f cluster.yaml -f tiflash-dt.yaml -f mock-test-dt.yaml down rm -rf ./data ./log - # run gtest cases. (only tics-gtest up) -docker-compose up -d --scale tics0=0 --scale tiflash0=0 --scale tikv0=0 --scale tidb0=0 --scale pd0=0 -docker-compose exec -T tics-gtest bash -c 'cd /tests && ./run-gtest.sh' -docker-compose down +docker-compose -f gtest.yaml up -d +docker-compose -f gtest.yaml exec -T tics-gtest bash -c 'cd /tests && ./run-gtest.sh' +docker-compose -f gtest.yaml down + + +rm -rf ./data ./log +# run fullstack-tests (for engine DeltaTree) +docker-compose -f cluster.yaml -f tiflash-dt.yaml up -d +sleep 60 +docker-compose -f cluster.yaml -f tiflash-dt.yaml up -d --build +sleep 10 +docker-compose -f cluster.yaml -f tiflash-dt.yaml exec -T tiflash0 bash -c 'cd /tests ; ./run-test.sh fullstack-test true' +docker-compose -f cluster.yaml -f tiflash-dt.yaml down + -# run fullstack-tests -docker-compose up -d --scale tics0=0 --scale tics-gtest=0 --scale tiflash0=0 +# We need to separate mock-test for dt and tmt, since this behavior +# is different in some tests +# * "tmt" engine ONLY support disable_bg_flush = false. +# * "dt" engine ONLY support disable_bg_flush = true. +rm -rf ./data ./log +# (only tics0 up) (for engine DetlaTree) +docker-compose -f mock-test-dt.yaml up -d +docker-compose -f mock-test-dt.yaml exec -T tics0 bash -c 'cd /tests ; ./run-test.sh delta-merge-test' +docker-compose -f mock-test-dt.yaml down + + + +rm -rf ./data ./log +# run fullstack-tests (for engine TxnMergeTree) +docker-compose -f cluster.yaml -f tiflash-tmt.yaml up -d sleep 60 -docker-compose up -d --scale tics0=0 --scale tics-gtest=0 --build +docker-compose -f cluster.yaml -f tiflash-tmt.yaml up -d --build sleep 10 -docker-compose exec -T tiflash0 bash -c 'cd /tests ; ./run-test.sh fullstack-test true' -docker-compose down +docker-compose -f cluster.yaml -f tiflash-tmt.yaml exec -T tiflash0 bash -c 'cd /tests ; ./run-test.sh fullstack-test true' +docker-compose -f cluster.yaml -f tiflash-tmt.yaml down + -# (only tics0 up) -docker-compose up -d --scale tics-gtest=0 --scale tiflash0=0 --scale tikv0=0 --scale tidb0=0 --scale pd0=0 -docker-compose exec -T tics0 bash -c 'cd /tests ; ./run-test.sh delta-merge-test && ./run-test.sh mutable-test' -docker-compose down +rm -rf ./data ./log +# (only tics0 up) (for engine TxnMergeTree) +docker-compose -f mock-test-tmt.yaml up -d +docker-compose -f mock-test-tmt.yaml exec -T tics0 bash -c 'cd /tests ; ./run-test.sh mutable-test' +docker-compose -f mock-test-tmt.yaml down diff --git a/tests/docker/tiflash-dt.yaml b/tests/docker/tiflash-dt.yaml new file mode 100644 index 00000000000..fbaf3c787ff --- /dev/null +++ b/tests/docker/tiflash-dt.yaml @@ -0,0 +1,28 @@ +version: '2.3' + +services: + # for tests under fullstack-test directory + # (engine DeltaTree) + tiflash0: + image: hub.pingcap.net/tiflash/tics:${TAG:-master} + ports: + - "3930:3930" + - "9000:9000" + - "8123:8123" + - "9009:9009" + - "20170:20170" + volumes: + - ./config/tiflash_dt.toml:/config.toml:ro + - ./config/tiflash-users.toml:/users.toml:ro + - ./data/tiflash_dt:/tmp/tiflash/data + - ./log/tiflash_dt:/tmp/tiflash/log + - ..:/tests + - ./_env.sh:/tests/_env.sh + - ./log/tiflash-cluster-manager:/tmp/tiflash/data/tmp + - ./config/tikv-learner.toml:/tikv.toml:ro + - ./data/tikv-learner_dt:/data + - ./log/tikv-learner_dt:/log + command: + - --config-file + - /config.toml + restart: on-failure diff --git a/tests/docker/tiflash-tmt.yaml b/tests/docker/tiflash-tmt.yaml new file mode 100644 index 00000000000..143616e4534 --- /dev/null +++ b/tests/docker/tiflash-tmt.yaml @@ -0,0 +1,28 @@ +version: '2.3' + +services: + # for tests under fullstack-test directory + # (engine TxnMergeTree) + tiflash0: + image: hub.pingcap.net/tiflash/tics:${TAG:-master} + ports: + - "3930:3930" + - "9000:9000" + - "8123:8123" + - "9009:9009" + - "20170:20170" + volumes: + - ./config/tiflash_tmt.toml:/config.toml:ro + - ./config/tiflash-users.toml:/users.toml:ro + - ./data/tiflash_tmt:/tmp/tiflash/data + - ./log/tiflash_tmt:/tmp/tiflash/log + - ..:/tests + - ./_env.sh:/tests/_env.sh + - ./log/tiflash-cluster-manager:/tmp/tiflash/data/tmp + - ./config/tikv-learner.toml:/tikv.toml:ro + - ./data/tikv-learner_tmt:/data + - ./log/tikv-learner_tmt:/log + command: + - --config-file + - /config.toml + restart: on-failure diff --git a/tests/fullstack-test/ddl/alter_datetime_default_value.test b/tests/fullstack-test/ddl/alter_datetime_default_value.test new file mode 100644 index 00000000000..229aae6081b --- /dev/null +++ b/tests/fullstack-test/ddl/alter_datetime_default_value.test @@ -0,0 +1,90 @@ +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + +mysql> insert into test.t values (1); +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++---+ +| a | ++---+ +| 1 | ++---+ + +mysql> alter table test.t add date_0 DATE NULL DEFAULT '1000-01-01' +mysql> alter table test.t add date_1 DATE NULL DEFAULT '9999-12-31' +mysql> alter table test.t add date_a DATE NOT NULL + +mysql> alter table test.t add time_0 TIME NULL DEFAULT '59' +mysql> alter table test.t add time_1 TIME(6) NULL DEFAULT '-838:59:59.000000' +mysql> alter table test.t add time_2 TIME(6) NULL DEFAULT '838:59:59.000000' +mysql> alter table test.t add time_3 TIME(6) NULL DEFAULT '0' +mysql> alter table test.t add time_a TIME NOT NULL +mysql> alter table test.t add time_b TIME(6) NOT NULL + +mysql> alter table test.t add datetime_0 DATETIME(6) NULL DEFAULT '1000-01-01 00:00:00.000000' +mysql> alter table test.t add datetime_1 DATETIME(6) NULL DEFAULT '9999-12-31 23:59:59.000000' +mysql> alter table test.t add datetime_a DATETIME NOT NULL +mysql> alter table test.t add datetime_b DATETIME(6) NOT NULL + +SLEEP 15 + +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+------------+------------+------------+----------+-------------------+------------------+-----------------+----------+-----------------+----------------------------+----------------------------+---------------------+----------------------------+ +| a | date_0 | date_1 | date_a | time_0 | time_1 | time_2 | time_3 | time_a | time_b | datetime_0 | datetime_1 | datetime_a | datetime_b | ++------+------------+------------+------------+----------+-------------------+------------------+-----------------+----------+-----------------+----------------------------+----------------------------+---------------------+----------------------------+ +| 1 | 1000-01-01 | 9999-12-31 | 0000-00-00 | 00:00:59 | -838:59:59.000000 | 838:59:59.000000 | 00:00:00.000000 | 00:00:00 | 00:00:00.000000 | 1000-01-01 00:00:00.000000 | 9999-12-31 23:59:59.000000 | 0000-00-00 00:00:00 | 0000-00-00 00:00:00.000000 | ++------+------------+------------+------------+----------+-------------------+------------------+-----------------+----------+-----------------+----------------------------+----------------------------+---------------------+----------------------------+ + + +### tests for timestamp +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + +mysql> insert into test.t values (1); + +# The min, max, 0 is related to time zone of the machine +#mysql> alter table test.t add timestamp_0 TIMESTAMP(6) NULL DEFAULT '1970-01-01 00:00:01.000000' +#mysql> alter table test.t add timestamp_1 TIMESTAMP(6) NULL DEFAULT '2038-01-19 03:14:07.999999' +#mysql> alter table test.t add timestamp_2 TIMESTAMP(6) NULL DEFAULT '0000-00-00 00:00:00.000000' +#mysql> alter table test.t add timestamp_0 TIMESTAMP(6) NULL DEFAULT '2000-01-20 03:14:07.999999' +mysql> alter table test.t add timestamp_a TIMESTAMP NOT NULL +mysql> alter table test.t add timestamp_b TIMESTAMP(6) NOT NULL + +SLEEP 15 + +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+---------------------+----------------------------+ +| a | timestamp_a | timestamp_b | ++------+---------------------+----------------------------+ +| 1 | 0000-00-00 00:00:00 | 0000-00-00 00:00:00.000000 | ++------+---------------------+----------------------------+ + +mysql> drop table if exists test.t + +### tests for year +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + +mysql> insert into test.t values (1); +mysql> alter table test.t add year_0 YEAR NULL DEFAULT '1901' +mysql> alter table test.t add year_1 YEAR NULL DEFAULT '2155' +mysql> alter table test.t add year_2 YEAR NULL DEFAULT '0000' +mysql> alter table test.t add year_a YEAR NOT NULL + +#TODO: fix zero value for year +#SLEEP 15 +#mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; +#+------+--------+--------+--------+--------+ +#| a | year_0 | year_1 | year_2 | year_a | +#+------+--------+--------+--------+--------+ +#| 1 | 1901 | 2155 | 0000 | 2000 | +#+------+--------+--------+--------+--------+ + diff --git a/tests/fullstack-test/ddl/alter_decimal_default_value.test b/tests/fullstack-test/ddl/alter_decimal_default_value.test new file mode 100644 index 00000000000..f83669c2328 --- /dev/null +++ b/tests/fullstack-test/ddl/alter_decimal_default_value.test @@ -0,0 +1,40 @@ +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + +mysql> insert into test.t values (1); + +# Decimal32 precision:[1,9] +mysql> alter table test.t ADD COLUMN dec32_0 DECIMAL(9,0) NULL DEFAULT '999999999' +mysql> alter table test.t ADD COLUMN dec32_1 DECIMAL(9,9) NULL DEFAULT '.999999999' +mysql> alter table test.t ADD COLUMN dec32_3 DECIMAL(9,0) NULL DEFAULT '9' +mysql> alter table test.t ADD COLUMN dec32_4 DECIMAL(9,9) NULL DEFAULT '.9' +# Decimal64 precision:(9,18] +mysql> alter table test.t ADD COLUMN dec64_0 DECIMAL(18,0) NULL DEFAULT '999999999999999999' +mysql> alter table test.t ADD COLUMN dec64_1 DECIMAL(18,18) NULL DEFAULT '.999999999999999999' +mysql> alter table test.t ADD COLUMN dec64_3 DECIMAL(18,0) NULL DEFAULT '9' +mysql> alter table test.t ADD COLUMN dec64_4 DECIMAL(18,18) NULL DEFAULT '.9' +# Decimal128 precision:(18,38] +mysql> alter table test.t ADD COLUMN dec128_0 DECIMAL(38,0) NULL DEFAULT '99999999999999999999999999999999999999' +mysql> alter table test.t ADD COLUMN dec128_1 DECIMAL(38,30) NULL DEFAULT '99999999.999999999999999999999999999999' +mysql> alter table test.t ADD COLUMN dec128_3 DECIMAL(38,0) NULL DEFAULT '9' +mysql> alter table test.t ADD COLUMN dec128_4 DECIMAL(38,30) NULL DEFAULT '.9' +# Decimal256 precision:(38,65] +mysql> alter table test.t ADD COLUMN dec256_0 DECIMAL(65,0) NULL DEFAULT '99999999999999999999999999999999999999999999999999999999999999999' +mysql> alter table test.t ADD COLUMN dec256_1 DECIMAL(65,30) NULL DEFAULT '99999999999999999999999999999999999.999999999999999999999999999999' +mysql> alter table test.t ADD COLUMN dec256_3 DECIMAL(65,0) NULL DEFAULT '9' +mysql> alter table test.t ADD COLUMN dec256_4 DECIMAL(65,30) NULL DEFAULT '.9' + +SLEEP 15 + +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++---+-----------+-------------+---------+-------------+--------------------+----------------------+---------+----------------------+----------------------------------------+-----------------------------------------+----------+----------------------------------+-------------------------------------------------------------------+--------------------------------------------------------------------+----------+----------------------------------+ +| a | dec32_0 | dec32_1 | dec32_3 | dec32_4 | dec64_0 | dec64_1 | dec64_3 | dec64_4 | dec128_0 | dec128_1 | dec128_3 | dec128_4 | dec256_0 | dec256_1 | dec256_3 | dec256_4 | ++---+-----------+-------------+---------+-------------+--------------------+----------------------+---------+----------------------+----------------------------------------+-----------------------------------------+----------+----------------------------------+-------------------------------------------------------------------+--------------------------------------------------------------------+----------+----------------------------------+ +| 1 | 999999999 | 0.999999999 | 9 | 0.900000000 | 999999999999999999 | 0.999999999999999999 | 9 | 0.900000000000000000 | 99999999999999999999999999999999999999 | 99999999.999999999999999999999999999999 | 9 | 0.900000000000000000000000000000 | 99999999999999999999999999999999999999999999999999999999999999999 | 99999999999999999999999999999999999.999999999999999999999999999999 | 9 | 0.900000000000000000000000000000 | ++---+-----------+-------------+---------+-------------+--------------------+----------------------+---------+----------------------+----------------------------------------+-----------------------------------------+----------+----------------------------------+-------------------------------------------------------------------+--------------------------------------------------------------------+----------+----------------------------------+ + + +mysql> drop table if exists test.t diff --git a/tests/fullstack-test/ddl/alter_default_value.test b/tests/fullstack-test/ddl/alter_default_value.test new file mode 100644 index 00000000000..74c97122f1f --- /dev/null +++ b/tests/fullstack-test/ddl/alter_default_value.test @@ -0,0 +1,21 @@ +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + +mysql> insert into test.t values (1); +mysql> alter table test.t add s varchar(999) default 'sss' +mysql> alter table test.t add e enum('unknown', 'male','female') default 'unknown' + +SLEEP 15 + +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++---+-----+---------+ +| a | s | e | ++---+-----+---------+ +| 1 | sss | unknown | ++---+-----+---------+ + + +mysql> drop table if exists test.t diff --git a/tests/fullstack-test/ddl/datetime_default_value.test b/tests/fullstack-test/ddl/datetime_default_value.test new file mode 100644 index 00000000000..913ccff7cc7 --- /dev/null +++ b/tests/fullstack-test/ddl/datetime_default_value.test @@ -0,0 +1,36 @@ +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t (a) values (1); +mysql> insert into test.t (a) values (1); + +SLEEP 15 + + +>> DBGInvoke __try_flush() + +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ * from test.t; ++------+ +| a | ++------+ +| 1 | +| 1 | ++------+ + +mysql> alter table test.t add column b datetime default '8124-05-31 23:47:33'; +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(t) */ * from test.t; ++------+---------------------+ +| a | b | ++------+---------------------+ +| 1 | 8124-05-31 23:47:33 | +| 1 | 8124-05-31 23:47:33 | ++------+---------------------+ + +mysql> alter table test.t add column c datetime default 19910905; +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(t) */ * from test.t; ++------+---------------------+---------------------+ +| a | b | c | ++------+---------------------+---------------------+ +| 1 | 8124-05-31 23:47:33 | 1991-09-05 00:00:00 | +| 1 | 8124-05-31 23:47:33 | 1991-09-05 00:00:00 | ++------+---------------------+---------------------+ diff --git a/tests/fullstack-test/expr/single_ifnull_in_predicate.test b/tests/fullstack-test/expr/single_ifnull_in_predicate.test index a3b69ace12e..ca907730ba0 100644 --- a/tests/fullstack-test/expr/single_ifnull_in_predicate.test +++ b/tests/fullstack-test/expr/single_ifnull_in_predicate.test @@ -14,3 +14,5 @@ mysql> select /*+ read_from_storage(tiflash[t]) */ c2 from test.t where ifnull(c | 123 | | 234 | +------+ + +mysql> drop table if exists test.t diff --git a/tests/fullstack-test/fault-inject/alter-table.test b/tests/fullstack-test/fault-inject/alter-table.test index d9f2fa7d1dd..c4ee5bce4c2 100644 --- a/tests/fullstack-test/fault-inject/alter-table.test +++ b/tests/fullstack-test/fault-inject/alter-table.test @@ -10,15 +10,30 @@ mysql> insert into test.t values (1, 2) SLEEP 15 +# ensure table is sync to tiflash +mysql> select table_schema,table_name,replica_count,location_labels,available from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------------+-----------+ +| table_schema | table_name | replica_count | location_labels | available | ++--------------+------------+---------------+-----------------+-----------+ +| test | t | 1 | rack,host,abc | 1 | ++--------------+------------+---------------+-----------------+-----------+ +>> select database,name from system.tables where database='test' and name='t'; +┌─database─┬─name─┐ +│ test │ t │ +└──────────┴──────┘ >> DBGInvoke __try_flush() >> DBGInvoke __init_fail_point() + +# Process crash when rename column data is done but not for meta. +# Note that this only affect TxnMergeTree. DeltaTree don't care this since we store +# column data according to its column-id in TiDB. >> DBGInvoke __enable_fail_point(exception_between_alter_data_and_meta) mysql> alter table test.t change a c int SLEEP 15 -mysql> select /*+ read_from_storage(tiflash[t]) */ * from test.t; +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; +---+---+ | c | b | +---+---+ diff --git a/tests/fullstack-test/fault-inject/drop-table.test b/tests/fullstack-test/fault-inject/drop-table.test index 920c081390a..52a66ba3ba2 100644 --- a/tests/fullstack-test/fault-inject/drop-table.test +++ b/tests/fullstack-test/fault-inject/drop-table.test @@ -11,18 +11,36 @@ mysql> insert into test.t values (1, 2) SLEEP 15 +# ensure table is sync to tiflash +mysql> select table_schema,table_name,replica_count,location_labels,available from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------------+-----------+ +| table_schema | table_name | replica_count | location_labels | available | ++--------------+------------+---------------+-----------------+-----------+ +| test | t | 1 | rack,host,abc | 1 | ++--------------+------------+---------------+-----------------+-----------+ +>> select database,name from system.tables where database='test' and name='t'; +┌─database─┬─name─┐ +│ test │ t │ +└──────────┴──────┘ >> DBGInvoke __try_flush() >> DBGInvoke __init_fail_point() + +# Process crash when drop table metadata is done but not for its data dir. >> DBGInvoke __enable_fail_point(exception_between_drop_meta_and_data) mysql> truncate table test.t ; -SLEEP 15 +# sleep to wait schema synced to tiflash +SLEEP 20 + +# After restart, test.t is truncated, it is empty +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; + + mysql> insert into test.t values (1, 1) mysql> insert into test.t values (1, 2) -mysql> select /*+ read_from_storage(tiflash[t]) */ * from test.t; - +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; +---+---+ | a | b | +---+---+ @@ -33,11 +51,14 @@ mysql> select /*+ read_from_storage(tiflash[t]) */ * from test.t; >> DBGInvoke __enable_fail_point(exception_drop_table_during_remove_meta) mysql> truncate table test.t ; -SLEEP 15 +SLEEP 20 + +# After restart, test.t is truncated, it is empty +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; mysql> insert into test.t values (1, 1) mysql> insert into test.t values (1, 2) -mysql> select /*+ read_from_storage(tiflash[t]) */ * from test.t; +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; +---+---+ | a | b | diff --git a/tests/fullstack-test/fault-inject/rename-table.test b/tests/fullstack-test/fault-inject/rename-table.test new file mode 100644 index 00000000000..9b66c5b76d8 --- /dev/null +++ b/tests/fullstack-test/fault-inject/rename-table.test @@ -0,0 +1,46 @@ + +#TODO: atomic rename table +#RETURN + +mysql> drop table if exists test.t; +mysql> create table test.t(a int not null, b int not null); +mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc' + +SLEEP 15 + + +mysql> insert into test.t values (1, 1),(1,2) + +SLEEP 15 + +# ensure table is sync to tiflash +mysql> select table_schema,table_name,replica_count,location_labels,available from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------------+-----------+ +| table_schema | table_name | replica_count | location_labels | available | ++--------------+------------+---------------+-----------------+-----------+ +| test | t | 1 | rack,host,abc | 1 | ++--------------+------------+---------------+-----------------+-----------+ +>> select database,name from system.tables where database='test' and name='t'; +┌─database─┬─name─┐ +│ test │ t │ +└──────────┴──────┘ +>> DBGInvoke __try_flush() +>> DBGInvoke __init_fail_point() + +# Process crash when rename table data is done but not for its metadata file. +>> DBGInvoke __enable_fail_point(exception_between_rename_table_metadata_and_data) + +mysql> drop table if exists test.t_2; +mysql> rename table test.t to test.t_2 ; + +SLEEP 15 + +# After restart, test.t is rename to test.t_2, we can still read data from tiflash +>> select database,name from system.tables where database='test' and name='t'; +>> select database,name from system.tables where database='test' and name='t_2'; +┌─database─┬─name─┐ +│ test │ t_2 │ +└──────────┴──────┘ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_2; + +mysql> drop table if exists test.t diff --git a/tests/fullstack-test/sample.test b/tests/fullstack-test/sample.test index 2ed33986731..8833c44d108 100644 --- a/tests/fullstack-test/sample.test +++ b/tests/fullstack-test/sample.test @@ -1,5 +1,6 @@ mysql> drop table if exists test.t -mysql> create table test.t(s varchar(256), i int) + +mysql> create table if not exists test.t(s varchar(256), i int) mysql> alter table test.t set tiflash replica 1 location labels 'rack', 'host', 'abc'; mysql> delete from test.t mysql> insert into test.t values('Hello world', 666) @@ -12,3 +13,5 @@ SLEEP 15 └─────────────┴─────┘ mysql> delete from test.t + +mysql> drop table if exists test.t diff --git a/tests/generate-fullstack-test.py b/tests/generate-fullstack-test.py index d60ed3cde79..fcfb254ab7d 100644 --- a/tests/generate-fullstack-test.py +++ b/tests/generate-fullstack-test.py @@ -17,7 +17,7 @@ update_stmt = Template("mysql> update $database.$table set $exprs $condition\n") delete_stmt = Template("mysql> delete from $database.$table $condition\n") select_stmt = Template(">> select $columns from $database.$table\n") -tidb_select_stmt = Template("mysql> select /*+ read_from_storage(tiflash[ttt]) */ $columns from $database.$table ttt\n") +tidb_select_stmt = Template("mysql> set SESSION tidb_isolation_read_engines = 'tiflash' ;select $columns from $database.$table ttt\n") sleep_string = "\nSLEEP 15\n\n" @@ -314,10 +314,10 @@ def generate_cases_inner(database, table, column_names, types, sample_data, "condition": condition})) case_data = new_case_data if op == SELECT: - file.write(select_stmt.substitute({"columns": ", ".join(column_names), - "database": database, - "table": table})) - file.write(generate_result(column_names, case_data) + "\n\n") + # file.write(select_stmt.substitute({"columns": ", ".join(column_names), + # "database": database, + # "table": table})) + # file.write(generate_result(column_names, case_data) + "\n\n") file.write(tidb_select_stmt.substitute({"columns": ", ".join(column_names), "database": database, "table": table})) diff --git a/tests/mutable-test/txn_mock/ingest_sst.test b/tests/mutable-test/txn_mock/ingest_sst.test new file mode 100644 index 00000000000..4b5705ea5af --- /dev/null +++ b/tests/mutable-test/txn_mock/ingest_sst.test @@ -0,0 +1,26 @@ +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') +=> DBGInvoke __region_snapshot(4, 0, 1000, default, test) +=> DBGInvoke __region_mock_ingest_sst(default, test, 4, 3, 6) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ -3 │ 3 │ +│ -4 │ 4 │ +│ -5 │ 5 │ +└───────┴─────────────┘ +=> DBGInvoke __region_mock_ingest_sst(default, test, 4, 7, 9) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ -3 │ 3 │ +│ -4 │ 4 │ +│ -5 │ 5 │ +│ -7 │ 7 │ +└───────┴─────────────┘ +┌─col_1─┬─_tidb_rowid─┐ +│ -8 │ 8 │ +└───────┴─────────────┘ +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/run-test.py b/tests/run-test.py index 9f8029fd56d..8cdc1696a87 100644 --- a/tests/run-test.py +++ b/tests/run-test.py @@ -160,6 +160,7 @@ def on_line(self, line): if line.startswith(SLEEP_PREFIX): time.sleep(float(line[len(SLEEP_PREFIX):])) elif line.startswith(CMD_PREFIX_TIDB): + if verbose: print 'running', line if self.outputs != None and ((not self.is_mysql and not matched(self.outputs, self.matches, self.fuzz)) or (self.is_mysql and not MySQLCompare.matched(self.outputs, self.matches))): return False self.is_mysql = True diff --git a/tests/run-test.sh b/tests/run-test.sh index d09f9b009f2..3e101b0a72e 100755 --- a/tests/run-test.sh +++ b/tests/run-test.sh @@ -13,7 +13,7 @@ function run_file() local ext=${path##*.} if [ "$ext" == "test" ]; then - python2 run-test.py "$dbc" "$path" "$fuzz" "$mysql_client" "$verbose" + python run-test.py "$dbc" "$path" "$fuzz" "$mysql_client" "$verbose" else if [ "$ext" == "visual" ]; then python run-test-gen-from-visual.py "$path" "$skip_raw_test" "$verbose" From 34a4a50b8a6a8e170ca5cfbc14423822c7e496c5 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 27 Mar 2020 12:17:36 +0800 Subject: [PATCH 41/54] fix type mismatch bug in broadcast join --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 117 +++++++++++------- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 10 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 63 +++++++++- .../Coprocessor/DAGQueryBlockInterpreter.h | 2 +- 4 files changed, 138 insertions(+), 54 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 20c79646e74..f27ff3fdcad 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -398,8 +398,7 @@ String DAGExpressionAnalyzer::appendTimeZoneCast( // column to the columns with session-level timezone info, the original ts columns with UTC timezone // are still kept, and the InterpreterDAG will choose the correct column based on encode type bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( - ExpressionActionsChain & chain, std::vector is_ts_column, - const tipb::DAGRequest & rqst, bool keep_UTC_column) + ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst, bool keep_UTC_column) { if (!hasMeaningfulTZInfo(rqst)) return false; @@ -428,55 +427,80 @@ bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( return ret; } -void DAGExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, SubqueryForSet & join_query, const NamesAndTypesList & columns_added_by_join) +void DAGExpressionAnalyzer::appendJoin( + ExpressionActionsChain & chain, SubqueryForSet & join_query, const NamesAndTypesList & columns_added_by_join) { initChain(chain, getCurrentInputColumns()); ExpressionActionsPtr actions = chain.getLastActions(); actions->add(ExpressionAction::ordinaryJoin(join_query.join, columns_added_by_join)); } /// return true if some actions is needed -bool DAGExpressionAnalyzer::appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left) +bool DAGExpressionAnalyzer::appendJoinKey( + ExpressionActionsChain & chain, const tipb::Join & join, const DataTypes & key_types, Names & key_names, bool tiflash_left) { bool ret = false; initChain(chain, getCurrentInputColumns()); ExpressionActionsPtr actions = chain.getLastActions(); - NamesWithAliases right_key_dummy_production; - if ((tiflash_left && join.inner_idx() == 1) || (!tiflash_left && join.inner_idx() == 0)) + std::vector type_expr_names; + const auto & keys = ((tiflash_left && join.inner_idx() == 1) || (!tiflash_left && join.inner_idx() == 0)) ? join.left_join_keys() + : join.right_join_keys(); + + for (int i = 0; i < keys.size(); i++) { - for(const auto & key : join.left_join_keys()) + const auto & key = keys.at(i); + bool has_actions = key.tp() != tipb::ExprType::ColumnRef; + + String key_name = getActions(key, actions); + DataTypePtr current_type = actions->getSampleBlock().getByName(key_name).type; + if (!removeNullable(current_type)->equals(*removeNullable(key_types[i]))) { - if (key.tp() != tipb::ExprType::ColumnRef) - ret = true; - if (!tiflash_left && key.tp() == tipb::ExprType::ColumnRef) - { - // for right side table, add a project - String org_key_name = getActions(key, actions); - String updated_key_name = "_r_k_" + org_key_name; - actions->add(ExpressionAction::copyColumn(org_key_name, updated_key_name)); - key_names.push_back(updated_key_name); - ret = true; - } - else - key_names.push_back(getActions(key, actions)); + /// need to convert to key type + tipb::Expr type_expr; + constructStringLiteralTiExpr(type_expr, key_types[i]->getName()); + auto type_expr_name = getActions(type_expr, actions); + key_name = applyFunction("CAST", {key_name, type_expr_name}, actions); + type_expr_names.emplace_back(type_expr_name); + has_actions = true; } - } - else - { - for(const auto & key : join.right_join_keys()) + if (!tiflash_left && !has_actions) { - if (key.tp() != tipb::ExprType::ColumnRef) - ret = true; - if (!tiflash_left && key.tp() == tipb::ExprType::ColumnRef) - { - // for right side table, add a project - String org_key_name = getActions(key, actions); - String updated_key_name = "_r_k_" + org_key_name; - actions->add(ExpressionAction::copyColumn(org_key_name, updated_key_name)); - key_names.push_back(updated_key_name); - ret = true; - } - else - key_names.push_back(getActions(key, actions)); + // for right side table, add a new column + String updated_key_name = "_r_k_" + key_name; + actions->add(ExpressionAction::copyColumn(key_name, updated_key_name)); + key_name = updated_key_name; + has_actions = true; + } + key_names.push_back(key_name); + ret |= has_actions; + } + /// remove useless columns to avoid duplicate columns + /// as when compiling the key expression, the origin + /// streams may be added some columns that have the + /// same name on left streams and right streams, for + /// example, if the join condition is something like: + /// id + 1 = id + 1, + /// the left streams and the right streams will have the + /// same constant column for `1` + /// Note that the origin left streams and right streams + /// will never have duplicated columns because in + /// DAGQueryBlockInterpreter we add qb_column_prefix in + /// final project step, so if the join condition is not + /// literal expression, the key names should never be + /// duplicated. In the above example, the final key names should be + /// something like `add(__qb_2_id, 1)` and `add(__qb_3_id, 1)` + if (actions->getSampleBlock().getNames().size() != getCurrentInputColumns().size() + key_names.size()) + { + std::unordered_set needed_columns; + for (const auto & c : getCurrentInputColumns()) + needed_columns.insert(c.name); + for (const auto & s : key_names) + needed_columns.insert(s); + + const auto & names = actions->getSampleBlock().getNames(); + for (const auto & name : names) + { + if (needed_columns.find(name) == needed_columns.end()) + actions->add(ExpressionAction::removeColumn(name)); } } return ret; @@ -575,6 +599,17 @@ String DAGExpressionAnalyzer::alignReturnType( return updated_name; } +String DAGExpressionAnalyzer::appendCast(const DataTypePtr & target_type, ExpressionActionsPtr & actions, const String & expr_name) +{ + // need to add cast function + // first construct the second argument + tipb::Expr type_expr; + constructStringLiteralTiExpr(type_expr, target_type->getName()); + auto type_expr_name = getActions(type_expr, actions); + String cast_expr_name = applyFunction("CAST", {expr_name, type_expr_name}, actions); + return cast_expr_name; +} + String DAGExpressionAnalyzer::appendCastIfNeeded( const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool explicit_cast) { @@ -595,13 +630,7 @@ String DAGExpressionAnalyzer::appendCastIfNeeded( { implicit_cast_count += !explicit_cast; - // need to add cast function - // first construct the second argument - tipb::Expr type_expr; - constructStringLiteralTiExpr(type_expr, expected_type->getName()); - auto type_expr_name = getActions(type_expr, actions); - String cast_expr_name = applyFunction("CAST", {expr_name, type_expr_name}, actions); - return cast_expr_name; + return appendCast(expected_type, actions, expr_name); } else { diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index f0a6e543986..c5cf8867b51 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -10,8 +10,8 @@ #include #include #include -#include #include +#include namespace DB { @@ -46,6 +46,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable void appendAggSelect( ExpressionActionsChain & chain, const tipb::Aggregation & agg, const tipb::DAGRequest & rqst, bool keep_session_timezone_info); String appendCastIfNeeded(const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool explicit_cast); + String appendCast(const DataTypePtr & target_type, ExpressionActionsPtr & actions, const String & expr_name); String alignReturnType(const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool force_uint8); void initChain(ExpressionActionsChain & chain, const std::vector & columns) const { @@ -68,9 +69,10 @@ class DAGExpressionAnalyzer : private boost::noncopyable void makeExplicitSetForIndex(const tipb::Expr & expr, const ManageableStoragePtr & storage); String applyFunction(const String & func_name, const Names & arg_names, ExpressionActionsPtr & actions); Int32 getImplicitCastCount() { return implicit_cast_count; }; - bool appendTimeZoneCastsAfterTS(ExpressionActionsChain & chain, std::vector is_ts_column, - const tipb::DAGRequest & rqst, bool keep_UTC_column); - bool appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left); + bool appendTimeZoneCastsAfterTS( + ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst, bool keep_UTC_column); + bool appendJoinKey( + ExpressionActionsChain & chain, const tipb::Join & join, const DataTypes & key_types, Names & key_names, bool tiflash_left); String appendTimeZoneCast(const String & tz_col, const String & ts_col, const String & func_name, ExpressionActionsPtr & actions); DAGPreparedSets & getPreparedSets() { return prepared_sets; } String convertToUInt8(ExpressionActionsPtr & actions, const String & column_name); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 88166553f0d..5003053f69f 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -45,6 +46,7 @@ extern const int TOO_MANY_COLUMNS; extern const int SCHEMA_VERSION_ERROR; extern const int UNKNOWN_EXCEPTION; extern const int COP_BAD_DAG_REQUEST; +extern const int NO_COMMON_TYPE; } // namespace ErrorCodes DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, @@ -497,18 +499,65 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & addTimeZoneCastAfterTS(is_ts_column, pipeline); } -void DAGQueryBlockInterpreter::prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left) +void DAGQueryBlockInterpreter::prepareJoinKeys( + const tipb::Join & join, const DataTypes & key_types, Pipeline & pipeline, Names & key_names, bool tiflash_left) { std::vector source_columns; for (auto const & p : pipeline.firstStream()->getHeader().getNamesAndTypesList()) source_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); ExpressionActionsChain chain; - if (dag_analyzer.appendJoinKey(chain, join, key_names, tiflash_left)) + if (dag_analyzer.appendJoinKey(chain, join, key_types, key_names, tiflash_left)) { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); } } + +/// ClickHouse require join key to be exactly the same type +/// TiDB only require the join key to be the same category +/// for example decimal(10,2) join decimal(20,0) is allowed in +/// TiDB and will throw exception in ClickHouse +void getJoinKeyTypes(const tipb::Join & join, DataTypes & key_types) +{ + for (int i = 0; i < join.left_join_keys().size(); i++) + { + if (!exprHasValidFieldType(join.left_join_keys(i)) || !exprHasValidFieldType(join.right_join_keys(i))) + throw Exception("Join key without field type", ErrorCodes::COP_BAD_DAG_REQUEST); + DataTypes types; + types.emplace_back(getDataTypeByFieldType(join.left_join_keys(i).field_type())); + types.emplace_back(getDataTypeByFieldType(join.right_join_keys(i).field_type())); + try + { + DataTypePtr common_type = getLeastSupertype(types); + key_types.emplace_back(common_type); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::NO_COMMON_TYPE) + { + DataTypePtr left_type = removeNullable(types[0]); + DataTypePtr right_type = removeNullable(types[1]); + if ((left_type->getTypeId() == TypeIndex::UInt64 && right_type->getTypeId() == TypeIndex::Int64) + || (left_type->getTypeId() == TypeIndex::Int64 && right_type->getTypeId() == TypeIndex::UInt64)) + { + /// special case for uint64 and int64 + /// inorder to not throw exception, use Decimal(20, 0) as the common type + DataTypePtr common_type = std::make_shared>(20, 0); + if (types[0]->isNullable() || types[1]->isNullable()) + common_type = makeNullable(common_type); + key_types.emplace_back(common_type); + } + else + throw; + } + else + { + throw; + } + } + } +} + void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query) { // build @@ -576,13 +625,16 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); } + DataTypes join_key_types; + getJoinKeyTypes(join, join_key_types); + /// add necessary transformation if the join key is an expression Pipeline left_pipeline; left_pipeline.streams = left_streams; - prepareJoinKeys(join, left_pipeline, left_key_names, true); + prepareJoinKeys(join, join_key_types, left_pipeline, left_key_names, true); Pipeline right_pipeline; right_pipeline.streams = right_streams; - prepareJoinKeys(join, right_pipeline, right_key_names, false); + prepareJoinKeys(join, join_key_types, right_pipeline, right_key_names, false); left_streams = left_pipeline.streams; right_streams = right_pipeline.streams; @@ -1045,7 +1097,8 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; - BlockInputStreamPtr input = std::make_shared(cluster, req, schema, context.getSettingsRef().max_threads, store_type); + BlockInputStreamPtr input + = std::make_shared(cluster, req, schema, context.getSettingsRef().max_threads, store_type); pipeline.streams = {input}; } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 0c962dcb6ee..9b397cbd672 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -90,7 +90,7 @@ class DAGQueryBlockInterpreter void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query); - void prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left); + void prepareJoinKeys(const tipb::Join & join, const DataTypes & key_types, Pipeline & pipeline, Names & key_names, bool tiflash_left); void executeSubqueryInJoin(Pipeline & pipeline, SubqueryForSet & subquery); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); From 87d118bdcd88fe98e9f7f74aab642406f736a496 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 27 Mar 2020 15:28:50 +0800 Subject: [PATCH 42/54] broadcast join support join keys with different data type (#580) * fix type mismatch bug in broadcast join * refine code * refine code --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 112 +++++++++++------- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 10 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 64 +++++++++- .../Coprocessor/DAGQueryBlockInterpreter.h | 2 +- 4 files changed, 134 insertions(+), 54 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 20c79646e74..bc2cf01a703 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -398,8 +398,7 @@ String DAGExpressionAnalyzer::appendTimeZoneCast( // column to the columns with session-level timezone info, the original ts columns with UTC timezone // are still kept, and the InterpreterDAG will choose the correct column based on encode type bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( - ExpressionActionsChain & chain, std::vector is_ts_column, - const tipb::DAGRequest & rqst, bool keep_UTC_column) + ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst, bool keep_UTC_column) { if (!hasMeaningfulTZInfo(rqst)) return false; @@ -428,55 +427,75 @@ bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( return ret; } -void DAGExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, SubqueryForSet & join_query, const NamesAndTypesList & columns_added_by_join) +void DAGExpressionAnalyzer::appendJoin( + ExpressionActionsChain & chain, SubqueryForSet & join_query, const NamesAndTypesList & columns_added_by_join) { initChain(chain, getCurrentInputColumns()); ExpressionActionsPtr actions = chain.getLastActions(); actions->add(ExpressionAction::ordinaryJoin(join_query.join, columns_added_by_join)); } /// return true if some actions is needed -bool DAGExpressionAnalyzer::appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left) +bool DAGExpressionAnalyzer::appendJoinKey( + ExpressionActionsChain & chain, const tipb::Join & join, const DataTypes & key_types, Names & key_names, bool tiflash_left) { bool ret = false; initChain(chain, getCurrentInputColumns()); ExpressionActionsPtr actions = chain.getLastActions(); - NamesWithAliases right_key_dummy_production; - if ((tiflash_left && join.inner_idx() == 1) || (!tiflash_left && join.inner_idx() == 0)) + const auto & keys = ((tiflash_left && join.inner_idx() == 1) || (!tiflash_left && join.inner_idx() == 0)) ? join.left_join_keys() + : join.right_join_keys(); + + for (int i = 0; i < keys.size(); i++) { - for(const auto & key : join.left_join_keys()) + const auto & key = keys.at(i); + bool has_actions = key.tp() != tipb::ExprType::ColumnRef; + + String key_name = getActions(key, actions); + DataTypePtr current_type = actions->getSampleBlock().getByName(key_name).type; + if (!removeNullable(current_type)->equals(*removeNullable(key_types[i]))) { - if (key.tp() != tipb::ExprType::ColumnRef) - ret = true; - if (!tiflash_left && key.tp() == tipb::ExprType::ColumnRef) - { - // for right side table, add a project - String org_key_name = getActions(key, actions); - String updated_key_name = "_r_k_" + org_key_name; - actions->add(ExpressionAction::copyColumn(org_key_name, updated_key_name)); - key_names.push_back(updated_key_name); - ret = true; - } - else - key_names.push_back(getActions(key, actions)); + /// need to convert to key type + key_name = appendCast(key_types[i], actions, key_name); + has_actions = true; } - } - else - { - for(const auto & key : join.right_join_keys()) + if (!tiflash_left && !has_actions) { - if (key.tp() != tipb::ExprType::ColumnRef) - ret = true; - if (!tiflash_left && key.tp() == tipb::ExprType::ColumnRef) - { - // for right side table, add a project - String org_key_name = getActions(key, actions); - String updated_key_name = "_r_k_" + org_key_name; - actions->add(ExpressionAction::copyColumn(org_key_name, updated_key_name)); - key_names.push_back(updated_key_name); - ret = true; - } - else - key_names.push_back(getActions(key, actions)); + // for right side table, add a new column + String updated_key_name = "_r_k_" + key_name; + actions->add(ExpressionAction::copyColumn(key_name, updated_key_name)); + key_name = updated_key_name; + has_actions = true; + } + key_names.push_back(key_name); + ret |= has_actions; + } + /// remove useless columns to avoid duplicate columns + /// as when compiling the key expression, the origin + /// streams may be added some columns that have the + /// same name on left streams and right streams, for + /// example, if the join condition is something like: + /// id + 1 = id + 1, + /// the left streams and the right streams will have the + /// same constant column for `1` + /// Note that the origin left streams and right streams + /// will never have duplicated columns because in + /// DAGQueryBlockInterpreter we add qb_column_prefix in + /// final project step, so if the join condition is not + /// literal expression, the key names should never be + /// duplicated. In the above example, the final key names should be + /// something like `add(__qb_2_id, 1)` and `add(__qb_3_id, 1)` + if (actions->getSampleBlock().getNames().size() != getCurrentInputColumns().size() + key_names.size()) + { + std::unordered_set needed_columns; + for (const auto & c : getCurrentInputColumns()) + needed_columns.insert(c.name); + for (const auto & s : key_names) + needed_columns.insert(s); + + const auto & names = actions->getSampleBlock().getNames(); + for (const auto & name : names) + { + if (needed_columns.find(name) == needed_columns.end()) + actions->add(ExpressionAction::removeColumn(name)); } } return ret; @@ -575,6 +594,17 @@ String DAGExpressionAnalyzer::alignReturnType( return updated_name; } +String DAGExpressionAnalyzer::appendCast(const DataTypePtr & target_type, ExpressionActionsPtr & actions, const String & expr_name) +{ + // need to add cast function + // first construct the second argument + tipb::Expr type_expr; + constructStringLiteralTiExpr(type_expr, target_type->getName()); + auto type_expr_name = getActions(type_expr, actions); + String cast_expr_name = applyFunction("CAST", {expr_name, type_expr_name}, actions); + return cast_expr_name; +} + String DAGExpressionAnalyzer::appendCastIfNeeded( const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool explicit_cast) { @@ -595,13 +625,7 @@ String DAGExpressionAnalyzer::appendCastIfNeeded( { implicit_cast_count += !explicit_cast; - // need to add cast function - // first construct the second argument - tipb::Expr type_expr; - constructStringLiteralTiExpr(type_expr, expected_type->getName()); - auto type_expr_name = getActions(type_expr, actions); - String cast_expr_name = applyFunction("CAST", {expr_name, type_expr_name}, actions); - return cast_expr_name; + return appendCast(expected_type, actions, expr_name); } else { diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index f0a6e543986..c5cf8867b51 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -10,8 +10,8 @@ #include #include #include -#include #include +#include namespace DB { @@ -46,6 +46,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable void appendAggSelect( ExpressionActionsChain & chain, const tipb::Aggregation & agg, const tipb::DAGRequest & rqst, bool keep_session_timezone_info); String appendCastIfNeeded(const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool explicit_cast); + String appendCast(const DataTypePtr & target_type, ExpressionActionsPtr & actions, const String & expr_name); String alignReturnType(const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name, bool force_uint8); void initChain(ExpressionActionsChain & chain, const std::vector & columns) const { @@ -68,9 +69,10 @@ class DAGExpressionAnalyzer : private boost::noncopyable void makeExplicitSetForIndex(const tipb::Expr & expr, const ManageableStoragePtr & storage); String applyFunction(const String & func_name, const Names & arg_names, ExpressionActionsPtr & actions); Int32 getImplicitCastCount() { return implicit_cast_count; }; - bool appendTimeZoneCastsAfterTS(ExpressionActionsChain & chain, std::vector is_ts_column, - const tipb::DAGRequest & rqst, bool keep_UTC_column); - bool appendJoinKey(ExpressionActionsChain & chain, const tipb::Join & join, Names & key_names, bool tiflash_left); + bool appendTimeZoneCastsAfterTS( + ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst, bool keep_UTC_column); + bool appendJoinKey( + ExpressionActionsChain & chain, const tipb::Join & join, const DataTypes & key_types, Names & key_names, bool tiflash_left); String appendTimeZoneCast(const String & tz_col, const String & ts_col, const String & func_name, ExpressionActionsPtr & actions); DAGPreparedSets & getPreparedSets() { return prepared_sets; } String convertToUInt8(ExpressionActionsPtr & actions, const String & column_name); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 88166553f0d..d6293a820c7 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -45,6 +46,7 @@ extern const int TOO_MANY_COLUMNS; extern const int SCHEMA_VERSION_ERROR; extern const int UNKNOWN_EXCEPTION; extern const int COP_BAD_DAG_REQUEST; +extern const int NO_COMMON_TYPE; } // namespace ErrorCodes DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, @@ -497,18 +499,66 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & addTimeZoneCastAfterTS(is_ts_column, pipeline); } -void DAGQueryBlockInterpreter::prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left) +void DAGQueryBlockInterpreter::prepareJoinKeys( + const tipb::Join & join, const DataTypes & key_types, Pipeline & pipeline, Names & key_names, bool tiflash_left) { std::vector source_columns; for (auto const & p : pipeline.firstStream()->getHeader().getNamesAndTypesList()) source_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); ExpressionActionsChain chain; - if (dag_analyzer.appendJoinKey(chain, join, key_names, tiflash_left)) + if (dag_analyzer.appendJoinKey(chain, join, key_types, key_names, tiflash_left)) { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); } } + +/// ClickHouse require join key to be exactly the same type +/// TiDB only require the join key to be the same category +/// for example decimal(10,2) join decimal(20,0) is allowed in +/// TiDB and will throw exception in ClickHouse +void getJoinKeyTypes(const tipb::Join & join, DataTypes & key_types) +{ + for (int i = 0; i < join.left_join_keys().size(); i++) + { + if (!exprHasValidFieldType(join.left_join_keys(i)) || !exprHasValidFieldType(join.right_join_keys(i))) + throw Exception("Join key without field type", ErrorCodes::COP_BAD_DAG_REQUEST); + DataTypes types; + types.emplace_back(getDataTypeByFieldType(join.left_join_keys(i).field_type())); + types.emplace_back(getDataTypeByFieldType(join.right_join_keys(i).field_type())); + try + { + DataTypePtr common_type = getLeastSupertype(types); + key_types.emplace_back(common_type); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::NO_COMMON_TYPE) + { + DataTypePtr left_type = removeNullable(types[0]); + DataTypePtr right_type = removeNullable(types[1]); + right_type->isUnsignedInteger(); + if ((left_type->getTypeId() == TypeIndex::UInt64 && right_type->isInteger() && !right_type->isUnsignedInteger()) + || (right_type->getTypeId() == TypeIndex::UInt64 && left_type->isInteger() && !left_type->isUnsignedInteger())) + { + /// special case for uint64 and int + /// inorder to not throw exception, use Decimal(20, 0) as the common type + DataTypePtr common_type = std::make_shared>(20, 0); + if (types[0]->isNullable() || types[1]->isNullable()) + common_type = makeNullable(common_type); + key_types.emplace_back(common_type); + } + else + throw; + } + else + { + throw; + } + } + } +} + void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query) { // build @@ -576,13 +626,16 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p final_project.emplace_back(p.name, query_block.qb_column_prefix + p.name); } + DataTypes join_key_types; + getJoinKeyTypes(join, join_key_types); + /// add necessary transformation if the join key is an expression Pipeline left_pipeline; left_pipeline.streams = left_streams; - prepareJoinKeys(join, left_pipeline, left_key_names, true); + prepareJoinKeys(join, join_key_types, left_pipeline, left_key_names, true); Pipeline right_pipeline; right_pipeline.streams = right_streams; - prepareJoinKeys(join, right_pipeline, right_key_names, false); + prepareJoinKeys(join, join_key_types, right_pipeline, right_key_names, false); left_streams = left_pipeline.streams; right_streams = right_pipeline.streams; @@ -1045,7 +1098,8 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; - BlockInputStreamPtr input = std::make_shared(cluster, req, schema, context.getSettingsRef().max_threads, store_type); + BlockInputStreamPtr input + = std::make_shared(cluster, req, schema, context.getSettingsRef().max_threads, store_type); pipeline.streams = {input}; } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 0c962dcb6ee..9b397cbd672 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -90,7 +90,7 @@ class DAGQueryBlockInterpreter void executeImpl(Pipeline & pipeline); void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query); - void prepareJoinKeys(const tipb::Join & join, Pipeline & pipeline, Names & key_names, bool tiflash_left); + void prepareJoinKeys(const tipb::Join & join, const DataTypes & key_types, Pipeline & pipeline, Names & key_names, bool tiflash_left); void executeSubqueryInJoin(Pipeline & pipeline, SubqueryForSet & subquery); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); From f0757568ec86951375362bff4e71ee3442b9f512 Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 7 Apr 2020 10:13:44 +0800 Subject: [PATCH 43/54] some improvement for broadcast join (#600) * some improvement for broadcast join * format code * refine code * address comment * fix bug * address comments --- contrib/client-c | 2 +- .../DataStreams/CoprocessorBlockInputStream.h | 9 +-- .../CreatingSetsBlockInputStream.cpp | 74 +++++++++++-------- .../CreatingSetsBlockInputStream.h | 21 ++++-- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 73 +++++++++--------- .../Coprocessor/DAGQueryBlockInterpreter.h | 4 +- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 26 +++++-- dbms/src/Flash/Coprocessor/InterpreterDAG.h | 2 +- .../StreamingDAGBlockOutputStream.cpp | 54 +++++++------- .../StreamingDAGBlockOutputStream.h | 5 +- 10 files changed, 156 insertions(+), 114 deletions(-) diff --git a/contrib/client-c b/contrib/client-c index 087ddad8378..5ad0e10aad3 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 087ddad837866d05fe615737479ba075abe41e6e +Subproject commit 5ad0e10aad3a9b27b320ea005637b87a2f65674c diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h index 6217ac66e62..aec082e6b40 100644 --- a/dbms/src/DataStreams/CoprocessorBlockInputStream.h +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -15,7 +15,7 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream Block getSampleBlock() const { ColumnsWithTypeAndName columns; - for (auto name_and_column : schema) + for (auto & name_and_column : schema) { auto tp = getDataTypeByColumnInfo(name_and_column.second); ColumnWithTypeAndName col(tp, name_and_column.first); @@ -25,10 +25,9 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream } public: - CoprocessorBlockInputStream(pingcap::kv::Cluster * cluster_, const pingcap::coprocessor::Request & req_, const DAGSchema & schema_, - int concurrency, pingcap::kv::StoreType store_type) - : req(req_), - resp_iter(pingcap::coprocessor::Client::send(cluster_, &req, concurrency, store_type)), + CoprocessorBlockInputStream( + pingcap::kv::Cluster * cluster_, std::vector tasks, const DAGSchema & schema_, int concurrency) + : resp_iter(std::move(tasks), cluster_, concurrency, &Logger::get("pingcap/coprocessor")), schema(schema_), sample_block(getSampleBlock()), log(&Logger::get("pingcap/coprocessor")) diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 61a8b0be10a..e51d1627d40 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -1,9 +1,10 @@ -#include -#include -#include -#include #include +#include +#include +#include +#include #include + #include @@ -12,25 +13,38 @@ namespace DB namespace ErrorCodes { - extern const int SET_SIZE_LIMIT_EXCEEDED; +extern const int SET_SIZE_LIMIT_EXCEEDED; } +CreatingSetsBlockInputStream::CreatingSetsBlockInputStream(const BlockInputStreamPtr & input, + std::vector && subqueries_for_sets_list_, + const SizeLimits & network_transfer_limits) + : subqueries_for_sets_list(std::move(subqueries_for_sets_list_)), network_transfer_limits(network_transfer_limits) +{ + init(input); +} CreatingSetsBlockInputStream::CreatingSetsBlockInputStream( - const BlockInputStreamPtr & input, - const SubqueriesForSets & subqueries_for_sets_, - const SizeLimits & network_transfer_limits) - : subqueries_for_sets(subqueries_for_sets_), - network_transfer_limits(network_transfer_limits) + const BlockInputStreamPtr & input, const SubqueriesForSets & subqueries_for_sets, const SizeLimits & network_transfer_limits) + : network_transfer_limits(network_transfer_limits) { - for (auto & elem : subqueries_for_sets) + subqueries_for_sets_list.push_back(subqueries_for_sets); + init(input); +} + +void CreatingSetsBlockInputStream::init(const BlockInputStreamPtr & input) +{ + for (auto & subqueries_for_sets : subqueries_for_sets_list) { - if (elem.second.source) + for (auto & elem : subqueries_for_sets) { - children.push_back(elem.second.source); + if (elem.second.source) + { + children.push_back(elem.second.source); - if (elem.second.set) - elem.second.set->setHeader(elem.second.source->getHeader()); + if (elem.second.set) + elem.second.set->setHeader(elem.second.source->getHeader()); + } } } @@ -51,10 +65,7 @@ Block CreatingSetsBlockInputStream::readImpl() } -void CreatingSetsBlockInputStream::readPrefixImpl() -{ - createAll(); -} +void CreatingSetsBlockInputStream::readPrefixImpl() { createAll(); } Block CreatingSetsBlockInputStream::getTotals() @@ -72,14 +83,17 @@ void CreatingSetsBlockInputStream::createAll() { if (!created) { - for (auto & elem : subqueries_for_sets) + for (auto & subqueries_for_sets : subqueries_for_sets_list) { - if (elem.second.source) /// There could be prepared in advance Set/Join - no source is specified for them. + for (auto & elem : subqueries_for_sets) { - if (isCancelledOrThrowIfKilled()) - return; + if (elem.second.source) /// There could be prepared in advance Set/Join - no source is specified for them. + { + if (isCancelledOrThrowIfKilled()) + return; - createOne(elem.second); + createOne(elem.second); + } } } @@ -87,12 +101,11 @@ void CreatingSetsBlockInputStream::createAll() } } - void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) { - LOG_TRACE(log, (subquery.set ? "Creating set. " : "") - << (subquery.join ? "Creating join. " : "") - << (subquery.table ? "Filling temporary table. " : "")); + LOG_TRACE(log, + (subquery.set ? "Creating set. " : "") << (subquery.join ? "Creating join. " : "") + << (subquery.table ? "Filling temporary table. " : "")); Stopwatch watch; BlockOutputStreamPtr table_out; @@ -137,7 +150,8 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) rows_to_transfer += block.rows(); bytes_to_transfer += block.bytes(); - if (!network_transfer_limits.check(rows_to_transfer, bytes_to_transfer, "IN/JOIN external table", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) + if (!network_transfer_limits.check( + rows_to_transfer, bytes_to_transfer, "IN/JOIN external table", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) done_with_table = true; } @@ -188,4 +202,4 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) } } -} +} // namespace DB diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h index ff8fe5683c7..79b3b730ac4 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h @@ -1,11 +1,14 @@ #pragma once -#include #include -#include /// SubqueriesForSets +#include /// SubqueriesForSets +#include -namespace Poco { class Logger; } +namespace Poco +{ +class Logger; +} namespace DB { @@ -18,8 +21,10 @@ class CreatingSetsBlockInputStream : public IProfilingBlockInputStream { public: CreatingSetsBlockInputStream( - const BlockInputStreamPtr & input, - const SubqueriesForSets & subqueries_for_sets_, + const BlockInputStreamPtr & input, const SubqueriesForSets & subqueries_for_sets_, const SizeLimits & network_transfer_limits); + + CreatingSetsBlockInputStream(const BlockInputStreamPtr & input, + std::vector && subqueries_for_sets_list_, const SizeLimits & network_transfer_limits); String getName() const override { return "CreatingSets"; } @@ -34,7 +39,9 @@ class CreatingSetsBlockInputStream : public IProfilingBlockInputStream void readPrefixImpl() override; private: - SubqueriesForSets subqueries_for_sets; + void init(const BlockInputStreamPtr & input); + + std::vector subqueries_for_sets_list; bool created = false; SizeLimits network_transfer_limits; @@ -49,4 +56,4 @@ class CreatingSetsBlockInputStream : public IProfilingBlockInputStream void createOne(SubqueryForSet & subquery); }; -} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 6dba124513a..b284f5a6212 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -51,7 +51,7 @@ extern const int NO_COMMON_TYPE; DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const tipb::DAGRequest & rqst_, ASTPtr dummy_query_, - const DAGQuerySource & dag_) + const DAGQuerySource & dag_, std::vector & subqueriesForSets_) : context(context_), input_streams_vec(input_streams_vec_), query_block(query_block_), @@ -59,6 +59,7 @@ DAGQueryBlockInterpreter::DAGQueryBlockInterpreter(Context & context_, const std rqst(rqst_), dummy_query(std::move(dummy_query_)), dag(dag_), + subqueriesForSets(subqueriesForSets_), log(&Logger::get("DAGQueryBlockInterpreter")) { if (query_block.selection != nullptr) @@ -454,13 +455,12 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & { try { - pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, - max_streams); + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); } catch (DB::Exception & e) { e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() - + "`, table_id: " + DB::toString(table_id) + ")"); + + "`, table_id: " + DB::toString(table_id) + ")"); throw; } } @@ -468,7 +468,6 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & { throw Exception("Should not reach here", ErrorCodes::LOGICAL_ERROR); } - LOG_INFO(log, "dag execution stream size: " << regions.size()); if (pipeline.streams.empty()) { @@ -978,16 +977,6 @@ void DAGQueryBlockInterpreter::executeOrder(Pipeline & pipeline, Strings & order // } //} -void DAGQueryBlockInterpreter::executeSubqueryInJoin(Pipeline & pipeline, SubqueryForSet & subquery) -{ - const Settings & settings = context.getSettingsRef(); - executeUnion(pipeline); - SubqueriesForSets subquries; - subquries[query_block.qb_column_prefix + "join"] = subquery; - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), subquries, - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); -} - void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb::Executor * root) { const tipb::Executor * current = root; @@ -1063,6 +1052,11 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) { + // remote query containing agg/limit/topN can not running + // in parellel, but current remote query is running in + // parellel, so just disable this corner case. + if (query_block.aggregation || query_block.limitOrTopN) + throw Exception("Remote query containing agg or limit or topN is not supported", ErrorCodes::COP_BAD_DAG_REQUEST); const auto & ts = query_block.source->tbl_scan(); std::vector> key_ranges; for (auto & range : ts.ranges()) @@ -1081,9 +1075,7 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) } ::tipb::DAGRequest dag_req; - copyExecutorTreeWithLocalTableScan(dag_req, query_block.root); - DAGSchema schema; ColumnsWithTypeAndName columns; for (int i = 0; i < (int)query_block.output_field_types.size(); i++) @@ -1098,18 +1090,33 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) } Block sample_block = Block(columns); - pingcap::coprocessor::Request req; - - dag_req.SerializeToString(&req.data); - req.tp = pingcap::coprocessor::ReqType::DAG; - req.start_ts = context.getSettingsRef().read_tso; - req.ranges = cop_key_ranges; + pingcap::coprocessor::RequestPtr req = std::make_shared(); + dag_req.SerializeToString(&(req->data)); + req->tp = pingcap::coprocessor::ReqType::DAG; + req->start_ts = context.getSettingsRef().read_tso; pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); + pingcap::kv::Backoffer bo(pingcap::kv::copBuildTaskMaxBackoff); pingcap::kv::StoreType store_type = pingcap::kv::TiFlash; - BlockInputStreamPtr input - = std::make_shared(cluster, req, schema, context.getSettingsRef().max_threads, store_type); - pipeline.streams = {input}; + sort(cop_key_ranges.begin(), cop_key_ranges.end()); + auto all_tasks = pingcap::coprocessor::buildCopTasks(bo, cluster, cop_key_ranges, req, store_type, &Logger::get("pingcap/coprocessor")); + + size_t concurrent_num = std::min(context.getSettingsRef().max_threads, all_tasks.size()); + size_t task_per_thread = all_tasks.size() / concurrent_num; + size_t rest_task = all_tasks.size() % concurrent_num; + for (size_t i = 0, task_start = 0; i < concurrent_num; i++) + { + size_t task_end = task_start + task_per_thread; + if (i < rest_task) + task_end++; + if (task_end == task_start) + continue; + std::vector tasks(all_tasks.begin() + task_start, all_tasks.begin() + task_end); + + BlockInputStreamPtr input = std::make_shared(cluster, tasks, schema, 1); + pipeline.streams.push_back(input); + task_start = task_end; + } } void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) @@ -1139,6 +1146,8 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) //if (dag.hasSelection()) //recordProfileStreams(pipeline, dag.getSelectionIndex()); } + LOG_INFO(log, + "execution stream size for query block(before aggregation) " << query_block.qb_column_prefix << " is " << pipeline.streams.size()); if (res.need_aggregate) { // execute aggregation @@ -1167,17 +1176,11 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) //recordProfileStreams(pipeline, dag.getLimitIndex()); } - if (dag.writer->writer != nullptr && query_block.isRootQueryBlock()) - { - for (auto & stream : pipeline.streams) - stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, - dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); - } - if (query_block.source->tp() == tipb::ExecType::TypeJoin) { - // add the - executeSubqueryInJoin(pipeline, right_query); + SubqueriesForSets subquries; + subquries[query_block.qb_column_prefix + "join"] = right_query; + subqueriesForSets.emplace_back(subquries); } } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 9b397cbd672..d2ef6519683 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -79,7 +79,7 @@ class DAGQueryBlockInterpreter public: DAGQueryBlockInterpreter(Context & context_, const std::vector & input_streams_vec_, const DAGQueryBlock & query_block_, bool keep_session_timezone_info_, const tipb::DAGRequest & rqst, ASTPtr dummp_query, - const DAGQuerySource & dag_); + const DAGQuerySource & dag_, std::vector & subqueriesForSets_); ~DAGQueryBlockInterpreter() = default; @@ -91,7 +91,6 @@ class DAGQueryBlockInterpreter void executeTS(const tipb::TableScan & ts, Pipeline & pipeline); void executeJoin(const tipb::Join & join, Pipeline & pipeline, SubqueryForSet & right_query); void prepareJoinKeys(const tipb::Join & join, const DataTypes & key_types, Pipeline & pipeline, Names & key_names, bool tiflash_left); - void executeSubqueryInJoin(Pipeline & pipeline, SubqueryForSet & subquery); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr); void executeOrder(Pipeline & pipeline, Strings & order_column_names); @@ -129,6 +128,7 @@ class DAGQueryBlockInterpreter std::vector conditions; const DAGQuerySource & dag; + std::vector & subqueriesForSets; Poco::Logger * log; }; diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 6e777a123cc..2bbf2ba731b 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -30,6 +31,8 @@ #include #include +#include "StreamingDAGBlockOutputStream.h" + namespace DB { @@ -919,24 +922,24 @@ void InterpreterDAG::executeUnion(Pipeline & pipeline) } } -BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block) +BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block, std::vector & subqueriesForSets) { if (!query_block.children.empty()) { std::vector input_streams_vec; for (auto & child : query_block.children) { - BlockInputStreams child_streams = executeQueryBlock(*child); + BlockInputStreams child_streams = executeQueryBlock(*child, subqueriesForSets); input_streams_vec.push_back(child_streams); } DAGQueryBlockInterpreter query_block_interpreter( - context, input_streams_vec, query_block, keep_session_timezone_info, dag.getDAGRequest(), dag.getAST(), dag); + context, input_streams_vec, query_block, keep_session_timezone_info, dag.getDAGRequest(), dag.getAST(), dag, subqueriesForSets); return query_block_interpreter.execute(); } else { DAGQueryBlockInterpreter query_block_interpreter( - context, {}, query_block, keep_session_timezone_info, dag.getDAGRequest(), dag.getAST(), dag); + context, {}, query_block, keep_session_timezone_info, dag.getDAGRequest(), dag.getAST(), dag, subqueriesForSets); return query_block_interpreter.execute(); } } @@ -946,12 +949,25 @@ BlockIO InterpreterDAG::execute() /// region_info should based on the source executor, however /// tidb does not support multi-table dag request yet, so /// it is ok to use the same region_info for the whole dag request - BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock()); + std::vector subqueriesForSets; + BlockInputStreams streams = executeQueryBlock(*dag.getQueryBlock(), subqueriesForSets); Pipeline pipeline; pipeline.streams = streams; + if (dag.writer->writer != nullptr) + { + for (auto & stream : pipeline.streams) + stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, + dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader()); + } executeUnion(pipeline); + if (!subqueriesForSets.empty()) + { + const Settings & settings = context.getSettingsRef(); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), std::move(subqueriesForSets), + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); + } BlockIO res; res.in = pipeline.firstStream(); diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index 8854c886412..0f6d3429297 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -37,7 +37,7 @@ class InterpreterDAG : public IInterpreter BlockIO execute(); private: - BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block); + BlockInputStreams executeQueryBlock(DAGQueryBlock & query_block, std::vector & subqueriesForSets); void executeUnion(Pipeline & pipeline); /* void executeImpl(Pipeline & pipeline); diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp index a77bb22a5db..7e1f2aae714 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.cpp @@ -1,8 +1,7 @@ -#include - #include #include #include +#include namespace DB { @@ -13,16 +12,15 @@ extern const int UNSUPPORTED_PARAMETER; extern const int LOGICAL_ERROR; } // namespace ErrorCodes -StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, - std::vector && result_field_types_, Block && header_) - : input(input_), - finished(false), - writer(writer_), - result_field_types(std::move(result_field_types_)), - header(std::move(header_)), - records_per_chunk(records_per_chunk_), - current_records_num(0), - encode_type(encode_type_) +StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, + tipb::EncodeType encode_type_, std::vector && result_field_types_, Block && header_) + : finished(false), + writer(writer_), + result_field_types(std::move(result_field_types_)), + header(std::move(header_)), + records_per_chunk(records_per_chunk_), + current_records_num(0), + encode_type(encode_type_) { if (encode_type == tipb::EncodeType::TypeDefault) { @@ -39,18 +37,17 @@ StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr i } else { - throw Exception("Only Default and Arrow encode type is supported in StreamingDAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); + throw Exception( + "Only Default and Arrow encode type is supported in StreamingDAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); } + children.push_back(input_); } -void StreamingDAGBlockInputStream::readPrefix() -{ - input->readPrefix(); -} +void StreamingDAGBlockInputStream::readPrefix() { children.back()->readPrefix(); } void StreamingDAGBlockInputStream::encodeChunkToDAGResponse() { - ::coprocessor::BatchResponse resp ; + ::coprocessor::BatchResponse resp; tipb::SelectResponse dag_response; dag_response.set_encode_type(encode_type); @@ -73,14 +70,15 @@ void StreamingDAGBlockInputStream::readSuffix() { encodeChunkToDAGResponse(); } - input->readSuffix(); + children.back()->readSuffix(); } Block StreamingDAGBlockInputStream::readImpl() { if (finished) return {}; - while(Block block = input->read()) { + while (Block block = children.back()->read()) + { if (!block) { finished = true; @@ -88,16 +86,22 @@ Block StreamingDAGBlockInputStream::readImpl() } if (block.columns() != result_field_types.size()) throw Exception("Output column size mismatch with field type size", ErrorCodes::LOGICAL_ERROR); - if (records_per_chunk == -1) { + if (records_per_chunk == -1) + { current_records_num = 0; - if (block.rows() > 0) { + if (block.rows() > 0) + { chunk_codec_stream->encode(block, 0, block.rows()); encodeChunkToDAGResponse(); } - } else { + } + else + { size_t rows = block.rows(); - for (size_t row_index = 0; row_index < rows;) { - if (current_records_num >= records_per_chunk) { + for (size_t row_index = 0; row_index < rows;) + { + if (current_records_num >= records_per_chunk) + { encodeChunkToDAGResponse(); } const size_t upper = std::min(row_index + (records_per_chunk - current_records_num), rows); diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h index f2d7a57f0ad..46a83f493aa 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockOutputStream.h @@ -20,8 +20,8 @@ namespace DB class StreamingDAGBlockInputStream : public IProfilingBlockInputStream { public: - StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, - std::vector && result_field_types, Block && header_); + StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, + tipb::EncodeType encodeType_, std::vector && result_field_types, Block && header_); Block getHeader() const override { return header; } Block readImpl() override; @@ -31,7 +31,6 @@ class StreamingDAGBlockInputStream : public IProfilingBlockInputStream void encodeChunkToDAGResponse(); private: - BlockInputStreamPtr input; bool finished; StreamWriterPtr writer; std::vector result_field_types; From 545653c1051b1362f35b4a8af9b15738db08c5cf Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 9 Apr 2020 00:52:22 +0800 Subject: [PATCH 44/54] fix bug --- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 29ef6ad7b0c..f700e28c75e 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -31,6 +31,7 @@ DAGDriver::DAGDriver(Context & context_, const tipb::DAGRequest & dag_req dag_request(dag_request_), regions(regions_), dag_response(dag_response_), + writer(nullptr), internal(internal_), log(&Logger::get("DAGDriver")) { From 8a07c8cbd7d1f221168b0d7e59bb213494655c91 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 9 Apr 2020 23:49:23 +0800 Subject: [PATCH 45/54] fix bug --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 51 +++++++++++++++---- .../Coprocessor/DAGQueryBlockInterpreter.h | 4 +- 2 files changed, 43 insertions(+), 12 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index b7b2d546818..d8131652217 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -480,6 +480,38 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & ss.str(); })); + DAGSchema schema; + ::tipb::DAGRequest dag_req; + + { + const auto & table_info = storage->getTableInfo(); + tipb::Executor * ts_exec = dag_req.add_executors(); + ts_exec->set_tp(tipb::ExecType::TypeTableScan); + *(ts_exec->mutable_tbl_scan()) = ts; + + for (int i = 0; i < ts.columns().size(); ++i) + { + const auto & col = ts.columns(i); + auto col_id = col.column_id(); + + if (col_id == DB::TiDBPkColumnID) + { + ColumnInfo ci; + ci.tp = TiDB::TypeLongLong; + ci.setPriKeyFlag(); + ci.setNotNullFlag(); + schema.emplace_back(std::make_pair(handle_column_name, std::move(ci))); + } + else + { + auto & col_info = table_info.getColumnInfo(col_id); + schema.emplace_back(std::make_pair(col_info.name, col_info)); + } + dag_req.add_output_offsets(i); + } + dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); + } + std::vector ranges; for (auto & info : region_retry) { @@ -487,7 +519,7 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & ranges.emplace_back(range.first, range.second); } sort(ranges.begin(), ranges.end()); - executeRemoteQueryWithRanges(pipeline, ranges); + executeRemoteQueryImpl(pipeline, ranges, dag_req, schema); } } @@ -1096,12 +1128,6 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) cop_key_ranges.emplace_back(static_cast(key_range.first), static_cast(key_range.second)); } sort(cop_key_ranges.begin(), cop_key_ranges.end()); - executeRemoteQueryWithRanges(pipeline, cop_key_ranges); -} - -void DAGQueryBlockInterpreter::executeRemoteQueryWithRanges( - Pipeline & pipeline, const std::vector & cop_key_ranges) -{ ::tipb::DAGRequest dag_req; copyExecutorTreeWithLocalTableScan(dag_req, query_block.root); @@ -1113,11 +1139,14 @@ void DAGQueryBlockInterpreter::executeRemoteQueryWithRanges( ColumnInfo info = fieldTypeToColumnInfo(query_block.output_field_types[i]); String col_name = query_block.qb_column_prefix + "col_" + std::to_string(i); schema.push_back(std::make_pair(col_name, info)); - auto tp = getDataTypeByColumnInfo(info); - ColumnWithTypeAndName col(tp, col_name); - columns.emplace_back(col); } - Block sample_block = Block(columns); + + executeRemoteQueryImpl(pipeline, cop_key_ranges, dag_req, schema); +} + +void DAGQueryBlockInterpreter::executeRemoteQueryImpl(Pipeline & pipeline, + const std::vector & cop_key_ranges, ::tipb::DAGRequest & dag_req, const DAGSchema & schema) +{ pingcap::coprocessor::RequestPtr req = std::make_shared(); dag_req.SerializeToString(&(req->data)); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 2b82a012f86..4a87b8ec470 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -7,6 +7,7 @@ #pragma GCC diagnostic pop #include +#include #include #include #include @@ -107,7 +108,8 @@ class DAGQueryBlockInterpreter bool addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline); private: - void executeRemoteQueryWithRanges(Pipeline & pipeline, const std::vector & key_ranges); + void executeRemoteQueryImpl(Pipeline & pipeline, const std::vector & cop_key_ranges, + ::tipb::DAGRequest & dag_req, const DAGSchema & schema); Context & context; std::vector input_streams_vec; From 1d13d59f284cceccc7d16fbfe705702fe9e9f00f Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 23 Apr 2020 14:22:58 +0800 Subject: [PATCH 46/54] make TiFlash backward compatible to old tipb (#653) * 1. re-enable exec info in dag response, 2. support old style dag request * basic support for execute summary * refine support of executor time for join plan * format code * address comments --- contrib/tipb | 2 +- dbms/src/Flash/Coprocessor/DAGContext.h | 11 +- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 22 +++- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 108 ++++++++-------- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 14 ++- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 119 ++++++++++++++---- .../Coprocessor/DAGQueryBlockInterpreter.h | 2 +- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 10 +- dbms/src/Flash/Coprocessor/DAGQuerySource.h | 2 + 9 files changed, 202 insertions(+), 88 deletions(-) diff --git a/contrib/tipb b/contrib/tipb index 5f3b53798f0..59279b98966 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 5f3b53798f0099d9016c9cb0dca1a04fdeac8116 +Subproject commit 59279b98966726a78ec4af59f8bd593b576d602b diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 385fc48cd68..e9ee4e4823b 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -8,11 +8,18 @@ namespace DB class Context; +struct ProfileStreamsInfo +{ + UInt32 qb_id; + BlockInputStreams input_streams; +}; /// A context used to track the information that needs to be passed around during DAG planning. struct DAGContext { - DAGContext(size_t profile_list_size) { profile_streams_list.resize(profile_list_size); }; - std::vector profile_streams_list; + DAGContext(){}; + std::map profile_streams_map; + std::unordered_map profile_streams_map_for_join_build_side; + std::unordered_map> qb_id_to_join_alias_map; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index f700e28c75e..5e129cf14ce 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -57,7 +57,7 @@ template void DAGDriver::execute() try { - DAGContext dag_context(dag_request.executors_size()); + DAGContext dag_context; DAGQuerySource dag(context, dag_context, regions, dag_request, writer, batch); BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); @@ -90,13 +90,13 @@ try if (!dag_request.has_collect_execution_summaries() || !dag_request.collect_execution_summaries()) return; // add ExecutorExecutionSummary info - for (auto & p_streams : dag_context.profile_streams_list) + for (auto & p : dag_context.profile_streams_map) { auto * executeSummary = dag_response->add_execution_summaries(); UInt64 time_processed_ns = 0; UInt64 num_produced_rows = 0; UInt64 num_iterations = 0; - for (auto & streamPtr : p_streams) + for (auto & streamPtr : p.second.input_streams) { if (auto * p_stream = dynamic_cast(streamPtr.get())) { @@ -105,9 +105,25 @@ try num_iterations += p_stream->getProfileInfo().blocks; } } + for (auto & join_alias : dag_context.qb_id_to_join_alias_map[p.second.qb_id]) + { + if (dag_context.profile_streams_map_for_join_build_side.find(join_alias) + != dag_context.profile_streams_map_for_join_build_side.end()) + { + UInt64 process_time_for_build = 0; + for (auto & join_stream : dag_context.profile_streams_map_for_join_build_side[join_alias]) + { + if (auto * p_stream = dynamic_cast(join_stream.get())) + process_time_for_build = std::max(process_time_for_build, p_stream->getProfileInfo().execution_time); + } + time_processed_ns += process_time_for_build; + } + } executeSummary->set_time_processed_ns(time_processed_ns); executeSummary->set_num_produced_rows(num_produced_rows); executeSummary->set_num_iterations(num_iterations); + if (dag_request.has_root_executor()) + executeSummary->set_executor_id(p.first); } } catch (const RegionException & e) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index 3bcd0928e4b..8632ef5244d 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -60,30 +60,36 @@ void collectOutPutFieldTypesFromAgg(std::vector & field_type, c } } -DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root_) - : id(id_), root(root_), qb_column_prefix("__QB_" + std::to_string(id_) + "_") +/// construct DAGQueryBlock from a tree struct based executors, which is the +/// format after supporting join in dag request +DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor & root_) + : id(id_), root(&root_), qb_column_prefix("__QB_" + std::to_string(id_) + "_"), qb_join_subquery_alias(qb_column_prefix + "join") { const tipb::Executor * current = root; - while (!isSourceNode(current)) + while (!isSourceNode(current) && current->has_executor_id()) { switch (current->tp()) { case tipb::ExecType::TypeSelection: assignOrThrowException(&selection, current, SEL_NAME); + selection_name = current->executor_id(); current = ¤t->selection().child(); break; case tipb::ExecType::TypeAggregation: case tipb::ExecType::TypeStreamAgg: assignOrThrowException(&aggregation, current, AGG_NAME); + aggregation_name = current->executor_id(); collectOutPutFieldTypesFromAgg(output_field_types, current->aggregation()); current = ¤t->aggregation().child(); break; case tipb::ExecType::TypeLimit: assignOrThrowException(&limitOrTopN, current, LIMIT_NAME); + limitOrTopN_name = current->executor_id(); current = ¤t->limit().child(); break; case tipb::ExecType::TypeTopN: assignOrThrowException(&limitOrTopN, current, TOPN_NAME); + limitOrTopN_name = current->executor_id(); current = ¤t->topn().child(); break; case tipb::ExecType::TypeIndexScan: @@ -92,87 +98,63 @@ DAGQueryBlock::DAGQueryBlock(UInt32 id_, const tipb::Executor * root_) throw Exception("Should not reach here", ErrorCodes::LOGICAL_ERROR); } } + + if (!current->has_executor_id()) + throw Exception("Tree struct based executor must have executor id", ErrorCodes::COP_BAD_DAG_REQUEST); + assignOrThrowException(&source, current, SOURCE_NAME); + source_name = current->executor_id(); if (current->tp() == tipb::ExecType::TypeJoin) { if (source->join().children_size() != 2) - throw Exception("Join executor children size not equal to 2field type", ErrorCodes::COP_BAD_DAG_REQUEST); - children.push_back(std::make_shared(id * 2, &source->join().children(0))); - children.push_back(std::make_shared(id * 2 + 1, &source->join().children(1))); + throw Exception("Join executor children size not equal to 2", ErrorCodes::COP_BAD_DAG_REQUEST); + children.push_back(std::make_shared(id * 2, source->join().children(0))); + children.push_back(std::make_shared(id * 2 + 1, source->join().children(1))); } fillOutputFieldTypes(); } -/* -DAGQueryBlock::DAGQueryBlock(UInt32 id_, std::vector & executors, int start_index, int end_index) -: id(id_), qb_column_prefix("___QB_" + std::to_string(id_) + "_") +/// construct DAGQueryBlock from a list struct based executors, which is the +/// format before supporting join in dag request +DAGQueryBlock::DAGQueryBlock(UInt32 id_, const ::google::protobuf::RepeatedPtrField & executors) + : id(id_), root(nullptr), qb_column_prefix("__QB_" + std::to_string(id_) + "_"), qb_join_subquery_alias(qb_column_prefix + "join") { - for (int i = end_index; i >= start_index; i--) + for (int i = (int)executors.size() - 1; i >= 0; i--) { - //int build_end_index, build_start_index; - //int probe_end_index, probe_start_index; - switch (executors[i]->tp()) + switch (executors[i].tp()) { case tipb::ExecType::TypeTableScan: - assignOrThrowException(&source, executors[i], SOURCE_NAME); + assignOrThrowException(&source, &executors[i], SOURCE_NAME); + /// use index as the prefix for executor name so when we sort by + /// the executor name, it will result in the same order as it is + /// in the dag_request, this is needed when filling executeSummary + /// in DAGDriver + source_name = std::to_string(i) + "_tablescan"; break; case tipb::ExecType::TypeSelection: - assignOrThrowException(&selection, executors[i], SEL_NAME); + assignOrThrowException(&selection, &executors[i], SEL_NAME); + selection_name = std::to_string(i) + "_selection"; break; case tipb::ExecType::TypeStreamAgg: case tipb::ExecType::TypeAggregation: - assignOrThrowException(&aggregation, executors[i], AGG_NAME); - collectOutPutFieldTypesFromAgg(output_field_types, executors[i]->aggregation()); + assignOrThrowException(&aggregation, &executors[i], AGG_NAME); + aggregation_name = std::to_string(i) + "_aggregation"; + collectOutPutFieldTypesFromAgg(output_field_types, executors[i].aggregation()); break; case tipb::ExecType::TypeTopN: - assignOrThrowException(&limitOrTopN, executors[i], TOPN_NAME); + assignOrThrowException(&limitOrTopN, &executors[i], TOPN_NAME); + limitOrTopN_name = std::to_string(i) + "_limitOrTopN"; break; case tipb::ExecType::TypeLimit: - assignOrThrowException(&limitOrTopN, executors[i], LIMIT_NAME); + assignOrThrowException(&limitOrTopN, &executors[i], LIMIT_NAME); + limitOrTopN_name = std::to_string(i) + "_limitOrTopN"; break; -// case tipb::ExecType::TypeJoin: -// if (i <= start_index) -// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); -// if (executors[i - 1] == &(executors[i]->join().build_exec())) -// { -// build_end_index = i - 1; -// build_start_index = build_end_index; -// while (build_start_index >= start_index && executors[build_start_index] != &(executors[i]->join().probe_exec())) -// build_start_index--; -// if (build_start_index < start_index) -// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); -// probe_end_index = build_start_index; -// build_start_index++; -// probe_start_index = start_index; -// } -// else if (executors[i - 1] == &(executors[i]->join().probe_exec())) -// { -// probe_end_index = i - 1; -// probe_start_index = probe_end_index; -// while (probe_start_index >= start_index && executors[probe_start_index] != &(executors[i]->join().build_exec())) -// probe_start_index--; -// if (probe_start_index < start_index) -// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); -// build_end_index = probe_start_index; -// probe_start_index++; -// build_start_index = start_index; -// } -// else -// { -// throw Exception("Join executor without child executor", ErrorCodes::LOGICAL_ERROR); -// } -// children.push_back(std::make_shared(id * 2, executors, probe_start_index, probe_end_index)); -// children.push_back(std::make_shared(id * 2 + 1, executors, build_start_index, build_end_index)); -// // to break the for loop -// i = start_index - 1; -// break; default: - throw Exception("Unsupported executor in DAG request: " + executors[i]->DebugString(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Unsupported executor in DAG request: " + executors[i].DebugString(), ErrorCodes::NOT_IMPLEMENTED); } } fillOutputFieldTypes(); } -*/ void DAGQueryBlock::fillOutputFieldTypes() { @@ -203,4 +185,16 @@ void DAGQueryBlock::fillOutputFieldTypes() } } +void DAGQueryBlock::collectAllPossibleChildrenJoinSubqueryAlias(std::unordered_map> & result) +{ + std::vector all_qb_join_subquery_alias; + for (auto & child : children) + { + child->collectAllPossibleChildrenJoinSubqueryAlias(result); + all_qb_join_subquery_alias.insert(all_qb_join_subquery_alias.end(), result[child->id].begin(), result[child->id].end()); + } + all_qb_join_subquery_alias.push_back(qb_join_subquery_alias); + result[id] = all_qb_join_subquery_alias; +} + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index c19087c1730..e8d7a276f08 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -24,20 +24,30 @@ using TiFlashMetricsPtr = std::shared_ptr; class DAGQueryBlock { public: - DAGQueryBlock(UInt32 id, const tipb::Executor * root); - //DAGQueryBlock(UInt32 id, std::vector & executors, int start_index, int end_index); + DAGQueryBlock(UInt32 id, const tipb::Executor & root); + DAGQueryBlock(UInt32 id, const ::google::protobuf::RepeatedPtrField & executors); + /// the xxx_name is added for compatibility issues: before join is supported, executor does not + /// has executor name, after join is supported in dag request, every executor has an unique + /// name(executor->executor_id()). Since We can not always get the executor name from executor + /// itself, we had to add xxx_name here const tipb::Executor * source = nullptr; + String source_name; const tipb::Executor * selection = nullptr; + String selection_name; const tipb::Executor * aggregation = nullptr; + String aggregation_name; const tipb::Executor * limitOrTopN = nullptr; + String limitOrTopN_name; UInt32 id; const tipb::Executor * root; String qb_column_prefix; + String qb_join_subquery_alias; std::vector> children; std::vector output_field_types; // kinds of project std::vector output_offsets; void fillOutputFieldTypes(); + void collectAllPossibleChildrenJoinSubqueryAlias(std::unordered_map> & result); bool isRootQueryBlock() const { return id == 1; }; bool isRemoteQuery() const { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index d8131652217..6bf879c9898 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -373,7 +373,8 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & else { for (size_t i = 0; i < required_columns.size(); i++) - /// for child query block, add alias start with qb_column_prefix to avoid column name conflict + /// for child query block, the final project is all the columns read from + /// the table and add alias start with qb_column_prefix to avoid column name conflict final_project.emplace_back(required_columns[i], query_block.qb_column_prefix + required_columns[i]); } } @@ -559,7 +560,27 @@ void DAGQueryBlockInterpreter::executeTS(const tipb::TableScan & ts, Pipeline & }); } - addTimeZoneCastAfterTS(is_ts_column, pipeline); + if (addTimeZoneCastAfterTS(is_ts_column, pipeline) && !query_block.aggregation + && (keep_session_timezone_info || !query_block.isRootQueryBlock())) + { + if (query_block.isRootQueryBlock()) + { + for (size_t i = 0; i < query_block.output_offsets.size(); i++) + { + int column_index = query_block.output_offsets[i]; + if (is_ts_column[column_index]) + final_project[i].first = analyzer->getCurrentInputColumns()[column_index].name; + } + } + else + { + for (size_t i = 0; i < final_project.size(); i++) + { + if (is_ts_column[i]) + final_project[i].first = analyzer->getCurrentInputColumns()[i].name; + } + } + } } void DAGQueryBlockInterpreter::prepareJoinKeys( @@ -710,6 +731,7 @@ void DAGQueryBlockInterpreter::executeJoin(const tipb::Join & join, Pipeline & p right_query.source = right_pipeline.firstStream(); right_query.join = joinPtr; right_query.join->setSampleBlock(right_query.source->getHeader()); + dag.getDAGContext().profile_streams_map_for_join_build_side[query_block.qb_join_subquery_alias].push_back(right_query.source); std::vector source_columns; for (const auto & p : left_streams[0]->getHeader().getNamesAndTypesList()) @@ -1023,23 +1045,27 @@ void DAGQueryBlockInterpreter::executeOrder(Pipeline & pipeline, Strings & order limit, settings.max_bytes_before_external_sort, context.getTemporaryPath()); } -//void DAGQueryBlockInterpreter::recordProfileStreams(Pipeline & pipeline, Int32 index) -//{ -// for (auto & stream : pipeline.streams) -// { -// dag.getDAGContext().profile_streams_list[index].push_back(stream); -// } -//} +void DAGQueryBlockInterpreter::recordProfileStreams(Pipeline & pipeline, const String & key) +{ + dag.getDAGContext().profile_streams_map[key].qb_id = query_block.id; + for (auto & stream : pipeline.streams) + { + dag.getDAGContext().profile_streams_map[key].input_streams.push_back(stream); + } +} -void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb::Executor * root) +void copyExecutorTreeWithLocalTableScan( + tipb::DAGRequest & dag_req, const tipb::Executor * root, tipb::EncodeType encode_type, const tipb::DAGRequest & org_req) { const tipb::Executor * current = root; - auto * exec = dag_req.add_executors(); + auto * exec = dag_req.mutable_root_executor(); + int exec_id = 0; while (current->tp() != tipb::ExecType::TypeTableScan) { if (current->tp() == tipb::ExecType::TypeSelection) { exec->set_tp(tipb::ExecType::TypeSelection); + exec->set_executor_id("selection_" + std::to_string(exec_id)); auto * sel = exec->mutable_selection(); for (auto const & condition : current->selection().conditions()) { @@ -1052,6 +1078,7 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: else if (current->tp() == tipb::ExecType::TypeAggregation || current->tp() == tipb::ExecType::TypeStreamAgg) { exec->set_tp(current->tp()); + exec->set_executor_id("aggregation_" + std::to_string(exec_id)); auto * agg = exec->mutable_aggregation(); for (auto const & expr : current->aggregation().agg_func()) { @@ -1070,6 +1097,7 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: else if (current->tp() == tipb::ExecType::TypeLimit) { exec->set_tp(current->tp()); + exec->set_executor_id("limit_" + std::to_string(exec_id)); auto * limit = exec->mutable_limit(); limit->set_limit(current->limit().limit()); exec = limit->mutable_child(); @@ -1078,6 +1106,7 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: else if (current->tp() == tipb::ExecType::TypeTopN) { exec->set_tp(current->tp()); + exec->set_executor_id("topN_" + std::to_string(exec_id)); auto * topn = exec->mutable_topn(); topn->set_limit(current->topn().limit()); for (auto const & expr : current->topn().order_by()) @@ -1092,16 +1121,22 @@ void copyExecutorTreeWithLocalTableScan(tipb::DAGRequest & dag_req, const tipb:: { throw Exception("Not supported yet"); } + exec_id++; } if (current->tp() != tipb::ExecType::TypeTableScan) throw Exception("Only support copy from table scan sourced query block"); exec->set_tp(tipb::ExecType::TypeTableScan); + exec->set_executor_id("tablescan_" + std::to_string(exec_id)); auto * new_ts = new tipb::TableScan(current->tbl_scan()); new_ts->set_next_read_engine(tipb::EngineType::Local); exec->set_allocated_tbl_scan(new_ts); - dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); + dag_req.set_encode_type(encode_type); + if (org_req.has_time_zone_name() && org_req.time_zone_name().length() > 0) + dag_req.set_time_zone_name(org_req.time_zone_name()); + else if (org_req.has_time_zone_offset()) + dag_req.set_time_zone_offset(org_req.time_zone_offset()); } void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) @@ -1130,18 +1165,59 @@ void DAGQueryBlockInterpreter::executeRemoteQuery(Pipeline & pipeline) sort(cop_key_ranges.begin(), cop_key_ranges.end()); ::tipb::DAGRequest dag_req; - copyExecutorTreeWithLocalTableScan(dag_req, query_block.root); + + tipb::EncodeType encode_type; + if (!isUnsupportedEncodeType(query_block.output_field_types, tipb::EncodeType::TypeCHBlock)) + encode_type = tipb::EncodeType::TypeCHBlock; + else if (!isUnsupportedEncodeType(query_block.output_field_types, tipb::EncodeType::TypeChunk)) + encode_type = tipb::EncodeType::TypeChunk; + else + encode_type = tipb::EncodeType::TypeDefault; + + copyExecutorTreeWithLocalTableScan(dag_req, query_block.root, encode_type, rqst); DAGSchema schema; ColumnsWithTypeAndName columns; + std::vector is_ts_column; + std::vector source_columns; for (int i = 0; i < (int)query_block.output_field_types.size(); i++) { dag_req.add_output_offsets(i); ColumnInfo info = fieldTypeToColumnInfo(query_block.output_field_types[i]); String col_name = query_block.qb_column_prefix + "col_" + std::to_string(i); schema.push_back(std::make_pair(col_name, info)); + is_ts_column.push_back(query_block.output_field_types[i].tp() == TiDB::TypeTimestamp); + source_columns.emplace_back(col_name, getDataTypeByFieldType(query_block.output_field_types[i])); + final_project.emplace_back(col_name, ""); } executeRemoteQueryImpl(pipeline, cop_key_ranges, dag_req, schema); + + analyzer = std::make_unique(std::move(source_columns), context); + bool need_append_final_project = false; + if (encode_type == tipb::EncodeType::TypeDefault) + { + /// if the encode type is default, the timestamp column in dag response is UTC based + /// so need to cast the timezone + if (addTimeZoneCastAfterTS(is_ts_column, pipeline)) + { + for (size_t i = 0; i < final_project.size(); i++) + { + if (is_ts_column[i]) + final_project[i].first = analyzer->getCurrentInputColumns()[i].name; + } + need_append_final_project = true; + } + } + + /// For simplicity, remote subquery only record the CopBlockInputStream time, for example, + /// if the remote subquery is TS, then it's execute time is the execute time of CopBlockInputStream, + /// if the remote subquery is TS SEL, then both TS and SEL's execute time is the same as the CopBlockInputStream + recordProfileStreams(pipeline, query_block.source_name); + if (query_block.selection) + recordProfileStreams(pipeline, query_block.selection_name); + + if (need_append_final_project) + executeFinalProject(pipeline); } void DAGQueryBlockInterpreter::executeRemoteQueryImpl(Pipeline & pipeline, @@ -1187,21 +1263,22 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) if (query_block.source->tp() == tipb::ExecType::TypeJoin) { executeJoin(query_block.source->join(), pipeline, right_query); + // todo the join execution time is not accurate because only probe time is + // recorded here + recordProfileStreams(pipeline, query_block.source_name); } else { executeTS(query_block.source->tbl_scan(), pipeline); + recordProfileStreams(pipeline, query_block.source_name); } - // todo enable profile stream info - //recordProfileStreams(pipeline, dag.getTSIndex()); auto res = analyzeExpressions(); // execute selection if (res.has_where) { executeWhere(pipeline, res.before_where, res.filter_column_name); - //if (dag.hasSelection()) - //recordProfileStreams(pipeline, dag.getSelectionIndex()); + recordProfileStreams(pipeline, query_block.selection_name); } LOG_INFO(log, "execution stream size for query block(before aggregation) " << query_block.qb_column_prefix << " is " << pipeline.streams.size()); @@ -1209,7 +1286,7 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { // execute aggregation executeAggregation(pipeline, res.before_aggregation, res.aggregation_keys, res.aggregate_descriptions); - //recordProfileStreams(pipeline, dag.getAggregationIndex()); + recordProfileStreams(pipeline, query_block.aggregation_name); } if (res.before_order_and_select) { @@ -1220,7 +1297,7 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) { // execute topN executeOrder(pipeline, res.order_column_names); - //recordProfileStreams(pipeline, dag.getTopNIndex()); + recordProfileStreams(pipeline, query_block.limitOrTopN_name); } // execute projection @@ -1230,13 +1307,13 @@ void DAGQueryBlockInterpreter::executeImpl(Pipeline & pipeline) if (query_block.limitOrTopN != nullptr && query_block.limitOrTopN->tp() == tipb::TypeLimit) { executeLimit(pipeline); - //recordProfileStreams(pipeline, dag.getLimitIndex()); + recordProfileStreams(pipeline, query_block.limitOrTopN_name); } if (query_block.source->tp() == tipb::ExecType::TypeJoin) { SubqueriesForSets subquries; - subquries[query_block.qb_column_prefix + "join"] = right_query; + subquries[query_block.qb_join_subquery_alias] = right_query; subqueriesForSets.emplace_back(subquries); } } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 4a87b8ec470..698cf6fa1a3 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -104,7 +104,7 @@ class DAGQueryBlockInterpreter void getAndLockStorageWithSchemaVersion(TableID table_id, Int64 schema_version); SortDescription getSortDescription(Strings & order_column_names); AnalysisResult analyzeExpressions(); - //void recordProfileStreams(Pipeline & pipeline, Int32 index); + void recordProfileStreams(Pipeline & pipeline, const String & key); bool addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline); private: diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 22770df5ac9..292fa0f3fe2 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -24,7 +24,15 @@ DAGQuerySource::DAGQuerySource(Context & context_, DAGContext & dag_context_, co metrics(context.getTiFlashMetrics()), is_batch_cop(is_batch_cop_) { - root_query_block = std::make_shared(1, &dag_request.executors(0)); + if (dag_request.has_root_executor()) + { + root_query_block = std::make_shared(1, dag_request.root_executor()); + } + else + { + root_query_block = std::make_shared(1, dag_request.executors()); + } + root_query_block->collectAllPossibleChildrenJoinSubqueryAlias(dag_context.qb_id_to_join_alias_map); for (Int32 i : dag_request.output_offsets()) root_query_block->output_offsets.push_back(i); if (root_query_block->aggregation != nullptr) diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index f69f5e8a1a2..077e49fca4c 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -64,6 +64,8 @@ class DAGQuerySource : public IQuerySource bool isBatchCop() const { return is_batch_cop; } + DAGContext & getDAGContext() const { return dag_context; } + StreamWriterPtr writer; protected: From bcb3d7bf2e4ff2db6c3a9cb39bd64a5e0211251e Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 28 Apr 2020 20:31:38 +0800 Subject: [PATCH 47/54] fix bug --- dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp index 6218e070a40..e68018b333d 100644 --- a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp @@ -51,7 +51,8 @@ template <> DAGBlockOutputStream::DAGBlockOutputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, tipb::EncodeType encode_type_, std::vector result_field_types_, Block && header_, DAGContext & dag_context_, bool collect_execute_summary_) - : writer(writer_), + : finished(false), + writer(writer_), result_field_types(std::move(result_field_types_)), header(std::move(header_)), records_per_chunk(records_per_chunk_), From 54a982516aa42c3bcbc112322f9adb073ace47e6 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 28 Apr 2020 20:52:23 +0800 Subject: [PATCH 48/54] refine code --- contrib/tiflash-proxy | 2 +- dbms/src/DataStreams/CoprocessorBlockInputStream.h | 1 - dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h | 3 --- dbms/src/Flash/CoprocessorHandler.h | 2 -- dbms/src/Storages/IManageableStorage.h | 9 --------- dbms/src/Storages/StorageMergeTree.cpp | 1 + dbms/src/Storages/Transaction/PartitionStreams.cpp | 3 +-- dbms/src/Storages/Transaction/TMTContext.h | 4 ---- 8 files changed, 3 insertions(+), 22 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 5de44c1bbef..146b2b417dd 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 5de44c1bbefb85958e1962b4d0f14619509abc3c +Subproject commit 146b2b417dd15b9ad6ae877fca3e0469b3a91829 diff --git a/dbms/src/DataStreams/CoprocessorBlockInputStream.h b/dbms/src/DataStreams/CoprocessorBlockInputStream.h index 35267ebb60e..3a90bdc41df 100644 --- a/dbms/src/DataStreams/CoprocessorBlockInputStream.h +++ b/dbms/src/DataStreams/CoprocessorBlockInputStream.h @@ -99,7 +99,6 @@ class CoprocessorBlockInputStream : public IProfilingBlockInputStream return true; } - pingcap::coprocessor::Request req; pingcap::coprocessor::ResponseIter resp_iter; DAGSchema schema; diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h index a0c5b1e4b5e..85843c8aa2a 100644 --- a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h @@ -26,8 +26,6 @@ class DAGBlockOutputStream : public std::conditional_t result_field_types, Block && header_, DAGContext & dag_context_, bool collect_execute_summary_); -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Winconsistent-missing-override" Block getHeader() const { return header; } String getName() const { return "StreamingWriter"; } void write(const Block & block); @@ -36,7 +34,6 @@ class DAGBlockOutputStream : public std::conditional_t #pragma GCC diagnostic pop #include -#include -#include namespace DB { diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index 2befcd45e23..7223aa95d00 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -1,21 +1,12 @@ #pragma once -#include #include -#include #include -#include #include #include -#include #include -#include #include -#include #include -#include -#include - namespace TiDB { struct TableInfo; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 20bc4e37507..6f57f9cd7ad 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -33,6 +33,7 @@ #include #include + namespace DB { diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index c11e40fd18d..d2c75fd2202 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -248,8 +248,7 @@ std::tuple RegionTable::readBlockByReg Block block; { bool ok = false; - std::tie(block, ok) = readRegionBlock(table_info, columns, column_names_to_read, data_list_read, - start_ts, true, scan_filter); + std::tie(block, ok) = readRegionBlock(table_info, columns, column_names_to_read, data_list_read, start_ts, true, scan_filter); if (!ok) // TODO: Enrich exception message. throw Exception("Read region " + std::to_string(region->id()) + " of table " + std::to_string(table_info.id) + " failed", diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index 858888697bb..a631eaf1393 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -6,10 +6,6 @@ #include #include -#include - -#include - namespace DB { From 21ce2b0bbddde7332d6174903f4ae1ad5c65aaee Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 28 Apr 2020 20:59:18 +0800 Subject: [PATCH 49/54] update header --- dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 9bdbe01d9de..8cc4603d540 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include From 043870d483748fb9573423bb54f429c318afd3ec Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 5 May 2020 23:52:12 +0800 Subject: [PATCH 50/54] Fix execute details regression after merge master (#678) * refine code * fix bug * fix bug * format code --- .../Coprocessor/DAGBlockOutputStream.cpp | 238 +----------------- .../Flash/Coprocessor/DAGBlockOutputStream.h | 31 +-- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 4 +- .../Flash/Coprocessor/DAGResponseWriter.cpp | 170 +++++++++++++ .../src/Flash/Coprocessor/DAGResponseWriter.h | 44 ++++ dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 15 +- .../StreamingDAGBlockInputStream.cpp | 45 ++++ .../StreamingDAGBlockInputStream.h | 39 +++ 8 files changed, 326 insertions(+), 260 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp create mode 100644 dbms/src/Flash/Coprocessor/DAGResponseWriter.h create mode 100644 dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.cpp create mode 100644 dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.h diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp index e68018b333d..4952dcc5b73 100644 --- a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp @@ -1,243 +1,27 @@ -#include -#include -#include #include -#include namespace DB { -namespace ErrorCodes -{ -extern const int UNSUPPORTED_PARAMETER; -extern const int LOGICAL_ERROR; -} // namespace ErrorCodes - -template <> -DAGBlockOutputStream::DAGBlockOutputStream(tipb::SelectResponse * dag_response_, Int64 records_per_chunk_, - tipb::EncodeType encode_type_, std::vector result_field_types_, Block && header_, DAGContext & dag_context_, - bool collect_execute_summary_) - : dag_response(dag_response_), - result_field_types(std::move(result_field_types_)), - header(std::move(header_)), - records_per_chunk(records_per_chunk_), - encode_type(encode_type_), - current_records_num(0), - dag_context(dag_context_), - collect_execute_summary(collect_execute_summary_) -{ - previous_execute_stats.resize(dag_context.profile_streams_map.size(), std::make_tuple(0, 0, 0)); - if (encode_type == tipb::EncodeType::TypeDefault) - { - chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); - } - else if (encode_type == tipb::EncodeType::TypeChunk) - { - chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); - } - else if (encode_type == tipb::EncodeType::TypeCHBlock) - { - chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); - records_per_chunk = -1; - } - else - { - throw Exception("Only Default and Arrow encode type is supported in DAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); - } - dag_response->set_encode_type(encode_type); -} +DAGBlockOutputStream::DAGBlockOutputStream(tipb::SelectResponse * dag_response, Int64 records_per_chunk, tipb::EncodeType encode_type, + std::vector result_field_types, Block && header_, DAGContext & dag_context, bool collect_execute_summary, + bool return_executor_id) + : header(std::move(header_)), + response_writer(dag_response, nullptr, records_per_chunk, encode_type, result_field_types, dag_context, collect_execute_summary, + return_executor_id) +{} -template <> -DAGBlockOutputStream::DAGBlockOutputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, - tipb::EncodeType encode_type_, std::vector result_field_types_, Block && header_, DAGContext & dag_context_, - bool collect_execute_summary_) - : finished(false), - writer(writer_), - result_field_types(std::move(result_field_types_)), - header(std::move(header_)), - records_per_chunk(records_per_chunk_), - encode_type(encode_type_), - current_records_num(0), - dag_context(dag_context_), - collect_execute_summary(collect_execute_summary_) -{ - previous_execute_stats.resize(dag_context.profile_streams_map.size(), std::make_tuple(0, 0, 0)); - if (encode_type == tipb::EncodeType::TypeDefault) - { - chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); - } - else if (encode_type == tipb::EncodeType::TypeChunk) - { - chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); - } - else if (encode_type == tipb::EncodeType::TypeCHBlock) - { - chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); - records_per_chunk = -1; - } - else - { - throw Exception( - "Only Default and Arrow encode type is supported in StreamingDAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); - } - children.push_back(input_); -} - -template -void DAGBlockOutputStream::writePrefix() +void DAGBlockOutputStream::writePrefix() { //something to do here? } -template <> -void DAGBlockOutputStream::readPrefix() -{ - children.back()->readPrefix(); -} - -template <> -void DAGBlockOutputStream::encodeChunkToDAGResponse() -{ - auto dag_chunk = dag_response->add_chunks(); - dag_chunk->set_rows_data(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); - current_records_num = 0; -} - -template <> -Block DAGBlockOutputStream::readImpl() -{ - if (finished) - return {}; - while (Block block = children.back()->read()) - { - if (!block) - { - finished = true; - return {}; - } - write(block); - } - return {}; -} +void DAGBlockOutputStream::write(const Block & block) { response_writer.write(block); } -template -void DAGBlockOutputStream::addExecuteSummaries(tipb::SelectResponse * response) -{ - if (!collect_execute_summary) - return; - // add ExecutorExecutionSummary info - for (auto & p : dag_context.profile_streams_map) - { - auto * executeSummary = response->add_execution_summaries(); - UInt64 time_processed_ns = 0; - UInt64 num_produced_rows = 0; - UInt64 num_iterations = 0; - for (auto & streamPtr : p.second.input_streams) - { - if (auto * p_stream = dynamic_cast(streamPtr.get())) - { - time_processed_ns = std::max(time_processed_ns, p_stream->getProfileInfo().execution_time); - num_produced_rows += p_stream->getProfileInfo().rows; - num_iterations += p_stream->getProfileInfo().blocks; - } - } - for (auto & join_alias : dag_context.qb_id_to_join_alias_map[p.second.qb_id]) - { - if (dag_context.profile_streams_map_for_join_build_side.find(join_alias) - != dag_context.profile_streams_map_for_join_build_side.end()) - { - UInt64 process_time_for_build = 0; - for (auto & join_stream : dag_context.profile_streams_map_for_join_build_side[join_alias]) - { - if (auto * p_stream = dynamic_cast(join_stream.get())) - process_time_for_build = std::max(process_time_for_build, p_stream->getProfileInfo().execution_time); - } - time_processed_ns += process_time_for_build; - } - } - executeSummary->set_time_processed_ns(time_processed_ns); - executeSummary->set_num_produced_rows(num_produced_rows); - executeSummary->set_num_iterations(num_iterations); - if constexpr (streaming) - executeSummary->set_executor_id(p.first); - } -} - -template <> -void DAGBlockOutputStream::encodeChunkToDAGResponse() -{ - ::coprocessor::BatchResponse resp; - - tipb::SelectResponse stream_dag_response; - stream_dag_response.set_encode_type(encode_type); - auto dag_chunk = stream_dag_response.add_chunks(); - dag_chunk->set_rows_data(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); - current_records_num = 0; - addExecuteSummaries(&stream_dag_response); - std::string dag_data; - stream_dag_response.SerializeToString(&dag_data); - resp.set_data(dag_data); - - writer->write(resp); -} - -template -void DAGBlockOutputStream::writeSuffix() +void DAGBlockOutputStream::writeSuffix() { // todo error handle - if (current_records_num > 0) - { - encodeChunkToDAGResponse(); - } - if constexpr (!streaming) - { - addExecuteSummaries(dag_response); - } + response_writer.finishWrite(); } -template -void DAGBlockOutputStream::write(const Block & block) -{ - if (block.columns() != result_field_types.size()) - throw Exception("Output column size mismatch with field type size", ErrorCodes::LOGICAL_ERROR); - if (records_per_chunk == -1) - { - current_records_num = 0; - if (block.rows() > 0) - { - chunk_codec_stream->encode(block, 0, block.rows()); - encodeChunkToDAGResponse(); - } - } - else - { - size_t rows = block.rows(); - for (size_t row_index = 0; row_index < rows;) - { - if (current_records_num >= records_per_chunk) - { - encodeChunkToDAGResponse(); - } - const size_t upper = std::min(row_index + (records_per_chunk - current_records_num), rows); - chunk_codec_stream->encode(block, row_index, upper); - current_records_num += (upper - row_index); - row_index = upper; - } - } -} - -template <> -void DAGBlockOutputStream::readSuffix() -{ - // todo error handle - if (current_records_num > 0) - { - encodeChunkToDAGResponse(); - } - children.back()->readSuffix(); -} - - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h index 85843c8aa2a..43d1bddbc38 100644 --- a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h @@ -8,49 +8,30 @@ #include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" +#include #include + #pragma GCC diagnostic pop namespace DB { /// Serializes the stream of blocks in TiDB DAG response format. -template -class DAGBlockOutputStream : public std::conditional_t +class DAGBlockOutputStream : public IBlockOutputStream { public: DAGBlockOutputStream(tipb::SelectResponse * response_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, - std::vector result_field_types, Block && header_, DAGContext & dag_context_, bool collect_execute_summary_); - - DAGBlockOutputStream(BlockInputStreamPtr input_, StreamWriterPtr writer, Int64 records_per_chunk_, tipb::EncodeType encodeType_, - std::vector result_field_types, Block && header_, DAGContext & dag_context_, bool collect_execute_summary_); - + std::vector result_field_types, Block && header_, DAGContext & dag_context_, bool collect_execute_summary_, + bool return_executor_id); Block getHeader() const { return header; } - String getName() const { return "StreamingWriter"; } void write(const Block & block); void writePrefix(); void writeSuffix(); - Block readImpl(); - void readPrefix(); - void readSuffix(); - - void encodeChunkToDAGResponse(); - void addExecuteSummaries(tipb::SelectResponse * dag_response); private: - bool finished; - tipb::SelectResponse * dag_response; - StreamWriterPtr writer; - std::vector result_field_types; Block header; - Int64 records_per_chunk; - tipb::EncodeType encode_type; - std::unique_ptr chunk_codec_stream; - Int64 current_records_num; - DAGContext & dag_context; - bool collect_execute_summary; - std::vector> previous_execute_stats; + DAGResponseWriter response_writer; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index f4ff279fbbe..5625fe97db3 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -71,8 +71,8 @@ try { bool collect_exec_summary = dag_request.has_collect_execution_summaries() && dag_request.collect_execution_summaries(); BlockOutputStreamPtr dag_output_stream - = std::make_shared>(dag_response, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), - dag.getResultFieldTypes(), streams.in->getHeader(), dag_context, collect_exec_summary); + = std::make_shared(dag_response, context.getSettings().dag_records_per_chunk, dag.getEncodeType(), + dag.getResultFieldTypes(), streams.in->getHeader(), dag_context, collect_exec_summary, dag_request.has_root_executor()); copyData(*streams.in, *dag_output_stream); } else diff --git a/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp new file mode 100644 index 00000000000..d71f53b7a92 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNSUPPORTED_PARAMETER; +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes + +template +DAGResponseWriter::DAGResponseWriter(tipb::SelectResponse * dag_response_, StreamWriterPtr writer_, Int64 records_per_chunk_, + tipb::EncodeType encode_type_, std::vector result_field_types_, DAGContext & dag_context_, + bool collect_execute_summary_, bool return_executor_id_) + : dag_response(dag_response_), + writer(std::move(writer_)), + result_field_types(std::move(result_field_types_)), + records_per_chunk(records_per_chunk_), + encode_type(encode_type_), + current_records_num(0), + dag_context(dag_context_), + collect_execute_summary(collect_execute_summary_), + return_executor_id(return_executor_id_) +{ + previous_execute_stats.resize(dag_context.profile_streams_map.size(), std::make_tuple(0, 0, 0)); + if (encode_type == tipb::EncodeType::TypeDefault) + { + chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); + } + else if (encode_type == tipb::EncodeType::TypeChunk) + { + chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); + } + else if (encode_type == tipb::EncodeType::TypeCHBlock) + { + chunk_codec_stream = std::make_unique()->newCodecStream(result_field_types); + records_per_chunk = -1; + } + else + { + throw Exception("Only Default and Arrow encode type is supported in DAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); + } + if (dag_response) + dag_response->set_encode_type(encode_type); +} + +template +void DAGResponseWriter::addExecuteSummaries(tipb::SelectResponse * response) +{ + if (!collect_execute_summary) + return; + // add ExecutorExecutionSummary info + for (auto & p : dag_context.profile_streams_map) + { + auto * executeSummary = response->add_execution_summaries(); + UInt64 time_processed_ns = 0; + UInt64 num_produced_rows = 0; + UInt64 num_iterations = 0; + for (auto & streamPtr : p.second.input_streams) + { + if (auto * p_stream = dynamic_cast(streamPtr.get())) + { + time_processed_ns = std::max(time_processed_ns, p_stream->getProfileInfo().execution_time); + num_produced_rows += p_stream->getProfileInfo().rows; + num_iterations += p_stream->getProfileInfo().blocks; + } + } + for (auto & join_alias : dag_context.qb_id_to_join_alias_map[p.second.qb_id]) + { + if (dag_context.profile_streams_map_for_join_build_side.find(join_alias) + != dag_context.profile_streams_map_for_join_build_side.end()) + { + UInt64 process_time_for_build = 0; + for (auto & join_stream : dag_context.profile_streams_map_for_join_build_side[join_alias]) + { + if (auto * p_stream = dynamic_cast(join_stream.get())) + process_time_for_build = std::max(process_time_for_build, p_stream->getProfileInfo().execution_time); + } + time_processed_ns += process_time_for_build; + } + } + executeSummary->set_time_processed_ns(time_processed_ns); + executeSummary->set_num_produced_rows(num_produced_rows); + executeSummary->set_num_iterations(num_iterations); + if (return_executor_id) + executeSummary->set_executor_id(p.first); + } +} + +template <> +void DAGResponseWriter::encodeChunkToDAGResponse() +{ + auto dag_chunk = dag_response->add_chunks(); + dag_chunk->set_rows_data(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); + current_records_num = 0; +} + +template <> +void DAGResponseWriter::encodeChunkToDAGResponse() +{ + ::coprocessor::BatchResponse resp; + + tipb::SelectResponse stream_dag_response; + stream_dag_response.set_encode_type(encode_type); + auto dag_chunk = stream_dag_response.add_chunks(); + dag_chunk->set_rows_data(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); + current_records_num = 0; + addExecuteSummaries(&stream_dag_response); + std::string dag_data; + stream_dag_response.SerializeToString(&dag_data); + resp.set_data(dag_data); + + writer->write(resp); +} + +template +void DAGResponseWriter::finishWrite() +{ + if (current_records_num > 0) + { + encodeChunkToDAGResponse(); + } + if constexpr (!streaming) + { + addExecuteSummaries(dag_response); + } +} + +template +void DAGResponseWriter::write(const Block & block) +{ + if (block.columns() != result_field_types.size()) + throw Exception("Output column size mismatch with field type size", ErrorCodes::LOGICAL_ERROR); + if (records_per_chunk == -1) + { + current_records_num = 0; + if (block.rows() > 0) + { + chunk_codec_stream->encode(block, 0, block.rows()); + encodeChunkToDAGResponse(); + } + } + else + { + size_t rows = block.rows(); + for (size_t row_index = 0; row_index < rows;) + { + if (current_records_num >= records_per_chunk) + { + encodeChunkToDAGResponse(); + } + const size_t upper = std::min(row_index + (records_per_chunk - current_records_num), rows); + chunk_codec_stream->encode(block, row_index, upper); + current_records_num += (upper - row_index); + row_index = upper; + } + } +} + +template class DAGResponseWriter; +template class DAGResponseWriter; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGResponseWriter.h b/dbms/src/Flash/Coprocessor/DAGResponseWriter.h new file mode 100644 index 00000000000..30022aa368f --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGResponseWriter.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include +#include +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +namespace DB +{ + +/// Serializes the stream of blocks in TiDB DAG response format. +template +class DAGResponseWriter +{ +public: + DAGResponseWriter(tipb::SelectResponse * response_, StreamWriterPtr writer_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, + std::vector result_field_types, DAGContext & dag_context_, bool collect_execute_summary_, + bool return_executor_id_); + + void write(const Block & block); + void finishWrite(); + void encodeChunkToDAGResponse(); + void addExecuteSummaries(tipb::SelectResponse * dag_response); + +private: + tipb::SelectResponse * dag_response; + StreamWriterPtr writer; + std::vector result_field_types; + Int64 records_per_chunk; + tipb::EncodeType encode_type; + std::unique_ptr chunk_codec_stream; + Int64 current_records_num; + DAGContext & dag_context; + bool collect_execute_summary; + bool return_executor_id; + std::vector> previous_execute_stats; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 9b6be8b62fe..e9245e1c4de 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -2,15 +2,16 @@ #include #include #include +#include #include #include #include -#include +#include #include #include -#include -#include #include + + namespace DB { @@ -82,10 +83,12 @@ BlockIO InterpreterDAG::execute() if (dag.writer->writer != nullptr) { - bool collect_exec_summary = dag.getDAGRequest().has_collect_execution_summaries() && dag.getDAGRequest().collect_execution_summaries(); + bool collect_exec_summary + = dag.getDAGRequest().has_collect_execution_summaries() && dag.getDAGRequest().collect_execution_summaries(); for (auto & stream : pipeline.streams) - stream = std::make_shared>(stream, dag.writer, context.getSettings().dag_records_per_chunk, - dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader(), dag.getDAGContext(), collect_exec_summary); + stream = std::make_shared(stream, dag.writer, context.getSettings().dag_records_per_chunk, + dag.getEncodeType(), dag.getResultFieldTypes(), stream->getHeader(), dag.getDAGContext(), collect_exec_summary, + dag.getDAGRequest().has_root_executor()); } executeUnion(pipeline); if (!subqueriesForSets.empty()) diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.cpp new file mode 100644 index 00000000000..71710bec07c --- /dev/null +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include + +namespace DB +{ + +StreamingDAGBlockInputStream::StreamingDAGBlockInputStream(BlockInputStreamPtr input, StreamWriterPtr writer, Int64 records_per_chunk, + tipb::EncodeType encode_type, std::vector && result_field_types, Block && header_, DAGContext & dag_context, + bool collect_execute_summary, bool return_executor_id) + : finished(false), + header(std::move(header_)), + response_writer( + nullptr, writer, records_per_chunk, encode_type, result_field_types, dag_context, collect_execute_summary, return_executor_id) +{ + children.push_back(input); +} + +void StreamingDAGBlockInputStream::readPrefix() { children.back()->readPrefix(); } + +void StreamingDAGBlockInputStream::readSuffix() +{ + // todo error handle + response_writer.finishWrite(); + children.back()->readSuffix(); +} + +Block StreamingDAGBlockInputStream::readImpl() +{ + if (finished) + return {}; + while (Block block = children.back()->read()) + { + if (!block) + { + finished = true; + return {}; + } + response_writer.write(block); + } + return {}; +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.h b/dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.h new file mode 100644 index 00000000000..b893fd8cdcd --- /dev/null +++ b/dbms/src/Flash/Coprocessor/StreamingDAGBlockInputStream.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include + +#pragma GCC diagnostic pop + +namespace DB +{ + +/// Serializes the stream of blocks in TiDB DAG response format. +/// TODO: May consider using some parallelism. +/// TODO: Consider using output schema in DAG request, do some conversion or checking between DAG schema and block schema. +class StreamingDAGBlockInputStream : public IProfilingBlockInputStream +{ +public: + StreamingDAGBlockInputStream(BlockInputStreamPtr input_, StreamWriterPtr writer_, Int64 records_per_chunk_, + tipb::EncodeType encodeType_, std::vector && result_field_types, Block && header_, DAGContext & dag_context_, + bool collect_execute_summary_, bool return_executor_id); + + Block getHeader() const override { return header; } + Block readImpl() override; + String getName() const override { return "StreamingWriter"; } + void readPrefix() override; + void readSuffix() override; + +private: + bool finished; + Block header; + DAGResponseWriter response_writer; +}; + +} // namespace DB From 05ed93126e0c393098c6ca3127987eadbf110a5f Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 21 May 2020 16:03:55 +0800 Subject: [PATCH 51/54] update kvproto --- contrib/kvproto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kvproto b/contrib/kvproto index 9bb769c3cee..a5af800ca2e 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 9bb769c3ceebbe3e66b6d4108d7e7f8106eb700f +Subproject commit a5af800ca2efb06e3c1dad87294fa6fd21712c0c From e14ef7101cf0599c1a07d23bb3cfe84954949d90 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 21 May 2020 18:17:04 +0800 Subject: [PATCH 52/54] fmt code --- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 4 ++-- .../src/Flash/Coprocessor/DAGQueryBlockInterpreter.h | 12 ------------ dbms/src/Flash/Coprocessor/DAGQuerySource.h | 3 +-- 3 files changed, 3 insertions(+), 16 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index 8632ef5244d..2231a8a34c5 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -3,8 +3,8 @@ #include #pragma GCC diagnostic pop -#include "DAGQueryBlock.h" -#include "DAGUtils.h" +#include +#include namespace DB { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 698cf6fa1a3..117bed0a737 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -27,18 +27,6 @@ class Context; class Region; using RegionPtr = std::shared_ptr; -//struct RegionInfo -//{ -// RegionInfo(RegionID region_id_, UInt64 region_version_, UInt64 region_conf_version_, -// const std::vector> & key_ranges_) -// : region_id(region_id_), region_version(region_version_), region_conf_version(region_conf_version_), key_ranges(key_ranges_) -// {} -// const RegionID region_id; -// const UInt64 region_version; -// const UInt64 region_conf_version; -// const std::vector> & key_ranges; -//}; - struct Pipeline { BlockInputStreams streams; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index 077e49fca4c..9e935fc7798 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -7,13 +7,12 @@ #include #include +#include #include #include #include #include -#include "DAGDriver.h" -#include "DAGQueryBlock.h" namespace DB { From 5844bb4410f50714263fea6ab7cfd6f207b69a67 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 17 Jun 2020 18:11:54 +0800 Subject: [PATCH 53/54] update client-c --- contrib/client-c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/client-c b/contrib/client-c index 05553c96920..34f9ca27f78 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 05553c969206f016b87acad5164c3535346f07c8 +Subproject commit 34f9ca27f78fa45bbe236352a4bdb1ca44b34194 From 4a10242cdf44ea0b07e319248f3981968c16714d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 18 Jun 2020 17:35:12 +0800 Subject: [PATCH 54/54] update tipb --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index 59279b98966..4fad48b4c8c 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 59279b98966726a78ec4af59f8bd593b576d602b +Subproject commit 4fad48b4c8c3e17805a520d6c3a7655077ffca46