diff --git a/.gitmodules b/.gitmodules index c2cf933d715..f2ae2c89935 100644 --- a/.gitmodules +++ b/.gitmodules @@ -37,6 +37,10 @@ [submodule "contrib/kvproto"] path = contrib/kvproto url = https://github.com/pingcap/kvproto.git +[submodule "contrib/tipb"] + path = contrib/tipb + url = https://github.com/pingcap/tipb.git + branch = master [submodule "contrib/client-c"] path = contrib/client-c url = git@github.com:tikv/client-c.git diff --git a/CMakeLists.txt b/CMakeLists.txt index b87cb423d47..f41f8373150 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -272,6 +272,7 @@ include (cmake/find_capnp.cmake) include (cmake/find_llvm.cmake) include (cmake/find_grpc.cmake) include (cmake/find_kvproto.cmake) +include (cmake/find_tipb.cmake) include (cmake/find_contrib_lib.cmake) diff --git a/cmake/find_tipb.cmake b/cmake/find_tipb.cmake new file mode 100644 index 00000000000..7e8bfff5092 --- /dev/null +++ b/cmake/find_tipb.cmake @@ -0,0 +1,10 @@ + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/tipb/cpp/tipb/select.pb.h") + if (EXISTS "${ClickHouse_SOURCE_DIR}/contrib/tipb/proto/select.proto") + message (FATAL_ERROR "tipb cpp files in contrib/tipb is missing. Try go to contrib/tipb, and run ./generate_cpp.sh") + else() + message (FATAL_ERROR "tipb submodule in contrib/tipb is missing. Try run 'git submodule update --init --recursive', and go to contrib/tipb, and run ./generate_cpp.sh") + endif() +endif () + +message(STATUS "Using tipb: ${ClickHouse_SOURCE_DIR}/contrib/tipb/cpp") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 6b87cce2701..13c2f861b5b 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -1,5 +1,6 @@ add_subdirectory (kvproto/cpp) add_subdirectory (client-c) +add_subdirectory (tipb/cpp) if (NOT MSVC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast") diff --git a/contrib/tipb b/contrib/tipb new file mode 160000 index 00000000000..b2d318af5e8 --- /dev/null +++ b/contrib/tipb @@ -0,0 +1 @@ +Subproject commit b2d318af5e8af28f54a2c6422bc18631f65a8506 diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 6fe83e0ce8b..018cf6ae0d6 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -60,6 +60,7 @@ add_headers_and_sources(dbms src/Storages/Page) add_headers_and_sources(dbms src/Raft) add_headers_and_sources(dbms src/TiDB) add_headers_and_sources(dbms src/Client) +add_headers_only(dbms src/Flash/Coprocessor) add_headers_only(dbms src/Server) list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) @@ -149,8 +150,10 @@ target_link_libraries (dbms clickhouse_parsers clickhouse_common_config clickhouse_common_io + flash_service kvproto kv_client + tipb ${Protobuf_LIBRARIES} gRPC::grpc++_unsecure ${CURL_LIBRARIES} diff --git a/dbms/src/CMakeLists.txt b/dbms/src/CMakeLists.txt index 7ee6bf08cc6..4e8ddf750d9 100644 --- a/dbms/src/CMakeLists.txt +++ b/dbms/src/CMakeLists.txt @@ -13,4 +13,5 @@ add_subdirectory (AggregateFunctions) add_subdirectory (Server) add_subdirectory (Client) add_subdirectory (TableFunctions) -add_subdirectory (Analyzers) \ No newline at end of file +add_subdirectory (Analyzers) +add_subdirectory (Flash) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 3971e051c65..a9dde87606c 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -395,6 +395,7 @@ namespace ErrorCodes extern const int SCHEMA_SYNC_ERROR = 10003; extern const int SCHEMA_VERSION_ERROR = 10004; extern const int DDL_ERROR = 10005; + extern const int COP_BAD_DAG_REQUEST = 10006; } } diff --git a/dbms/src/Common/MyTime.cpp b/dbms/src/Common/MyTime.cpp index bef7a8d608a..e8b12450e3b 100644 --- a/dbms/src/Common/MyTime.cpp +++ b/dbms/src/Common/MyTime.cpp @@ -473,4 +473,15 @@ void convertTimeZone(UInt64 from_time, UInt64 & to_time, const DateLUTImpl & tim to_time = to_my_time.toPackedUInt(); } +void convertTimeZoneByOffset(UInt64 from_time, UInt64 & to_time, Int64 offset, const DateLUTImpl & time_zone) +{ + MyDateTime from_my_time(from_time); + time_t epoch = time_zone.makeDateTime( + from_my_time.year, from_my_time.month, from_my_time.day, from_my_time.hour, from_my_time.minute, from_my_time.second); + epoch += offset; + MyDateTime to_my_time(time_zone.toYear(epoch), time_zone.toMonth(epoch), time_zone.toDayOfMonth(epoch), + time_zone.toHour(epoch), time_zone.toMinute(epoch), time_zone.toSecond(epoch), from_my_time.micro_second); + to_time = to_my_time.toPackedUInt(); +} + } // namespace DB diff --git a/dbms/src/Common/MyTime.h b/dbms/src/Common/MyTime.h index bcca1324f8a..f8e12f3a825 100644 --- a/dbms/src/Common/MyTime.h +++ b/dbms/src/Common/MyTime.h @@ -65,4 +65,6 @@ Field parseMyDateTime(const String & str); void convertTimeZone(UInt64 from_time, UInt64 & to_time, const DateLUTImpl & time_zone_from, const DateLUTImpl & time_zone_to); +void convertTimeZoneByOffset(UInt64 from_time, UInt64 & to_time, Int64 offset, const DateLUTImpl & time_zone); + } // namespace DB diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 85eb7d499d1..acb722ba567 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -29,6 +29,8 @@ #define DEFAULT_MAX_READ_TSO 0xFFFFFFFFFFFFFFFF #define DEFAULT_UNSPECIFIED_SCHEMA_VERSION -1 +#define DEFAULT_DAG_RECORDS_PER_CHUNK 64L + /** Which blocks by default read the data (by number of rows). * Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query. */ diff --git a/dbms/src/DataStreams/BlockStreamProfileInfo.h b/dbms/src/DataStreams/BlockStreamProfileInfo.h index 578b0ed5b26..e546dd20e2a 100644 --- a/dbms/src/DataStreams/BlockStreamProfileInfo.h +++ b/dbms/src/DataStreams/BlockStreamProfileInfo.h @@ -29,6 +29,11 @@ struct BlockStreamProfileInfo size_t rows = 0; size_t blocks = 0; size_t bytes = 0; + // execution time is the total time spent on current stream and all its children streams + // note that it is different from total_stopwatch.elapsed(), which includes not only the + // time spent on current stream and all its children streams, but also the time of its + // parent streams + UInt64 execution_time = 0; using BlockStreamProfileInfos = std::vector; @@ -45,6 +50,8 @@ struct BlockStreamProfileInfo void update(Block & block); + void updateExecutionTime(UInt64 time) { execution_time += time; } + /// Binary serialization and deserialization of main fields. /// Writes only main fields i.e. fields that required by internal transmission protocol. void read(ReadBuffer & in); diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 09eeff2225c..09bf609833f 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -42,6 +42,8 @@ Block IProfilingBlockInputStream::read() if (isCancelledOrThrowIfKilled()) return res; + auto start_time = info.total_stopwatch.elapsed(); + if (!checkTimeLimit()) limit_exceeded_need_break = true; @@ -83,6 +85,7 @@ Block IProfilingBlockInputStream::read() } #endif + info.updateExecutionTime(info.total_stopwatch.elapsed() - start_time); return res; } diff --git a/dbms/src/DataTypes/DataTypeMyDateTime.cpp b/dbms/src/DataTypes/DataTypeMyDateTime.cpp index 752931dc3fa..5eb6aba8cff 100644 --- a/dbms/src/DataTypes/DataTypeMyDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeMyDateTime.cpp @@ -20,7 +20,7 @@ DataTypeMyDateTime::DataTypeMyDateTime(int fraction_) { fraction = fraction_; if (fraction < 0 || fraction > 6) - throw Exception("fraction must >= 0 and < 6", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("fraction must >= 0 and <= 6", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } void DataTypeMyDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index 7511e90c65d..e1411d6f071 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -30,48 +31,50 @@ void dbgFuncSleep(Context &, const ASTs & args, DBGInvoker::Printer output) DBGInvoker::DBGInvoker() { - regFunc("echo", dbgFuncEcho); + regSchemalessFunc("echo", dbgFuncEcho); // TODO: remove this, use sleep in bash script - regFunc("sleep", dbgFuncSleep); - - regFunc("mock_tidb_table", MockTiDBTable::dbgFuncMockTiDBTable); - regFunc("mock_tidb_db", MockTiDBTable::dbgFuncMockTiDBDB); - regFunc("mock_tidb_partition", MockTiDBTable::dbgFuncMockTiDBPartition); - regFunc("drop_tidb_partition", MockTiDBTable::dbgFuncDropTiDBPartition); - regFunc("drop_tidb_table", MockTiDBTable::dbgFuncDropTiDBTable); - regFunc("drop_tidb_db", MockTiDBTable::dbgFuncDropTiDBDB); - regFunc("add_column_to_tidb_table", MockTiDBTable::dbgFuncAddColumnToTiDBTable); - regFunc("drop_column_from_tidb_table", MockTiDBTable::dbgFuncDropColumnFromTiDBTable); - regFunc("modify_column_in_tidb_table", MockTiDBTable::dbgFuncModifyColumnInTiDBTable); - regFunc("rename_column_in_tidb_table", MockTiDBTable::dbgFuncRenameColumnInTiDBTable); - regFunc("rename_tidb_table", MockTiDBTable::dbgFuncRenameTiDBTable); - regFunc("truncate_tidb_table", MockTiDBTable::dbgFuncTruncateTiDBTable); - - regFunc("set_flush_threshold", dbgFuncSetFlushThreshold); - - regFunc("raft_insert_row", dbgFuncRaftInsertRow); - regFunc("raft_insert_row_full", dbgFuncRaftInsertRowFull); - regFunc("raft_insert_rows", dbgFuncRaftInsertRows); - regFunc("raft_update_rows", dbgFuncRaftUpdateRows); - regFunc("raft_delete_rows", dbgFuncRaftDelRows); - regFunc("raft_delete_row", dbgFuncRaftDeleteRow); - - regFunc("put_region", dbgFuncPutRegion); - regFunc("region_snapshot", dbgFuncRegionSnapshot); - regFunc("region_snapshot_data", dbgFuncRegionSnapshotWithData); - - regFunc("try_flush", dbgFuncTryFlush); - regFunc("try_flush_region", dbgFuncTryFlushRegion); - - regFunc("dump_all_region", dbgFuncDumpAllRegion); - regFunc("dump_all_mock_region", dbgFuncDumpAllMockRegion); - - regFunc("enable_schema_sync_service", dbgFuncEnableSchemaSyncService); - regFunc("refresh_schemas", dbgFuncRefreshSchemas); - regFunc("reset_schemas", dbgFuncResetSchemas); - - regFunc("dump_region_table", ClusterManage::dumpRegionTable); - regFunc("find_region_by_range", ClusterManage::findRegionByRange); + regSchemalessFunc("sleep", dbgFuncSleep); + + regSchemalessFunc("mock_tidb_table", MockTiDBTable::dbgFuncMockTiDBTable); + regSchemalessFunc("mock_tidb_db", MockTiDBTable::dbgFuncMockTiDBDB); + regSchemalessFunc("mock_tidb_partition", MockTiDBTable::dbgFuncMockTiDBPartition); + regSchemalessFunc("drop_tidb_table", MockTiDBTable::dbgFuncDropTiDBTable); + regSchemalessFunc("drop_tidb_db", MockTiDBTable::dbgFuncDropTiDBDB); + regSchemalessFunc("drop_tidb_partition", MockTiDBTable::dbgFuncDropTiDBPartition); + regSchemalessFunc("add_column_to_tidb_table", MockTiDBTable::dbgFuncAddColumnToTiDBTable); + regSchemalessFunc("drop_column_from_tidb_table", MockTiDBTable::dbgFuncDropColumnFromTiDBTable); + regSchemalessFunc("modify_column_in_tidb_table", MockTiDBTable::dbgFuncModifyColumnInTiDBTable); + regSchemalessFunc("rename_column_in_tidb_table", MockTiDBTable::dbgFuncRenameColumnInTiDBTable); + regSchemalessFunc("rename_tidb_table", MockTiDBTable::dbgFuncRenameTiDBTable); + regSchemalessFunc("truncate_tidb_table", MockTiDBTable::dbgFuncTruncateTiDBTable); + + regSchemalessFunc("set_flush_threshold", dbgFuncSetFlushThreshold); + + regSchemalessFunc("raft_insert_row", dbgFuncRaftInsertRow); + regSchemalessFunc("raft_insert_row_full", dbgFuncRaftInsertRowFull); + regSchemalessFunc("raft_insert_rows", dbgFuncRaftInsertRows); + regSchemalessFunc("raft_update_rows", dbgFuncRaftUpdateRows); + regSchemalessFunc("raft_delete_rows", dbgFuncRaftDelRows); + regSchemalessFunc("raft_delete_row", dbgFuncRaftDeleteRow); + + regSchemalessFunc("put_region", dbgFuncPutRegion); + regSchemalessFunc("region_snapshot", dbgFuncRegionSnapshot); + regSchemalessFunc("region_snapshot_data", dbgFuncRegionSnapshotWithData); + + regSchemalessFunc("try_flush", dbgFuncTryFlush); + regSchemalessFunc("try_flush_region", dbgFuncTryFlushRegion); + + regSchemalessFunc("dump_all_region", dbgFuncDumpAllRegion); + regSchemalessFunc("dump_all_mock_region", dbgFuncDumpAllMockRegion); + regSchemalessFunc("dump_region_table", ClusterManage::dumpRegionTable); + regSchemalessFunc("find_region_by_range", ClusterManage::findRegionByRange); + + regSchemalessFunc("enable_schema_sync_service", dbgFuncEnableSchemaSyncService); + regSchemalessFunc("refresh_schemas", dbgFuncRefreshSchemas); + regSchemalessFunc("reset_schemas", dbgFuncResetSchemas); + + regSchemafulFunc("dag", dbgFuncDAG); + regSchemafulFunc("mock_dag", dbgFuncMockDAG); } void replaceSubstr(std::string & str, const std::string & target, const std::string & replacement) @@ -103,10 +106,25 @@ BlockInputStreamPtr DBGInvoker::invoke(Context & context, const std::string & or name = ori_name.substr(prefix_not_print_res.size(), ori_name.size() - prefix_not_print_res.size()); } - auto it = funcs.find(name); - if (it == funcs.end()) - throw Exception("DBG function not found", ErrorCodes::BAD_ARGUMENTS); + BlockInputStreamPtr res; + auto it_schemaless = schemaless_funcs.find(name); + if (it_schemaless != schemaless_funcs.end()) + res = invokeSchemaless(context, name, it_schemaless->second, args); + else + { + auto it_schemaful = schemaful_funcs.find(name); + if (it_schemaful != schemaful_funcs.end()) + res = invokeSchemaful(context, name, it_schemaful->second, args); + else + throw Exception("DBG function not found", ErrorCodes::BAD_ARGUMENTS); + } + + return print_res ? res : std::shared_ptr(); +} +BlockInputStreamPtr DBGInvoker::invokeSchemaless( + Context & context, const std::string & name, const SchemalessDBGFunc & func, const ASTs & args) +{ std::stringstream col_name; col_name << name << "("; for (size_t i = 0; i < args.size(); ++i) @@ -119,9 +137,14 @@ BlockInputStreamPtr DBGInvoker::invoke(Context & context, const std::string & or std::shared_ptr res = std::make_shared(col_name.str()); Printer printer = [&](const std::string & s) { res->append(s); }; - (it->second)(context, args, printer); + func(context, args, printer); - return print_res ? res : std::shared_ptr(); + return res; +} + +BlockInputStreamPtr DBGInvoker::invokeSchemaful(Context & context, const std::string &, const SchemafulDBGFunc & func, const ASTs & args) +{ + return func(context, args); } } // namespace DB diff --git a/dbms/src/Debug/DBGInvoker.h b/dbms/src/Debug/DBGInvoker.h index 71e8487f1fa..95b2449fd9b 100644 --- a/dbms/src/Debug/DBGInvoker.h +++ b/dbms/src/Debug/DBGInvoker.h @@ -25,16 +25,21 @@ class DBGInvoker { public: using Printer = std::function; - using DBGFunc = std::function; + using SchemalessDBGFunc = std::function; + using SchemafulDBGFunc = std::function; DBGInvoker(); - void regFunc(const std::string & name, DBGFunc func) { funcs[name] = func; } + void regSchemalessFunc(const std::string & name, SchemalessDBGFunc func) { schemaless_funcs[name] = func; } + void regSchemafulFunc(const std::string & name, SchemafulDBGFunc func) { schemaful_funcs[name] = func; } BlockInputStreamPtr invoke(Context & context, const std::string & ori_name, const ASTs & args); + BlockInputStreamPtr invokeSchemaless(Context & context, const std::string & name, const SchemalessDBGFunc & func, const ASTs & args); + BlockInputStreamPtr invokeSchemaful(Context & context, const std::string & name, const SchemafulDBGFunc & func, const ASTs & args); private: - std::unordered_map funcs; + std::unordered_map schemaless_funcs; + std::unordered_map schemaful_funcs; }; } // namespace DB diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index cc2233f6a9d..e6d59fc6c20 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -10,10 +9,10 @@ #include #include #include -#include #include #include #include +#include namespace DB { @@ -114,90 +113,6 @@ void MockTiDB::dropTable(Context & context, const String & database_name, const version_diff[version] = diff; } -template -bool tryGetDecimalType(const IDataType * nested_type, ColumnInfo & column_info) -{ - using TypeDec = DataTypeDecimal; - if (checkDataType(nested_type)) - { - auto decimal_type = checkAndGetDataType(nested_type); - column_info.flen = decimal_type->getPrec(); - column_info.decimal = decimal_type->getScale(); - column_info.tp = TiDB::TypeNewDecimal; - return true; - } - return false; -} - -ColumnInfo getColumnInfoFromColumn(const NameAndTypePair & column, ColumnID id, const Field & default_value) -{ - ColumnInfo column_info; - column_info.id = id; - column_info.name = column.name; - const IDataType * nested_type = column.type.get(); - if (!column.type->isNullable()) - { - column_info.setNotNullFlag(); - } - else - { - auto nullable_type = checkAndGetDataType(nested_type); - nested_type = nullable_type->getNestedType().get(); - } - if (nested_type->isUnsignedInteger()) - { - column_info.setUnsignedFlag(); - } - else if (tryGetDecimalType(nested_type, column_info)) - { - } - else if (tryGetDecimalType(nested_type, column_info)) - { - } - else if (tryGetDecimalType(nested_type, column_info)) - { - } - else if (tryGetDecimalType(nested_type, column_info)) - { - } - -#ifdef M -#error "Please undefine macro M first." -#endif -#define M(tt, v, cf, ct, w) \ - if (checkDataType(nested_type)) \ - column_info.tp = TiDB::Type##tt; \ - else - COLUMN_TYPES(M) -#undef M - if (checkDataType(nested_type)) - column_info.tp = TiDB::TypeTiny; - else if (checkDataType(nested_type)) - column_info.tp = TiDB::TypeShort; - else if (checkDataType(nested_type)) - column_info.tp = TiDB::TypeLong; - - if (auto type = checkAndGetDataType(nested_type)) - column_info.decimal = type->getFraction(); - // UInt64 is hijacked by the macro expansion, we check it again. - if (checkDataType(nested_type)) - column_info.tp = TiDB::TypeLongLong; - - // Default value, currently we only support int. - if (!default_value.isNull()) - { - // convert any type to string , this is TiDB's style. - - column_info.origin_default_value = applyVisitor(FieldVisitorToString(), default_value); - } - else - { - column_info.setNoDefaultValueFlag(); - } - - return column_info; -} - DatabaseID MockTiDB::newDataBase(const String & database_name) { DatabaseID schema_id = 0; @@ -218,7 +133,8 @@ DatabaseID MockTiDB::newDataBase(const String & database_name) return schema_id; } -TableID MockTiDB::newTable(const String & database_name, const String & table_name, const ColumnsDescription & columns, Timestamp tso) +TableID MockTiDB::newTable(const String & database_name, const String & table_name, + const ColumnsDescription & columns, Timestamp tso, const String & handle_pk_name) { std::lock_guard lock(tables_mutex); @@ -238,14 +154,21 @@ TableID MockTiDB::newTable(const String & database_name, const String & table_na table_info.db_name = database_name; table_info.id = table_id_allocator++; table_info.name = table_name; + table_info.pk_is_handle = false; - int i = 0; + int i = 1; for (auto & column : columns.getAllPhysical()) { - table_info.columns.emplace_back(getColumnInfoFromColumn(column, i++, Field())); + table_info.columns.emplace_back(reverseGetColumnInfo(column, i++, Field())); + if (handle_pk_name == column.name) + { + if (!column.type->isInteger() && !column.type->isUnsignedInteger()) + throw Exception("MockTiDB pk column must be integer or unsigned integer type", ErrorCodes::LOGICAL_ERROR); + table_info.columns.back().setPriKeyFlag(); + table_info.pk_is_handle = true; + } } - table_info.pk_is_handle = false; table_info.comment = "Mocked."; table_info.update_timestamp = tso; @@ -337,7 +260,7 @@ void MockTiDB::addColumnToTable( != columns.end()) throw Exception("Column " + column.name + " already exists in TiDB table " + qualified_name, ErrorCodes::LOGICAL_ERROR); - ColumnInfo column_info = getColumnInfoFromColumn(column, table->allocColumnID(), default_value); + ColumnInfo column_info = reverseGetColumnInfo(column, table->allocColumnID(), default_value); columns.emplace_back(column_info); version++; @@ -384,7 +307,7 @@ void MockTiDB::modifyColumnInTable(const String & database_name, const String & if (it == columns.end()) throw Exception("Column " + column.name + " does not exist in TiDB table " + qualified_name, ErrorCodes::LOGICAL_ERROR); - ColumnInfo column_info = getColumnInfoFromColumn(column, 0, Field()); + ColumnInfo column_info = reverseGetColumnInfo(column, 0, Field()); if (it->hasUnsignedFlag() != column_info.hasUnsignedFlag()) throw Exception("Modify column " + column.name + " UNSIGNED flag is not allowed", ErrorCodes::LOGICAL_ERROR); if (it->tp == column_info.tp && it->hasNotNullFlag() == column_info.hasNotNullFlag()) diff --git a/dbms/src/Debug/MockTiDB.h b/dbms/src/Debug/MockTiDB.h index ed3d30a1929..20afa87144a 100644 --- a/dbms/src/Debug/MockTiDB.h +++ b/dbms/src/Debug/MockTiDB.h @@ -55,7 +55,8 @@ class MockTiDB : public ext::singleton using TablePtr = std::shared_ptr; public: - TableID newTable(const String & database_name, const String & table_name, const ColumnsDescription & columns, Timestamp tso); + TableID newTable(const String & database_name, const String & table_name, + const ColumnsDescription & columns, Timestamp tso, const String & handle_pk_name); DatabaseID newDataBase(const String & database_name); diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp new file mode 100644 index 00000000000..962ad335a1c --- /dev/null +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -0,0 +1,656 @@ +#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 BAD_ARGUMENTS; +extern const int LOGICA_ERROR; +} // namespace ErrorCodes + +using TiDB::DatumFlat; +using TiDB::TableInfo; + +using DAGColumnInfo = std::pair; +using DAGSchema = std::vector; +using SchemaFetcher = std::function; +std::tuple compileQuery( + Context & context, const String & query, SchemaFetcher schema_fetcher, Timestamp start_ts, + Int64 tz_offset, const String & tz_name); +tipb::SelectResponse executeDAGRequest( + Context & context, const tipb::DAGRequest & dag_request, RegionID region_id, UInt64 region_version, + UInt64 region_conf_version, std::vector> & key_ranges); +BlockInputStreamPtr outputDAGResponse(Context & context, const DAGSchema & schema, const tipb::SelectResponse & dag_response); + +BlockInputStreamPtr dbgFuncDAG(Context & context, const ASTs & args) +{ + if (args.size() < 1 || args.size() > 4) + throw Exception("Args not matched, should be: query[, region-id, tz_offset, tz_name]", ErrorCodes::BAD_ARGUMENTS); + + String query = safeGet(typeid_cast(*args[0]).value); + RegionID region_id = InvalidRegionID; + if (args.size() >= 2) + region_id = safeGet(typeid_cast(*args[1]).value); + Int64 tz_offset = 0; + String tz_name = ""; + if (args.size() >= 3) + tz_offset = get(typeid_cast(*args[2]).value); + if (args.size() >= 4) + tz_name = safeGet(typeid_cast(*args[3]).value); + Timestamp start_ts = context.getTMTContext().getPDClient()->getTS(); + + auto [table_id, schema, dag_request] = compileQuery( + context, query, + [&](const String & database_name, const String & table_name) { + auto storage = context.getTable(database_name, table_name); + auto mmt = std::dynamic_pointer_cast(storage); + if (!mmt || mmt->getData().merging_params.mode != MergeTreeData::MergingParams::Txn) + throw Exception("Not TMT", ErrorCodes::BAD_ARGUMENTS); + return mmt->getTableInfo(); + }, + start_ts, tz_offset, tz_name); + + RegionPtr region; + if (region_id == InvalidRegionID) + { + auto regions = context.getTMTContext().getRegionTable().getRegionsByTable(table_id); + if (regions.empty()) + throw Exception("No region for table", ErrorCodes::BAD_ARGUMENTS); + region = context.getTMTContext().getRegionTable().getRegionsByTable(table_id).front().second; + } + else + { + region = context.getTMTContext().getKVStore()->getRegion(region_id); + if (!region) + throw Exception("No such region", ErrorCodes::BAD_ARGUMENTS); + } + + auto handle_range = region->getHandleRangeByTable(table_id); + std::vector> key_ranges; + DecodedTiKVKey start_key = RecordKVFormat::genRawKey(table_id, handle_range.first.handle_id); + DecodedTiKVKey end_key = RecordKVFormat::genRawKey(table_id, handle_range.second.handle_id); + key_ranges.emplace_back(std::make_pair(std::move(start_key), std::move(end_key))); + tipb::SelectResponse dag_response = executeDAGRequest(context, dag_request, region->id(), region->version(), + region->confVer(), key_ranges); + + return outputDAGResponse(context, schema, dag_response); +} + +BlockInputStreamPtr dbgFuncMockDAG(Context & context, const ASTs & args) +{ + if (args.size() < 2 || args.size() > 5) + throw Exception("Args not matched, should be: query, region-id[, start-ts, tz_offset, tz_name]", ErrorCodes::BAD_ARGUMENTS); + + String query = safeGet(typeid_cast(*args[0]).value); + RegionID region_id = safeGet(typeid_cast(*args[1]).value); + Timestamp start_ts = DEFAULT_MAX_READ_TSO; + if (args.size() >= 3) + start_ts = safeGet(typeid_cast(*args[2]).value); + if (start_ts == 0) + start_ts = context.getTMTContext().getPDClient()->getTS(); + Int64 tz_offset = 0; + String tz_name = ""; + if (args.size() >= 3) + tz_offset = safeGet(typeid_cast(*args[2]).value); + if (args.size() >= 4) + tz_name = safeGet(typeid_cast(*args[3]).value); + + auto [table_id, schema, dag_request] = compileQuery( + context, query, + [&](const String & database_name, const String & table_name) { + return MockTiDB::instance().getTableByName(database_name, table_name)->table_info; + }, + start_ts, tz_offset, tz_name); + std::ignore = table_id; + + RegionPtr region = context.getTMTContext().getKVStore()->getRegion(region_id); + auto handle_range = region->getHandleRangeByTable(table_id); + std::vector> key_ranges; + DecodedTiKVKey start_key = RecordKVFormat::genRawKey(table_id, handle_range.first.handle_id); + DecodedTiKVKey end_key = RecordKVFormat::genRawKey(table_id, handle_range.second.handle_id); + key_ranges.emplace_back(std::make_pair(std::move(start_key), std::move(end_key))); + tipb::SelectResponse dag_response = executeDAGRequest(context, dag_request, region_id, region->version(), + region->confVer(), key_ranges); + + return outputDAGResponse(context, schema, dag_response); +} + +const String VOID_COL_NAME = "_void"; + +struct ExecutorCtx +{ + tipb::Executor * input; + DAGSchema output; + std::unordered_map> col_ref_map; +}; + +void compileExpr(const DAGSchema & input, ASTPtr ast, tipb::Expr * expr, std::unordered_set & referred_columns, + std::unordered_map> & col_ref_map) +{ + if (ASTIdentifier * id = typeid_cast(ast.get())) + { + auto ft = std::find_if(input.begin(), input.end(), [&](const auto & field) { return field.first == id->getColumnName(); }); + if (ft == input.end()) + throw Exception("No such column " + id->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + expr->set_tp(tipb::ColumnRef); + *(expr->mutable_field_type()) = columnInfoToFieldType((*ft).second); + + referred_columns.emplace((*ft).first); + if (col_ref_map.find((*ft).first) == col_ref_map.end()) + col_ref_map[(*ft).first] = {}; + col_ref_map[(*ft).first].push_back(expr); + } + else if (ASTFunction * func = typeid_cast(ast.get())) + { + // TODO: Support agg functions. + for (const auto & child_ast : func->arguments->children) + { + tipb::Expr * child = expr->add_children(); + compileExpr(input, child_ast, child, referred_columns, col_ref_map); + } + + String func_name_lowercase = Poco::toLower(func->name); + // TODO: Support more functions. + // TODO: Support type inference. + if (func_name_lowercase == "equals") + { + expr->set_sig(tipb::ScalarFuncSig::EQInt); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else if (func_name_lowercase == "and") + { + expr->set_sig(tipb::ScalarFuncSig::LogicalAnd); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else if (func_name_lowercase == "or") + { + expr->set_sig(tipb::ScalarFuncSig::LogicalOr); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else if (func_name_lowercase == "greater") + { + expr->set_sig(tipb::ScalarFuncSig::GTInt); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else if (func_name_lowercase == "greaterorequals") + { + expr->set_sig(tipb::ScalarFuncSig::GEInt); + auto *ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else if (func_name_lowercase == "less") + { + expr->set_sig(tipb::ScalarFuncSig::LTInt); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else if (func_name_lowercase == "lessorequals") + { + expr->set_sig(tipb::ScalarFuncSig::LEInt); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned); + } + else + { + throw Exception("Unsupported function: " + func_name_lowercase, ErrorCodes::LOGICAL_ERROR); + } + expr->set_tp(tipb::ExprType::ScalarFunc); + } + else if (ASTLiteral * lit = typeid_cast(ast.get())) + { + std::stringstream ss; + switch (lit->value.getType()) + { + case Field::Types::Which::Null: + expr->set_tp(tipb::Null); + // Null literal expr doesn't need value. + break; + case Field::Types::Which::UInt64: + expr->set_tp(tipb::Uint64); + encodeDAGUInt64(lit->value.get(), ss); + break; + case Field::Types::Which::Int64: + expr->set_tp(tipb::Int64); + encodeDAGInt64(lit->value.get(), ss); + break; + case Field::Types::Which::Float64: + expr->set_tp(tipb::Float64); + encodeDAGFloat64(lit->value.get(), ss); + break; + case Field::Types::Which::Decimal32: + case Field::Types::Which::Decimal64: + case Field::Types::Which::Decimal128: + case Field::Types::Which::Decimal256: + expr->set_tp(tipb::MysqlDecimal); + encodeDAGDecimal(lit->value, ss); + break; + case Field::Types::Which::String: + expr->set_tp(tipb::String); + // TODO: Align with TiDB. + encodeDAGBytes(lit->value.get(), ss); + break; + default: + throw Exception(String("Unsupported literal type: ") + lit->value.getTypeName(), ErrorCodes::LOGICAL_ERROR); + } + expr->set_val(ss.str()); + } + else + { + throw Exception("Unsupported expression " + ast->getColumnName(), ErrorCodes::LOGICAL_ERROR); + } +} + +void compileFilter(const DAGSchema & input, ASTPtr ast, tipb::Selection * filter, std::unordered_set & referred_columns, + std::unordered_map> & col_ref_map) +{ + if (auto * func = typeid_cast(ast.get())) + { + if (func->name == "and") + { + for (auto & child : func->arguments->children) + { + compileFilter(input, child, filter, referred_columns, col_ref_map); + } + return; + } + } + tipb::Expr * cond = filter->add_conditions(); + compileExpr(input, ast, cond, referred_columns, col_ref_map); +} + +std::tuple compileQuery( + Context & context, const String & query, SchemaFetcher schema_fetcher, + Timestamp start_ts, Int64 tz_offset, const String & tz_name) +{ + DAGSchema schema; + tipb::DAGRequest dag_request; + dag_request.set_time_zone_name(tz_name); + dag_request.set_time_zone_offset(tz_offset); + + dag_request.set_start_ts(start_ts); + + ParserSelectQuery parser; + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "from DAG compiler", 0); + ASTSelectQuery & ast_query = typeid_cast(*ast); + + /// Get table metadata. + TableInfo table_info; + { + String database_name, table_name; + auto query_database = ast_query.database(); + auto query_table = ast_query.table(); + if (query_database) + database_name = typeid_cast(*query_database).name; + if (query_table) + table_name = typeid_cast(*query_table).name; + if (!query_table) + { + database_name = "system"; + table_name = "one"; + } + else if (!query_database) + { + database_name = context.getCurrentDatabase(); + } + + table_info = schema_fetcher(database_name, table_name); + } + + std::unordered_map executor_ctx_map; + std::unordered_set referred_columns; + tipb::TableScan * ts = nullptr; + tipb::Executor * last_executor = nullptr; + + /// Table scan. + { + tipb::Executor * ts_exec = dag_request.add_executors(); + ts_exec->set_tp(tipb::ExecType::TypeTableScan); + ts = ts_exec->mutable_tbl_scan(); + ts->set_table_id(table_info.id); + DAGSchema ts_output; + for (const auto & column_info : table_info.columns) + { + ColumnInfo ci; + ci.tp = column_info.tp; + ci.flag = column_info.flag; + ci.flen = column_info.flen; + ci.decimal = column_info.decimal; + ci.elems = column_info.elems; + // a hack to test timestamp type in mock test + if (column_info.tp == TiDB::TypeDatetime && ci.decimal == 5) + ci.tp = TiDB::TypeTimestamp; + ts_output.emplace_back(std::make_pair(column_info.name, std::move(ci))); + } + executor_ctx_map.emplace(ts_exec, ExecutorCtx{nullptr, std::move(ts_output), std::unordered_map>{}}); + last_executor = ts_exec; + } + + /// Filter. + if (ast_query.where_expression) + { + tipb::Executor * filter_exec = dag_request.add_executors(); + filter_exec->set_tp(tipb::ExecType::TypeSelection); + tipb::Selection * filter = filter_exec->mutable_selection(); + std::unordered_map> col_ref_map; + compileFilter(executor_ctx_map[last_executor].output, ast_query.where_expression, filter, referred_columns, col_ref_map); + executor_ctx_map.emplace(filter_exec, ExecutorCtx{last_executor, executor_ctx_map[last_executor].output, std::move(col_ref_map)}); + last_executor = filter_exec; + } + + /// TopN. + if (ast_query.order_expression_list && ast_query.limit_length) + { + tipb::Executor * topn_exec = dag_request.add_executors(); + topn_exec->set_tp(tipb::ExecType::TypeTopN); + tipb::TopN * topn = topn_exec->mutable_topn(); + std::unordered_map> col_ref_map; + for (const auto & child : ast_query.order_expression_list->children) + { + ASTOrderByElement * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = topn->add_order_by(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + compileExpr(executor_ctx_map[last_executor].output, elem->children[0], expr, referred_columns, col_ref_map); + } + auto limit = safeGet(typeid_cast(*ast_query.limit_length).value); + topn->set_limit(limit); + executor_ctx_map.emplace(topn_exec, ExecutorCtx{last_executor, executor_ctx_map[last_executor].output, std::move(col_ref_map)}); + last_executor = topn_exec; + } + else if (ast_query.limit_length) + { + tipb::Executor * limit_exec = dag_request.add_executors(); + limit_exec->set_tp(tipb::ExecType::TypeLimit); + tipb::Limit * limit = limit_exec->mutable_limit(); + auto limit_length = safeGet(typeid_cast(*ast_query.limit_length).value); + limit->set_limit(limit_length); + executor_ctx_map.emplace( + limit_exec, ExecutorCtx{last_executor, executor_ctx_map[last_executor].output, std::unordered_map>{}}); + last_executor = limit_exec; + } + + /// Column pruner. + std::function column_pruner = [&](ExecutorCtx & executor_ctx) { + if (!executor_ctx.input) + { + executor_ctx.output.erase(std::remove_if(executor_ctx.output.begin(), executor_ctx.output.end(), + [&](const auto & field) { return referred_columns.count(field.first) == 0; }), + executor_ctx.output.end()); + + for (const auto & info : executor_ctx.output) + { + tipb::ColumnInfo * ci = ts->add_columns(); + ci->set_column_id(table_info.getColumnID(info.first)); + ci->set_tp(info.second.tp); + ci->set_flag(info.second.flag); + ci->set_columnlen(info.second.flen); + ci->set_decimal(info.second.decimal); + if (!info.second.elems.empty()) + { + for (auto & pair : info.second.elems) + { + ci->add_elems(pair.first); + } + } + } + + return; + } + column_pruner(executor_ctx_map[executor_ctx.input]); + const auto & last_output = executor_ctx_map[executor_ctx.input].output; + for (const auto & pair : executor_ctx.col_ref_map) + { + auto iter = std::find_if(last_output.begin(), last_output.end(), [&](const auto & field) { return field.first == pair.first; }); + if (iter == last_output.end()) + throw Exception("Column not found when pruning: " + pair.first, ErrorCodes::LOGICAL_ERROR); + std::stringstream ss; + encodeDAGInt64(iter - last_output.begin(), ss); + auto s_val = ss.str(); + for (auto * expr : pair.second) + expr->set_val(s_val); + } + executor_ctx.output = last_output; + }; + + /// Aggregation finalize. + { + bool has_gby = ast_query.group_expression_list != nullptr; + bool has_agg_func = false; + for (const auto & child : ast_query.select_expression_list->children) + { + const ASTFunction * func = typeid_cast(child.get()); + if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + { + has_agg_func = true; + break; + } + } + + if (has_gby || has_agg_func) + { + if (last_executor->has_limit() || last_executor->has_topn()) + throw Exception("Limit/TopN and Agg cannot co-exist.", ErrorCodes::LOGICAL_ERROR); + + tipb::Executor * agg_exec = dag_request.add_executors(); + agg_exec->set_tp(tipb::ExecType::TypeAggregation); + tipb::Aggregation * agg = agg_exec->mutable_aggregation(); + std::unordered_map> col_ref_map; + for (const auto & expr : ast_query.select_expression_list->children) + { + const ASTFunction * func = typeid_cast(expr.get()); + if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) + throw Exception("Only agg function is allowed in select for a query with aggregation", ErrorCodes::LOGICAL_ERROR); + + tipb::Expr * agg_func = agg->add_agg_func(); + + for (const auto & arg : func->arguments->children) + { + tipb::Expr * arg_expr = agg_func->add_children(); + compileExpr(executor_ctx_map[last_executor].output, arg, arg_expr, referred_columns, col_ref_map); + } + + if (func->name == "count") + { + agg_func->set_tp(tipb::Count); + auto ft = agg_func->mutable_field_type(); + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull); + } + else if (func->name == "max") + { + agg_func->set_tp(tipb::Max); + if (agg_func->children_size() != 1) + throw Exception("udaf max only accept 1 argument"); + auto ft = agg_func->mutable_field_type(); + ft->set_tp(agg_func->children(0).field_type().tp()); + } + // TODO: Other agg func. + else + { + throw Exception("Unsupported agg function " + func->name, ErrorCodes::LOGICAL_ERROR); + } + + schema.emplace_back(std::make_pair(func->getColumnName(), fieldTypeToColumnInfo(agg_func->field_type()))); + } + + if (has_gby) + { + for (const auto & child : ast_query.group_expression_list->children) + { + tipb::Expr * gby = agg->add_group_by(); + compileExpr(executor_ctx_map[last_executor].output, child, gby, referred_columns, col_ref_map); + schema.emplace_back(std::make_pair(child->getColumnName(), fieldTypeToColumnInfo(gby->field_type()))); + } + } + + executor_ctx_map.emplace(agg_exec, ExecutorCtx{last_executor, DAGSchema{}, std::move(col_ref_map)}); + last_executor = agg_exec; + + column_pruner(executor_ctx_map[last_executor]); + } + } + + /// Non-aggregation finalize. + if (!last_executor->has_aggregation()) + { + std::vector final_output; + for (const auto & expr : ast_query.select_expression_list->children) + { + if (ASTIdentifier * id = typeid_cast(expr.get())) + { + referred_columns.emplace(id->getColumnName()); + final_output.emplace_back(id->getColumnName()); + } + else if (typeid_cast(expr.get())) + { + const auto & last_output = executor_ctx_map[last_executor].output; + for (const auto & field : last_output) + { + referred_columns.emplace(field.first); + final_output.push_back(field.first); + } + } + else + { + throw Exception("Unsupported expression type in select", ErrorCodes::LOGICAL_ERROR); + } + } + + column_pruner(executor_ctx_map[last_executor]); + + const auto & last_output = executor_ctx_map[last_executor].output; + + // For testing VOID column, ignore any other select expressions, unless table contains it. + if (std::find(final_output.begin(), final_output.end(), VOID_COL_NAME) != final_output.end() + && std::find_if( + last_output.begin(), last_output.end(), [&](const auto & last_field) { return last_field.first == VOID_COL_NAME; }) + == last_output.end()) + { + dag_request.add_output_offsets(0); + + // Set column ID to -1 to trigger `void` column in DAG processing. + tipb::ColumnInfo * ci = ts->add_columns(); + ci->set_column_id(-1); + + // Set column name to VOID and tp to Nullable(UInt64), + // as chunk decoding doesn't do strict field type check so Nullable(UInt64) should be enough. + ColumnInfo ti_ci; + ti_ci.name = VOID_COL_NAME; + ti_ci.tp = TiDB::TypeLongLong; + ti_ci.setNotNullFlag(); + schema.emplace_back(DAGColumnInfo{VOID_COL_NAME, std::move(ti_ci)}); + } + else + { + for (const auto & field : final_output) + { + auto iter = std::find_if( + last_output.begin(), last_output.end(), [&](const auto & last_field) { return last_field.first == field; }); + if (iter == last_output.end()) + throw Exception("Column not found after pruning: " + field, ErrorCodes::LOGICAL_ERROR); + dag_request.add_output_offsets(iter - last_output.begin()); + schema.push_back(*iter); + } + } + } + + return std::make_tuple(table_info.id, std::move(schema), std::move(dag_request)); +} + +tipb::SelectResponse executeDAGRequest( + Context & context, const tipb::DAGRequest & dag_request, RegionID region_id, UInt64 region_version, + UInt64 region_conf_version, std::vector> & key_ranges) +{ + static Logger * log = &Logger::get("MockDAG"); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); + context.setSetting("dag_planner", "optree"); + tipb::SelectResponse dag_response; + DAGDriver driver(context, dag_request, region_id, region_version, region_conf_version, std::move(key_ranges), dag_response, true); + driver.execute(); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle DAG request done"); + return dag_response; +} + +BlockInputStreamPtr outputDAGResponse(Context &, const DAGSchema & schema, const tipb::SelectResponse & dag_response) +{ + if (dag_response.has_error()) + throw Exception(dag_response.error().msg(), dag_response.error().code()); + + BlocksList blocks; + for (const auto & chunk : dag_response.chunks()) + { + std::vector> rows; + std::vector curr_row; + const std::string & data = chunk.rows_data(); + size_t cursor = 0; + while (cursor < data.size()) + { + curr_row.push_back(DecodeDatum(cursor, data)); + if (curr_row.size() == schema.size()) + { + rows.emplace_back(std::move(curr_row)); + curr_row.clear(); + } + } + + ColumnsWithTypeAndName columns; + for (auto & field : schema) + { + const auto & name = field.first; + auto data_type = getDataTypeByColumnInfo(field.second); + ColumnWithTypeAndName col(data_type, name); + col.column->assumeMutable()->reserve(rows.size()); + columns.emplace_back(std::move(col)); + } + for (const auto & row : rows) + { + for (size_t i = 0; i < row.size(); i++) + { + const Field & field = row[i]; + columns[i].column->assumeMutable()->insert(DatumFlat(field, schema[i].second.tp).field()); + } + } + + blocks.emplace_back(Block(columns)); + } + + return std::make_shared(std::move(blocks)); +} + +} // namespace DB diff --git a/dbms/src/Debug/dbgFuncCoprocessor.h b/dbms/src/Debug/dbgFuncCoprocessor.h new file mode 100644 index 00000000000..eb8cc989fd5 --- /dev/null +++ b/dbms/src/Debug/dbgFuncCoprocessor.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Context; + +// Coprocessor debug tools + +// Run a DAG request using given query that will be compiled to DAG request, with the given (optional) region ID. +// Usage: +// ./storages-client.sh "DBGInvoke dag(query[, region_id])" +BlockInputStreamPtr dbgFuncDAG(Context & context, const ASTs & args); + +// Mock a DAG request using given query that will be compiled (with the metadata from MockTiDB) to DAG request, with the given region ID and (optional) start ts. +// Usage: +// ./storages-client.sh "DBGInvoke mock_dag(query, region_id[, start_ts])" +BlockInputStreamPtr dbgFuncMockDAG(Context & context, const ASTs & args); + +} // namespace DB diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp index 879c2c7663e..41badcadb75 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp @@ -25,13 +25,17 @@ extern const int LOGICAL_ERROR; void MockTiDBTable::dbgFuncMockTiDBTable(Context & context, const ASTs & args, DBGInvoker::Printer output) { - if (args.size() != 3) - throw Exception("Args not matched, should be: database-name, table-name, schema-string", ErrorCodes::BAD_ARGUMENTS); + if (args.size() != 3 && args.size() != 4) + throw Exception("Args not matched, should be: database-name, table-name, schema-string [, handle_pk_name]", ErrorCodes::BAD_ARGUMENTS); const String & database_name = typeid_cast(*args[0]).name; const String & table_name = typeid_cast(*args[1]).name; auto schema_str = safeGet(typeid_cast(*args[2]).value); + String handle_pk_name = ""; + if (args.size() == 4) + handle_pk_name = safeGet(typeid_cast(*args[3]).value); + ASTPtr columns_ast; ParserColumnDeclarationList schema_parser; Tokens tokens(schema_str.data(), schema_str.data() + schema_str.length()); @@ -43,7 +47,7 @@ void MockTiDBTable::dbgFuncMockTiDBTable(Context & context, const ASTs & args, D = InterpreterCreateQuery::getColumnsDescription(typeid_cast(*columns_ast), context); auto tso = context.getTMTContext().getPDClient()->getTS(); - TableID table_id = MockTiDB::instance().newTable(database_name, table_name, columns, tso); + TableID table_id = MockTiDB::instance().newTable(database_name, table_name, columns, tso, handle_pk_name); std::stringstream ss; ss << "mock table #" << table_id; diff --git a/dbms/src/Debug/dbgFuncRegion.cpp b/dbms/src/Debug/dbgFuncRegion.cpp index c3d9eb0f4ba..1e8131454d0 100644 --- a/dbms/src/Debug/dbgFuncRegion.cpp +++ b/dbms/src/Debug/dbgFuncRegion.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -140,7 +139,9 @@ void dbgFuncRegionSnapshotWithData(Context & context, const ASTs & args, DBGInvo } TiKVKey key = RecordKVFormat::genKey(table_id, handle_id); - TiKVValue value = RecordKVFormat::EncodeRow(table->table_info, fields); + std::stringstream ss; + RegionBench::encodeRow(table->table_info, fields, ss); + TiKVValue value(ss.str()); UInt64 commit_ts = tso; UInt64 prewrite_ts = tso; TiKVValue commit_value = del ? RecordKVFormat::encodeWriteCfValue(Region::DelFlag, prewrite_ts) diff --git a/dbms/src/Debug/dbgTools.cpp b/dbms/src/Debug/dbgTools.cpp index 921caa234e7..77c9e0c50da 100644 --- a/dbms/src/Debug/dbgTools.cpp +++ b/dbms/src/Debug/dbgTools.cpp @@ -23,6 +23,9 @@ extern const int LOGICAL_ERROR; namespace RegionBench { +using TiDB::ColumnInfo; +using TiDB::TableInfo; + RegionPtr createRegion(TableID table_id, RegionID region_id, const HandleID & start, const HandleID & end) { enginepb::SnapshotRequest request; @@ -121,7 +124,130 @@ void addRequestsToRaftCmd(enginepb::CommandRequest * cmd, RegionID region_id, co } } -bool isDateTimeType(TiDB::TP tp) { return tp == TiDB::TypeTimestamp || tp == TiDB::TypeDate || tp == TiDB::TypeDatetime; } +template +T convertNumber(const Field & field) +{ + switch (field.getType()) + { + case Field::Types::Int64: + return static_cast(field.get()); + case Field::Types::UInt64: + return static_cast(field.get()); + case Field::Types::Float64: + return static_cast(field.get()); + case Field::Types::Decimal32: + return static_cast(field.get>()); + case Field::Types::Decimal64: + return static_cast(field.get>()); + case Field::Types::Decimal128: + return static_cast(field.get>()); + case Field::Types::Decimal256: + return static_cast(field.get>()); + default: + throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to number", ErrorCodes::LOGICAL_ERROR); + } +} + +Field convertDecimal(const ColumnInfo & column_info, const Field & field) +{ + switch (field.getType()) + { + case Field::Types::Int64: + return column_info.getDecimalValue(std::to_string(field.get())); + case Field::Types::UInt64: + return column_info.getDecimalValue(std::to_string(field.get())); + case Field::Types::Float64: + return column_info.getDecimalValue(std::to_string(field.get())); + case Field::Types::Decimal32: + return column_info.getDecimalValue(field.get().toString(column_info.decimal)); + case Field::Types::Decimal64: + return column_info.getDecimalValue(field.get().toString(column_info.decimal)); + case Field::Types::Decimal128: + return column_info.getDecimalValue(field.get().toString(column_info.decimal)); + case Field::Types::Decimal256: + return column_info.getDecimalValue(field.get().toString(column_info.decimal)); + default: + throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to number", ErrorCodes::LOGICAL_ERROR); + } +} + +Field convertEnum(const ColumnInfo & column_info, const Field & field) +{ + switch (field.getType()) + { + case Field::Types::Int64: + case Field::Types::UInt64: + return convertNumber(field); + case Field::Types::String: + return static_cast(column_info.getEnumIndex(field.get())); + default: + throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to Enum", ErrorCodes::LOGICAL_ERROR); + } +} + +Field convertField(const ColumnInfo & column_info, const Field & field) +{ + if (field.isNull()) + return field; + + switch (column_info.tp) + { + case TiDB::TypeTiny: + case TiDB::TypeShort: + case TiDB::TypeLong: + case TiDB::TypeLongLong: + case TiDB::TypeInt24: + case TiDB::TypeBit: + if (column_info.hasUnsignedFlag()) + return convertNumber(field); + else + return convertNumber(field); + case TiDB::TypeFloat: + case TiDB::TypeDouble: + return convertNumber(field); + case TiDB::TypeDate: + case TiDB::TypeDatetime: + case TiDB::TypeTimestamp: + return parseMyDateTime(field.safeGet()); + case TiDB::TypeVarchar: + case TiDB::TypeTinyBlob: + case TiDB::TypeMediumBlob: + case TiDB::TypeLongBlob: + case TiDB::TypeBlob: + case TiDB::TypeVarString: + case TiDB::TypeString: + return field; + case TiDB::TypeEnum: + return convertEnum(column_info, field); + case TiDB::TypeNull: + return Field(); + case TiDB::TypeDecimal: + case TiDB::TypeNewDecimal: + return convertDecimal(column_info, field); + case TiDB::TypeTime: + throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to Time", ErrorCodes::LOGICAL_ERROR); + case TiDB::TypeYear: + throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to Year", ErrorCodes::LOGICAL_ERROR); + case TiDB::TypeSet: + throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to Set", ErrorCodes::LOGICAL_ERROR); + default: + return Field(); + } +} + +void encodeRow(const TiDB::TableInfo & table_info, const std::vector & fields, std::stringstream & ss) +{ + if (table_info.columns.size() != fields.size() + table_info.pk_is_handle) + throw Exception("Encoding row has different sizes between columns and values", ErrorCodes::LOGICAL_ERROR); + for (size_t i = 0; i < fields.size(); i++) + { + const TiDB::ColumnInfo & column_info = table_info.columns[i]; + EncodeDatum(Field(column_info.id), TiDB::CodecFlagInt, ss); + Field field = convertField(column_info, fields[i]); + TiDB::DatumBumpy datum = TiDB::DatumBumpy(field, column_info.tp); + EncodeDatum(datum.field(), column_info.getCodecFlag(), ss); + } +} void insert(const TiDB::TableInfo & table_info, RegionID region_id, HandleID handle_id, ASTs::const_iterator begin, ASTs::const_iterator end, Context & context, const std::optional> & tso_del) @@ -132,14 +258,10 @@ void insert(const TiDB::TableInfo & table_info, RegionID region_id, HandleID han while ((it = begin++) != end) { auto field = typeid_cast((*it).get())->value; - if (isDateTimeType(table_info.columns[idx].tp)) - { - field = parseMyDateTime(field.safeGet()); - } fields.emplace_back(field); idx++; } - if (fields.size() != table_info.columns.size()) + if (fields.size() + table_info.pk_is_handle != table_info.columns.size()) throw Exception("Number of insert values and columns do not match.", ErrorCodes::LOGICAL_ERROR); TMTContext & tmt = context.getTMTContext(); @@ -151,7 +273,9 @@ void insert(const TiDB::TableInfo & table_info, RegionID region_id, HandleID han TableID table_id = RecordKVFormat::getTableId(range->rawKeys().first); TiKVKey key = RecordKVFormat::genKey(table_id, handle_id); - TiKVValue value = RecordKVFormat::EncodeRow(table_info, fields); + std::stringstream ss; + encodeRow(table_info, fields, ss); + TiKVValue value(ss.str()); UInt64 prewrite_ts = pd_client->getTS(); UInt64 commit_ts = pd_client->getTS(); diff --git a/dbms/src/Debug/dbgTools.h b/dbms/src/Debug/dbgTools.h index 5ed25a4eb02..70372a2c765 100644 --- a/dbms/src/Debug/dbgTools.h +++ b/dbms/src/Debug/dbgTools.h @@ -23,6 +23,8 @@ RegionPtr createRegion(TableID table_id, RegionID region_id, const HandleID & st Regions createRegions(TableID table_id, size_t region_num, size_t key_num_each_region, HandleID handle_begin, RegionID new_region_id_begin); +void encodeRow(const TiDB::TableInfo & table_info, const std::vector & fields, std::stringstream & ss); + void insert(const TiDB::TableInfo & table_info, RegionID region_id, HandleID handle_id, ASTs::const_iterator begin, ASTs::const_iterator end, Context & context, const std::optional> & tso_del = {}); diff --git a/dbms/src/Flash/BatchCommandsHandler.cpp b/dbms/src/Flash/BatchCommandsHandler.cpp new file mode 100644 index 00000000000..f90c92848e2 --- /dev/null +++ b/dbms/src/Flash/BatchCommandsHandler.cpp @@ -0,0 +1,97 @@ +#include +#include + +namespace DB +{ + +BatchCommandsHandler::BatchCommandsHandler(BatchCommandsContext & batch_commands_context_, const tikvpb::BatchCommandsRequest & request_, + tikvpb::BatchCommandsResponse & response_) + : batch_commands_context(batch_commands_context_), request(request_), response(response_), log(&Logger::get("BatchCommandsHandler")) +{} + +ThreadPool::Job BatchCommandsHandler::handleCommandJob( + const tikvpb::BatchCommandsRequest::Request & req, tikvpb::BatchCommandsResponse::Response & resp, grpc::Status & ret) const +{ + return [&]() { + if (!req.has_coprocessor()) + { + ret = grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, ""); + return; + } + + const auto & cop_req = req.coprocessor(); + auto cop_resp = resp.mutable_coprocessor(); + + auto [context, status] = batch_commands_context.db_context_creation_func(&batch_commands_context.grpc_server_context); + if (!status.ok()) + { + ret = status; + return; + } + + CoprocessorContext cop_context(context, cop_req.context(), batch_commands_context.grpc_server_context); + CoprocessorHandler cop_handler(cop_context, &cop_req, cop_resp); + + ret = cop_handler.execute(); + }; +} + +grpc::Status BatchCommandsHandler::execute() +{ + if (request.requests_size() == 0) + return grpc::Status::OK; + + // TODO: Fill transport_layer_load into BatchCommandsResponse. + + /// Shortcut for only one request by not going to thread pool. + if (request.requests_size() == 1) + { + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling the only batch command in place."); + + const auto & req = request.requests(0); + auto resp = response.add_responses(); + response.add_request_ids(request.request_ids(0)); + auto ret = grpc::Status::OK; + handleCommandJob(req, *resp, ret)(); + return ret; + } + + /// Use thread pool to handle requests concurrently. + const Settings & settings = batch_commands_context.db_context.getSettingsRef(); + size_t max_threads = settings.batch_commands_threads ? static_cast(settings.batch_commands_threads) + : static_cast(settings.max_threads); + + LOG_DEBUG( + log, __PRETTY_FUNCTION__ << ": Handling " << request.requests_size() << " batch commands using " << max_threads << " threads."); + + ThreadPool thread_pool(max_threads); + + std::vector rets; + rets.reserve(request.requests_size()); + size_t i = 0; + + for (const auto & req : request.requests()) + { + auto resp = response.add_responses(); + response.add_request_ids(request.request_ids(i++)); + rets.emplace_back(grpc::Status::OK); + + thread_pool.schedule(handleCommandJob(req, *resp, rets.back())); + } + + thread_pool.wait(); + + // Iterate all return values of each individual commands, returns the first non-OK one if any. + for (const auto & ret : rets) + { + if (!ret.ok()) + { + response.Clear(); + return ret; + } + } + + return grpc::Status::OK; +} + +} // namespace DB diff --git a/dbms/src/Flash/BatchCommandsHandler.h b/dbms/src/Flash/BatchCommandsHandler.h new file mode 100644 index 00000000000..55b07a628fd --- /dev/null +++ b/dbms/src/Flash/BatchCommandsHandler.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +namespace DB +{ + +struct BatchCommandsContext +{ + /// Context for this batch commands. + Context & db_context; + + /// Context creation function for each individual command - they should be handled isolated, + /// given that context is being used to pass arguments regarding queries. + using DBContextCreationFunc = std::function(const grpc::ServerContext *)>; + DBContextCreationFunc db_context_creation_func; + + const grpc::ServerContext & grpc_server_context; + + BatchCommandsContext( + Context & db_context_, DBContextCreationFunc && db_context_creation_func_, grpc::ServerContext & grpc_server_context_) + : db_context(db_context_), db_context_creation_func(std::move(db_context_creation_func_)), grpc_server_context(grpc_server_context_) + {} +}; + +class BatchCommandsHandler +{ +public: + BatchCommandsHandler(BatchCommandsContext & batch_commands_context_, const tikvpb::BatchCommandsRequest & request_, + tikvpb::BatchCommandsResponse & response_); + + ~BatchCommandsHandler() = default; + + grpc::Status execute(); + +protected: + ThreadPool::Job handleCommandJob( + const tikvpb::BatchCommandsRequest::Request & req, tikvpb::BatchCommandsResponse::Response & resp, grpc::Status & ret) const; + +protected: + const BatchCommandsContext & batch_commands_context; + const tikvpb::BatchCommandsRequest & request; + tikvpb::BatchCommandsResponse & response; + + Logger * log; +}; + +} // namespace DB diff --git a/dbms/src/Flash/CMakeLists.txt b/dbms/src/Flash/CMakeLists.txt new file mode 100644 index 00000000000..365716927af --- /dev/null +++ b/dbms/src/Flash/CMakeLists.txt @@ -0,0 +1,11 @@ +include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) + +add_headers_and_sources(flash_service .) +add_headers_and_sources(flash_service ./Coprocessor) + +add_library(flash_service ${flash_service_headers} ${flash_service_sources}) +target_link_libraries(flash_service dbms) + +if (ENABLE_TESTS) + add_subdirectory (Coprocessor/tests) +endif () diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp new file mode 100644 index 00000000000..3514c1d006d --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.cpp @@ -0,0 +1,86 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNSUPPORTED_PARAMETER; +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes + +using TiDB::DatumBumpy; +using TiDB::TP; + +DAGBlockOutputStream::DAGBlockOutputStream(tipb::SelectResponse & dag_response_, Int64 records_per_chunk_, tipb::EncodeType encodeType_, + std::vector && result_field_types_, Block header_) + : dag_response(dag_response_), + records_per_chunk(records_per_chunk_), + encodeType(encodeType_), + result_field_types(result_field_types_), + header(header_) +{ + if (encodeType == tipb::EncodeType::TypeArrow) + { + throw Exception("Encode type TypeArrow is not supported yet in DAGBlockOutputStream.", ErrorCodes::UNSUPPORTED_PARAMETER); + } + current_chunk = nullptr; + current_records_num = 0; +} + + +void DAGBlockOutputStream::writePrefix() +{ + //something to do here? +} + +void DAGBlockOutputStream::writeSuffix() +{ + // error handle, + if (current_chunk != nullptr && current_records_num > 0) + { + current_chunk->set_rows_data(current_ss.str()); + dag_response.add_output_counts(current_records_num); + } +} + +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); + + // TODO: Check compatibility between field_tp_and_flags and block column types. + + // Encode data to chunk + size_t rows = block.rows(); + for (size_t i = 0; i < rows; i++) + { + if (current_chunk == nullptr || current_records_num >= records_per_chunk) + { + if (current_chunk) + { + // set the current ss to current chunk + current_chunk->set_rows_data(current_ss.str()); + dag_response.add_output_counts(current_records_num); + } + current_chunk = dag_response.add_chunks(); + current_ss.str(""); + current_records_num = 0; + } + for (size_t j = 0; j < block.columns(); j++) + { + const auto & field = (*block.getByPosition(j).column.get())[i]; + DatumBumpy datum(field, static_cast(result_field_types[j].tp())); + EncodeDatum(datum.field(), getCodecFlagByFieldType(result_field_types[j]), current_ss); + } + // Encode current row + current_records_num++; + } +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h new file mode 100644 index 00000000000..9ac6c5495fa --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h @@ -0,0 +1,43 @@ +#pragma once + +#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 DAGBlockOutputStream : public IBlockOutputStream +{ +public: + DAGBlockOutputStream(tipb::SelectResponse & response_, 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; + +private: + tipb::SelectResponse & dag_response; + + const Int64 records_per_chunk; + tipb::EncodeType encodeType; + std::vector result_field_types; + + Block header; + + tipb::Chunk * current_chunk; + Int64 current_records_num; + std::stringstream current_ss; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGCodec.cpp b/dbms/src/Flash/Coprocessor/DAGCodec.cpp new file mode 100644 index 00000000000..2316cdcad99 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGCodec.cpp @@ -0,0 +1,65 @@ +#include + +#include +#include + +namespace DB +{ + +void encodeDAGInt64(Int64 i, std::stringstream & ss) +{ + auto u = RecordKVFormat::encodeInt64(i); + ss.write(reinterpret_cast(&u), sizeof(u)); +} + +void encodeDAGUInt64(UInt64 i, std::stringstream & ss) +{ + auto u = RecordKVFormat::encodeUInt64(i); + ss.write(reinterpret_cast(&u), sizeof(u)); +} + +void encodeDAGFloat32(Float32 f, std::stringstream & ss) { EncodeFloat64(f, ss); } + +void encodeDAGFloat64(Float64 f, std::stringstream & ss) { EncodeFloat64(f, ss); } + +void encodeDAGString(const String & s, std::stringstream & ss) { ss << s; } + +void encodeDAGBytes(const String & bytes, std::stringstream & ss) { ss << bytes; } + +void encodeDAGDecimal(const Field & field, std::stringstream & ss) { EncodeDecimal(field, ss); } + +Int64 decodeDAGInt64(const String & s) +{ + auto u = *(reinterpret_cast(s.data())); + return RecordKVFormat::decodeInt64(u); +} + +UInt64 decodeDAGUInt64(const String & s) +{ + auto u = *(reinterpret_cast(s.data())); + return RecordKVFormat::decodeUInt64(u); +} + +Float32 decodeDAGFloat32(const String & s) +{ + size_t cursor = 0; + return DecodeFloat64(cursor, s); +} + +Float64 decodeDAGFloat64(const String & s) +{ + size_t cursor = 0; + return DecodeFloat64(cursor, s); +} + +String decodeDAGString(const String & s) { return s; } + +String decodeDAGBytes(const String & s) { return s; } + +Field decodeDAGDecimal(const String & s) +{ + size_t cursor = 0; + return DecodeDecimal(cursor, s); +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGCodec.h b/dbms/src/Flash/Coprocessor/DAGCodec.h new file mode 100644 index 00000000000..44fb9e5bc3f --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGCodec.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + +namespace DB +{ + +void encodeDAGInt64(Int64, std::stringstream &); +void encodeDAGUInt64(UInt64, std::stringstream &); +void encodeDAGFloat32(Float32, std::stringstream &); +void encodeDAGFloat64(Float64, std::stringstream &); +void encodeDAGString(const String &, std::stringstream &); +void encodeDAGBytes(const String &, std::stringstream &); +void encodeDAGDecimal(const Field &, std::stringstream &); + +Int64 decodeDAGInt64(const String &); +UInt64 decodeDAGUInt64(const String &); +Float32 decodeDAGFloat32(const String &); +Float64 decodeDAGFloat64(const String &); +String decodeDAGString(const String &); +String decodeDAGBytes(const String &); +Field decodeDAGDecimal(const String &); + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h new file mode 100644 index 00000000000..453e6df219e --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Context; + +/// 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; + + tipb::FieldType void_result_ft; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp new file mode 100644 index 00000000000..a0499693def --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -0,0 +1,124 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +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_, 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_), + key_ranges(std::move(key_ranges_)), + dag_response(dag_response_), + internal(internal_), + log(&Logger::get("DAGDriver")) +{} + +void DAGDriver::execute() +try +{ + context.setSetting("read_tso", UInt64(dag_request.start_ts())); + + DAGContext dag_context(dag_request.executors_size()); + DAGQuerySource dag(context, dag_context, region_id, region_version, region_conf_version, key_ranges, dag_request); + BlockIO streams; + + String planner = context.getSettings().dag_planner; + if (planner == "sql") + { + DAGStringConverter converter(context, dag_request); + String query = converter.buildSqlString(); + if (!query.empty()) + streams = executeQuery(query, context, internal, QueryProcessingStage::Complete); + } + else if (planner == "optree") + { + streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); + } + else + { + throw Exception("Unknown DAG planner type " + planner, ErrorCodes::LOGICAL_ERROR); + } + + 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(dag_response, + context.getSettings().dag_records_per_chunk, + dag_request.encode_type(), + 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__ << ": Exception: " << e.getStackTrace().toString()); + recordError(e.code(), e.message()); +} +catch (const std::exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": Exception: " << e.what()); + recordError(ErrorCodes::UNKNOWN_EXCEPTION, e.what()); +} + +void DAGDriver::recordError(Int32 err_code, const String & err_msg) +{ + dag_response.Clear(); + tipb::Error * error = dag_response.mutable_error(); + error->set_code(err_code); + error->set_msg(err_msg); +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.h b/dbms/src/Flash/Coprocessor/DAGDriver.h new file mode 100644 index 00000000000..38be627dc18 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGDriver.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +/// 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_, std::vector> && key_ranges_, + tipb::SelectResponse & dag_response_, bool internal_ = false); + + void execute(); + +private: + Context & context; + + const tipb::DAGRequest & dag_request; + + RegionID region_id; + UInt64 region_version; + UInt64 region_conf_version; + std::vector> key_ranges; + + tipb::SelectResponse & dag_response; + + bool internal; + + Poco::Logger * log; + + void recordError(Int32 err_code, const String & err_msg); +}; +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp new file mode 100644 index 00000000000..1a362d688e4 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -0,0 +1,530 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int COP_BAD_DAG_REQUEST; +extern const int UNSUPPORTED_METHOD; +} // namespace ErrorCodes + +static String genFuncString(const String & func_name, const Names & argument_names) +{ + std::stringstream ss; + ss << func_name << "("; + bool first = true; + for (const String & argument_name : argument_names) + { + if (first) + { + first = false; + } + else + { + ss << ", "; + } + ss << argument_name; + } + ss << ") "; + return ss.str(); +} + +DAGExpressionAnalyzer::DAGExpressionAnalyzer(std::vector && source_columns_, const Context & context_) + : source_columns(std::move(source_columns_)), + context(context_), + after_agg(false), + implicit_cast_count(0), + log(&Logger::get("DAGExpressionAnalyzer")) +{ + settings = context.getSettings(); +} + +void DAGExpressionAnalyzer::appendAggregation( + ExpressionActionsChain & chain, const tipb::Aggregation & agg, Names & aggregation_keys, AggregateDescriptions & aggregate_descriptions) +{ + if (agg.group_by_size() == 0 && agg.agg_func_size() == 0) + { + //should not reach here + throw Exception("Aggregation executor without group by/agg exprs", ErrorCodes::COP_BAD_DAG_REQUEST); + } + initChain(chain, getCurrentInputColumns()); + ExpressionActionsChain::Step & step = chain.steps.back(); + + for (const tipb::Expr & expr : agg.agg_func()) + { + const String & agg_func_name = getAggFunctionName(expr); + AggregateDescription aggregate; + DataTypes types(expr.children_size()); + aggregate.argument_names.resize(expr.children_size()); + for (Int32 i = 0; i < expr.children_size(); i++) + { + String arg_name = getActions(expr.children(i), step.actions); + types[i] = step.actions->getSampleBlock().getByName(arg_name).type; + aggregate.argument_names[i] = arg_name; + step.required_output.push_back(arg_name); + } + String func_string = genFuncString(agg_func_name, aggregate.argument_names); + bool duplicate = false; + for (const auto & pre_agg : aggregate_descriptions) + { + if (pre_agg.column_name == func_string) + { + aggregated_columns.emplace_back(func_string, pre_agg.function->getReturnType()); + duplicate = true; + break; + } + } + if (duplicate) + continue; + aggregate.column_name = func_string; + aggregate.parameters = Array(); + aggregate.function = AggregateFunctionFactory::instance().get(agg_func_name, types); + aggregate_descriptions.push_back(aggregate); + DataTypePtr result_type = aggregate.function->getReturnType(); + // this is a temp result since implicit cast maybe added on these aggregated_columns + aggregated_columns.emplace_back(func_string, result_type); + } + + for (const tipb::Expr & expr : agg.group_by()) + { + String name = getActions(expr, step.actions); + step.required_output.push_back(name); + // this is a temp result since implicit cast maybe added on these aggregated_columns + aggregated_columns.emplace_back(name, step.actions->getSampleBlock().getByName(name).type); + aggregation_keys.push_back(name); + } + after_agg = true; +} + +bool isUInt8Type(const DataTypePtr & type) +{ + auto non_nullable_type = type->isNullable() ? std::dynamic_pointer_cast(type)->getNestedType() : type; + return std::dynamic_pointer_cast(non_nullable_type) != nullptr; +} + +String DAGExpressionAnalyzer::applyFunction(const String & func_name, Names & arg_names, ExpressionActionsPtr & actions) +{ + String result_name = genFuncString(func_name, arg_names); + if (actions->getSampleBlock().has(result_name)) + return result_name; + const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(func_name, context); + const ExpressionAction & apply_function = ExpressionAction::applyFunction(function_builder, arg_names, result_name); + actions->add(apply_function); + return result_name; +} + +void DAGExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, const tipb::Selection & sel, String & filter_column_name) +{ + if (sel.conditions_size() == 0) + { + throw Exception("Selection executor without condition exprs", ErrorCodes::COP_BAD_DAG_REQUEST); + } + + initChain(chain, getCurrentInputColumns()); + ExpressionActionsChain::Step & last_step = chain.steps.back(); + Names arg_names; + for (auto & condition : sel.conditions()) + { + arg_names.push_back(getActions(condition, last_step.actions)); + } + if (arg_names.size() == 1) + { + filter_column_name = arg_names[0]; + } + else + { + // connect all the conditions by logical and + filter_column_name = applyFunction("and", arg_names, last_step.actions); + } + + auto & filter_column_type = chain.steps.back().actions->getSampleBlock().getByName(filter_column_name).type; + if (!isUInt8Type(filter_column_type)) + { + // find the original unit8 column + auto & last_actions = last_step.actions->getActions(); + for (auto it = last_actions.rbegin(); it != last_actions.rend(); ++it) + { + if (it->type == ExpressionAction::Type::APPLY_FUNCTION && it->result_name == filter_column_name + && it->function->getName() == "CAST") + { + // for cast function, the casted column is the first argument + filter_column_name = it->argument_names[0]; + break; + } + } + } + chain.steps.back().required_output.push_back(filter_column_name); +} + +void DAGExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, const tipb::TopN & topN, Strings & order_column_names) +{ + if (topN.order_by_size() == 0) + { + throw Exception("TopN executor without order by exprs", ErrorCodes::COP_BAD_DAG_REQUEST); + } + initChain(chain, getCurrentInputColumns()); + ExpressionActionsChain::Step & step = chain.steps.back(); + for (const tipb::ByItem & byItem : topN.order_by()) + { + String name = getActions(byItem.expr(), step.actions); + step.required_output.push_back(name); + order_column_names.push_back(name); + } +} + +const std::vector & DAGExpressionAnalyzer::getCurrentInputColumns() +{ + return after_agg ? aggregated_columns : source_columns; +} + +void DAGExpressionAnalyzer::appendFinalProject(ExpressionActionsChain & chain, const NamesWithAliases & final_project) +{ + initChain(chain, getCurrentInputColumns()); + for (const auto & name : final_project) + { + chain.steps.back().required_output.push_back(name.first); + } +} + +void constructTZExpr(tipb::Expr & tz_expr, const tipb::DAGRequest & rqst, bool from_utc) +{ + if (rqst.has_time_zone_name() && rqst.time_zone_name().length() > 0) + { + tz_expr.set_tp(tipb::ExprType::String); + tz_expr.set_val(rqst.time_zone_name()); + auto * field_type = tz_expr.mutable_field_type(); + field_type->set_tp(TiDB::TypeString); + field_type->set_flag(TiDB::ColumnFlagNotNull); + } + else + { + tz_expr.set_tp(tipb::ExprType::Int64); + std::stringstream ss; + encodeDAGInt64(from_utc ? rqst.time_zone_offset() : -rqst.time_zone_offset(), ss); + tz_expr.set_val(ss.str()); + auto * field_type = tz_expr.mutable_field_type(); + field_type->set_tp(TiDB::TypeLongLong); + field_type->set_flag(TiDB::ColumnFlagNotNull); + } +} + +bool hasMeaningfulTZInfo(const tipb::DAGRequest & rqst) +{ + if (rqst.has_time_zone_name() && rqst.time_zone_name().length() > 0) + return rqst.time_zone_name() != "UTC"; + if (rqst.has_time_zone_offset()) + return rqst.has_time_zone_offset() != 0; + return false; +} + +String DAGExpressionAnalyzer::appendTimeZoneCast( + const String & tz_col, const String & ts_col, const String & func_name, ExpressionActionsPtr & actions) +{ + Names cast_argument_names; + cast_argument_names.push_back(ts_col); + cast_argument_names.push_back(tz_col); + String cast_expr_name = applyFunction(func_name, cast_argument_names, actions); + return cast_expr_name; +} + +// add timezone cast after table scan, this is used for session level timezone support +// the basic idea of supporting session level timezone is that: +// 1. for every timestamp column used in the dag request, after reading it from table scan, we add +// cast function to convert its timezone to the timezone specified in DAG request +// 2. for every timestamp column that will be returned to TiDB, we add cast function to convert its +// timezone to UTC +// for timestamp columns without any transformation or calculation(e.g. select ts_col from table), +// this will introduce two useless casts, in order to avoid these redundant cast, when cast the ts +// column to the columns with session-level timezone info, the original ts columns with UTC +// timezone are still kept +// for DAG request that does not contain agg, the final project will select the ts column with UTC +// timezone, which is exactly what TiDB want +// for DAG request that contains agg, any ts column after agg has session-level timezone info(since the ts +// column with UTC timezone will never be used in during agg), all the column with ts datatype will +// convert back to UTC timezone +bool DAGExpressionAnalyzer::appendTimeZoneCastsAfterTS( + ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst) +{ + if (!hasMeaningfulTZInfo(rqst)) + return false; + + bool ret = false; + initChain(chain, getCurrentInputColumns()); + ExpressionActionsPtr actions = chain.getLastActions(); + tipb::Expr tz_expr; + constructTZExpr(tz_expr, rqst, true); + String tz_col; + String func_name + = rqst.has_time_zone_name() && rqst.time_zone_name().length() > 0 ? "ConvertTimeZoneFromUTC" : "ConvertTimeZoneByOffset"; + for (size_t i = 0; i < is_ts_column.size(); i++) + { + if (is_ts_column[i]) + { + if (tz_col.length() == 0) + tz_col = getActions(tz_expr, actions); + String casted_name = appendTimeZoneCast(tz_col, source_columns[i].name, func_name, actions); + source_columns.emplace_back(source_columns[i].name, source_columns[i].type); + source_columns[i].name = casted_name; + ret = true; + } + } + return ret; +} + +void DAGExpressionAnalyzer::appendAggSelect( + ExpressionActionsChain & chain, const tipb::Aggregation & aggregation, const tipb::DAGRequest & rqst) +{ + initChain(chain, getCurrentInputColumns()); + bool need_update_aggregated_columns = false; + std::vector updated_aggregated_columns; + ExpressionActionsChain::Step step = chain.steps.back(); + bool need_append_timezone_cast = hasMeaningfulTZInfo(rqst); + tipb::Expr tz_expr; + if (need_append_timezone_cast) + constructTZExpr(tz_expr, rqst, false); + String tz_col; + String tz_cast_func_name + = rqst.has_time_zone_name() && rqst.time_zone_name().length() > 0 ? "ConvertTimeZoneToUTC" : "ConvertTimeZoneByOffset"; + for (Int32 i = 0; i < aggregation.agg_func_size(); i++) + { + String & name = aggregated_columns[i].name; + String updated_name = appendCastIfNeeded(aggregation.agg_func(i), step.actions, name); + if (need_append_timezone_cast && aggregation.agg_func(i).field_type().tp() == TiDB::TypeTimestamp) + { + if (tz_col.length() == 0) + tz_col = getActions(tz_expr, step.actions); + updated_name = appendTimeZoneCast(tz_col, updated_name, tz_cast_func_name, step.actions); + } + if (name != updated_name) + { + need_update_aggregated_columns = true; + DataTypePtr type = step.actions->getSampleBlock().getByName(updated_name).type; + updated_aggregated_columns.emplace_back(updated_name, type); + step.required_output.push_back(updated_name); + } + else + { + updated_aggregated_columns.emplace_back(name, aggregated_columns[i].type); + step.required_output.push_back(name); + } + } + for (Int32 i = 0; i < aggregation.group_by_size(); i++) + { + String & name = aggregated_columns[i + aggregation.agg_func_size()].name; + String updated_name = appendCastIfNeeded(aggregation.group_by(i), step.actions, name); + if (need_append_timezone_cast && aggregation.group_by(i).field_type().tp() == TiDB::TypeTimestamp) + { + if (tz_col.length() == 0) + tz_col = getActions(tz_expr, step.actions); + updated_name = appendTimeZoneCast(tz_col, updated_name, tz_cast_func_name, step.actions); + } + if (name != updated_name) + { + need_update_aggregated_columns = true; + DataTypePtr type = step.actions->getSampleBlock().getByName(updated_name).type; + updated_aggregated_columns.emplace_back(updated_name, type); + step.required_output.push_back(updated_name); + } + else + { + updated_aggregated_columns.emplace_back(name, aggregated_columns[i].type); + step.required_output.push_back(name); + } + } + + if (need_update_aggregated_columns) + { + aggregated_columns.clear(); + for (size_t i = 0; i < updated_aggregated_columns.size(); i++) + { + aggregated_columns.emplace_back(updated_aggregated_columns[i].name, updated_aggregated_columns[i].type); + } + } +} + +String DAGExpressionAnalyzer::appendCastIfNeeded(const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name) +{ + if (!expr.has_field_type() && context.getSettingsRef().dag_expr_field_type_strict_check) + { + throw Exception("Expression without field type", ErrorCodes::COP_BAD_DAG_REQUEST); + } + if (exprHasValidFieldType(expr) && isFunctionExpr(expr)) + { + DataTypePtr expected_type = getDataTypeByFieldType(expr.field_type()); + DataTypePtr actual_type = actions->getSampleBlock().getByName(expr_name).type; + //todo maybe use a more decent compare method + // todo ignore nullable info?? + if (expected_type->getName() != actual_type->getName()) + { + LOG_DEBUG( + log, __PRETTY_FUNCTION__ << " Add implicit cast: from " << actual_type->getName() << " to " << expected_type->getName()); + implicit_cast_count++; + // need to add cast function + // first construct the second argument + tipb::Expr type_expr; + type_expr.set_tp(tipb::ExprType::String); + type_expr.set_val(expected_type->getName()); + auto * type_field_type = type_expr.mutable_field_type(); + type_field_type->set_tp(TiDB::TypeString); + type_field_type->set_flag(TiDB::ColumnFlagNotNull); + auto type_expr_name = getActions(type_expr, actions); + + Names cast_argument_names; + cast_argument_names.push_back(expr_name); + cast_argument_names.push_back(type_expr_name); + String cast_expr_name = applyFunction("CAST", cast_argument_names, actions); + return cast_expr_name; + } + else + { + return expr_name; + } + } + return expr_name; +} + +void DAGExpressionAnalyzer::makeExplicitSetForIndex(const tipb::Expr & expr, const TMTStoragePtr & storage) +{ + for (auto & child : expr.children()) + { + makeExplicitSetForIndex(child, storage); + } + if (expr.tp() != tipb::ExprType::ScalarFunc) + { + return; + } + const String & func_name = getFunctionName(expr); + // only support col_name in (value_list) + if (isInOrGlobalInOperator(func_name) && expr.children(0).tp() == tipb::ExprType::ColumnRef && !prepared_sets.count(&expr)) + { + NamesAndTypesList column_list; + for (const auto & col : getCurrentInputColumns()) + { + column_list.emplace_back(col.name, col.type); + } + ExpressionActionsPtr temp_actions = std::make_shared(column_list, settings); + String name = getActions(expr.children(0), temp_actions); + ASTPtr name_ast = std::make_shared(name); + if (storage->mayBenefitFromIndexForIn(name_ast)) + { + makeExplicitSet(expr, temp_actions->getSampleBlock(), true, name); + } + } +} + +void DAGExpressionAnalyzer::makeExplicitSet( + const tipb::Expr & expr, const Block & sample_block, bool create_ordered_set, const String & left_arg_name) +{ + if (prepared_sets.count(&expr)) + { + return; + } + DataTypes set_element_types; + // todo support tuple in, i.e. (a,b) in ((1,2), (3,4)), currently TiDB convert tuple in into a series of or/and/eq exprs + // which means tuple in is never be pushed to coprocessor, but it is quite in-efficient + set_element_types.push_back(sample_block.getByName(left_arg_name).type); + + // todo if this is a single value in, then convert it to equal expr + SetPtr set = std::make_shared(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode)); + set->createFromDAGExpr(set_element_types, expr, create_ordered_set); + prepared_sets[&expr] = std::move(set); +} + +static String getUniqueName(const Block & block, const String & prefix) +{ + int i = 1; + while (block.has(prefix + toString(i))) + ++i; + return prefix + toString(i); +} + +String DAGExpressionAnalyzer::getActions(const tipb::Expr & expr, ExpressionActionsPtr & actions) +{ + if (isLiteralExpr(expr)) + { + Field value = decodeLiteral(expr); + DataTypePtr flash_type = applyVisitor(FieldToDataType(), value); + DataTypePtr target_type = exprHasValidFieldType(expr) ? getDataTypeByFieldType(expr.field_type()) : flash_type; + String name = exprToString(expr, getCurrentInputColumns()) + "_" + target_type->getName(); + if (actions->getSampleBlock().has(name)) + return name; + + ColumnWithTypeAndName column; + column.column = target_type->createColumnConst(1, convertFieldToType(value, *target_type, flash_type.get())); + column.name = name; + column.type = target_type; + + actions->add(ExpressionAction::addColumn(column)); + return name; + } + else if (isColumnExpr(expr)) + { + //todo check if the column type need to be cast to field type + return getColumnNameForColumnExpr(expr, getCurrentInputColumns()); + } + else if (isFunctionExpr(expr)) + { + if (isAggFunctionExpr(expr)) + { + throw Exception("agg function is not supported yet", ErrorCodes::UNSUPPORTED_METHOD); + } + const String & func_name = getFunctionName(expr); + Names argument_names; + DataTypes argument_types; + + if (isInOrGlobalInOperator(func_name)) + { + String name = getActions(expr.children(0), actions); + argument_names.push_back(name); + argument_types.push_back(actions->getSampleBlock().getByName(name).type); + makeExplicitSet(expr, actions->getSampleBlock(), false, name); + ColumnWithTypeAndName column; + column.type = std::make_shared(); + + const SetPtr & set = prepared_sets[&expr]; + + column.name = getUniqueName(actions->getSampleBlock(), "___set"); + column.column = ColumnSet::create(1, set); + actions->add(ExpressionAction::addColumn(column)); + argument_names.push_back(column.name); + argument_types.push_back(column.type); + } + else + { + for (auto & child : expr.children()) + { + String name = getActions(child, actions); + argument_names.push_back(name); + argument_types.push_back(actions->getSampleBlock().getByName(name).type); + } + } + + String expr_name = applyFunction(func_name, argument_names, actions); + // add cast if needed + expr_name = appendCastIfNeeded(expr, actions, expr_name); + return expr_name; + } + else + { + throw Exception("Unsupported expr type: " + getTypeName(expr), ErrorCodes::UNSUPPORTED_METHOD); + } +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h new file mode 100644 index 00000000000..d2a6b5751be --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -0,0 +1,72 @@ +#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 Set; +using SetPtr = std::shared_ptr; +using DAGPreparedSets = std::unordered_map; + +/** Transforms an expression from DAG expression into a sequence of actions to execute it. + * + */ +class DAGExpressionAnalyzer : private boost::noncopyable +{ +private: + using ExpressionActionsPtr = std::shared_ptr; + // all columns from table scan + std::vector source_columns; + // all columns after aggregation + std::vector aggregated_columns; + DAGPreparedSets prepared_sets; + Settings settings; + const Context & context; + bool after_agg; + Int32 implicit_cast_count; + Poco::Logger * log; + +public: + DAGExpressionAnalyzer(std::vector && source_columns_, const Context & context_); + void appendWhere(ExpressionActionsChain & chain, const tipb::Selection & sel, String & filter_column_name); + void appendOrderBy(ExpressionActionsChain & chain, const tipb::TopN & topN, Strings & order_column_names); + void appendAggregation(ExpressionActionsChain & chain, const tipb::Aggregation & agg, Names & aggregate_keys, + AggregateDescriptions & aggregate_descriptions); + void appendAggSelect(ExpressionActionsChain & chain, const tipb::Aggregation & agg, const tipb::DAGRequest & rqst); + String appendCastIfNeeded(const tipb::Expr & expr, ExpressionActionsPtr & actions, const String & expr_name); + void initChain(ExpressionActionsChain & chain, const std::vector & columns) const + { + if (chain.steps.empty()) + { + chain.settings = settings; + NamesAndTypesList column_list; + for (const auto & col : columns) + { + column_list.emplace_back(col.name, col.type); + } + chain.steps.emplace_back(std::make_shared(column_list, settings)); + } + } + void appendFinalProject(ExpressionActionsChain & chain, const NamesWithAliases & final_project); + String getActions(const tipb::Expr & expr, ExpressionActionsPtr & actions); + const std::vector & getCurrentInputColumns(); + void makeExplicitSet(const tipb::Expr & expr, const Block & sample_block, bool create_ordered_set, const String & left_arg_name); + void makeExplicitSetForIndex(const tipb::Expr & expr, const TMTStoragePtr & storage); + String applyFunction(const String & func_name, Names & arg_names, ExpressionActionsPtr & actions); + Int32 getImplicitCastCount() { return implicit_cast_count; }; + bool appendTimeZoneCastsAfterTS(ExpressionActionsChain & chain, std::vector is_ts_column, const tipb::DAGRequest & rqst); + String appendTimeZoneCast(const String & tz_col, const String & ts_col, const String & func_name, ExpressionActionsPtr & actions); + DAGPreparedSets getPreparedSets() { return prepared_sets; } +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryInfo.h b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h new file mode 100644 index 00000000000..cb01768d473 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGQueryInfo.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +struct DAGQueryInfo +{ + DAGQueryInfo(const DAGQuerySource & dag_, DAGPreparedSets dag_sets_, const std::vector & source_columns_) + : dag(dag_), dag_sets(std::move(dag_sets_)), source_columns(source_columns_){}; + const DAGQuerySource & dag; + DAGPreparedSets dag_sets; + const std::vector & source_columns; +}; +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp new file mode 100644 index 00000000000..c9d7b22871f --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -0,0 +1,161 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +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"); + +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_, + 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_), + key_ranges(key_ranges_), + dag_request(dag_request_) +{ + for (int i = 0; i < dag_request.executors_size(); i++) + { + switch (dag_request.executors(i).tp()) + { + case tipb::ExecType::TypeTableScan: + assignOrThrowException(ts_index, i, TS_NAME); + break; + case tipb::ExecType::TypeSelection: + assignOrThrowException(sel_index, i, SEL_NAME); + break; + case tipb::ExecType::TypeStreamAgg: + case tipb::ExecType::TypeAggregation: + assignOrThrowException(agg_index, i, AGG_NAME); + break; + case tipb::ExecType::TypeTopN: + assignOrThrowException(order_index, i, TOPN_NAME); + assignOrThrowException(limit_index, i, TOPN_NAME); + break; + case tipb::ExecType::TypeLimit: + assignOrThrowException(limit_index, i, LIMIT_NAME); + break; + default: + throw Exception("Unsupported executor in DAG request: " + dag_request.executors(i).DebugString(), ErrorCodes::NOT_IMPLEMENTED); + } + } +} + +std::tuple DAGQuerySource::parse(size_t max_query_size) +{ + // this is a WAR to avoid NPE when the MergeTreeDataSelectExecutor trying + // to extract key range of the query. + // todo find a way to enable key range extraction for dag query + String tmp = "select 1"; + ParserQuery parser(tmp.data() + tmp.size()); + ASTPtr parent = parseQuery(parser, tmp.data(), tmp.data() + tmp.size(), "", max_query_size); + auto query = dag_request.DebugString(); + ast = ((ASTSelectWithUnionQuery *)parent.get())->list_of_selects->children.at(0); + return std::make_tuple(query, ast); +} + +String DAGQuerySource::str(size_t) { return dag_request.DebugString(); } + +std::unique_ptr DAGQuerySource::interpreter(Context &, QueryProcessingStage::Enum) +{ + return std::make_unique(context, *this); +} + +bool fillExecutorOutputFieldTypes(const tipb::Executor & executor, std::vector & output_field_types) +{ + tipb::FieldType field_type; + switch (executor.tp()) + { + case tipb::ExecType::TypeTableScan: + for (auto & ci : executor.tbl_scan().columns()) + { + if (ci.column_id() == -1) + continue; + 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); + } + return true; + case tipb::ExecType::TypeStreamAgg: + case tipb::ExecType::TypeAggregation: + for (auto & expr : executor.aggregation().agg_func()) + { + if (!exprHasValidFieldType(expr)) + { + throw Exception("Agg expression without valid field type", ErrorCodes::COP_BAD_DAG_REQUEST); + } + output_field_types.push_back(expr.field_type()); + } + for (auto & expr : executor.aggregation().group_by()) + { + if (!exprHasValidFieldType(expr)) + { + throw Exception("Group by expression without valid field type", ErrorCodes::COP_BAD_DAG_REQUEST); + } + output_field_types.push_back(expr.field_type()); + } + return true; + default: + return false; + } +} + +std::vector DAGQuerySource::getResultFieldTypes() const +{ + std::vector executor_output; + for (int i = dag_request.executors_size() - 1; i >= 0; i--) + { + if (fillExecutorOutputFieldTypes(dag_request.executors(i), executor_output)) + { + if (executor_output.empty()) + executor_output.push_back(dag_context.void_result_ft); + break; + } + } + if (executor_output.empty()) + { + throw Exception("Do not found result field type for current dag request", ErrorCodes::COP_BAD_DAG_REQUEST); + } + // tispark assumes that if there is a agg, the output offset is + // ignored and the request output is the same as the agg's output. + // todo should always use output offset to re-construct the output field types + if (hasAggregation()) + { + return executor_output; + } + std::vector ret; + for (int i : dag_request.output_offsets()) + { + ret.push_back(executor_output[i]); + } + return ret; +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h new file mode 100644 index 00000000000..b7f4791ad56 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -0,0 +1,116 @@ +#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; + + +/// Query source of a DAG request via gRPC. +/// This is also an IR of a DAG. +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; + + 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_); + + std::tuple parse(size_t max_query_size) override; + String str(size_t max_query_size) override; + std::unique_ptr interpreter(Context & context, QueryProcessingStage::Enum stage) override; + + 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(); + }; + const tipb::DAGRequest & getDAGRequest() const { return dag_request; }; + + std::vector getResultFieldTypes() const; + + ASTPtr getAST() const { return ast; }; + +protected: + void assertValid(Int32 index, const String & name) const + { + if (index < 0 || index > dag_request.executors_size()) + { + throw Exception("Access invalid executor: " + name); + } + } + +protected: + Context & context; + DAGContext & dag_context; + + const RegionID region_id; + const UInt64 region_version; + const UInt64 region_conf_version; + const std::vector> & key_ranges; + + const tipb::DAGRequest & dag_request; + + Int32 ts_index = -1; + Int32 sel_index = -1; + Int32 agg_index = -1; + Int32 order_index = -1; + Int32 limit_index = -1; + + ASTPtr ast; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp new file mode 100644 index 00000000000..ef5efcc08f7 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGStringConverter.cpp @@ -0,0 +1,155 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNKNOWN_TABLE; +extern const int COP_BAD_DAG_REQUEST; +extern const int NOT_IMPLEMENTED; +} // namespace ErrorCodes + +void DAGStringConverter::buildTSString(const tipb::TableScan & ts, std::stringstream & ss) +{ + TableID table_id; + if (ts.has_table_id()) + { + table_id = ts.table_id(); + } + else + { + // do not have table id + throw Exception("Table id not specified in table scan executor", ErrorCodes::COP_BAD_DAG_REQUEST); + } + auto & tmt_ctx = context.getTMTContext(); + auto storage = tmt_ctx.getStorages().get(table_id); + if (storage == nullptr) + { + throw Exception("Table " + std::to_string(table_id) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + } + const auto * merge_tree = dynamic_cast(storage.get()); + if (!merge_tree) + { + throw Exception("Only MergeTree table is supported in DAG request", ErrorCodes::COP_BAD_DAG_REQUEST); + } + + if (ts.columns_size() == 0) + { + // no column selected, must be something wrong + throw Exception("No column is selected in table scan executor", ErrorCodes::COP_BAD_DAG_REQUEST); + } + const auto & column_list = storage->getColumns().getAllPhysical(); + for (auto & column : column_list) + { + columns_from_ts.emplace_back(column.name, column.type); + } + for (const tipb::ColumnInfo & ci : ts.columns()) + { + ColumnID cid = ci.column_id(); + if (cid <= 0 || cid > (ColumnID)columns_from_ts.size()) + { + throw Exception("column id out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); + } + String name = merge_tree->getTableInfo().columns[cid - 1].name; + output_from_ts.push_back(std::move(name)); + } + ss << "FROM " << storage->getDatabaseName() << "." << storage->getTableName() << " "; +} + +void DAGStringConverter::buildSelString(const tipb::Selection & sel, std::stringstream & ss) +{ + bool first = true; + for (const tipb::Expr & expr : sel.conditions()) + { + auto s = exprToString(expr, getCurrentColumns()); + if (first) + { + ss << "WHERE "; + first = false; + } + else + { + ss << "AND "; + } + ss << s << " "; + } +} + +void DAGStringConverter::buildLimitString(const tipb::Limit & limit, std::stringstream & ss) { ss << "LIMIT " << limit.limit() << " "; } + +//todo return the error message +void DAGStringConverter::buildString(const tipb::Executor & executor, std::stringstream & ss) +{ + switch (executor.tp()) + { + case tipb::ExecType::TypeTableScan: + return buildTSString(executor.tbl_scan(), ss); + case tipb::ExecType::TypeIndexScan: + // index scan not supported + throw Exception("IndexScan is not supported", ErrorCodes::NOT_IMPLEMENTED); + case tipb::ExecType::TypeSelection: + return buildSelString(executor.selection(), ss); + case tipb::ExecType::TypeAggregation: + // stream agg is not supported, treated as normal agg + case tipb::ExecType::TypeStreamAgg: + //todo support agg + throw Exception("Aggregation is not supported", ErrorCodes::NOT_IMPLEMENTED); + case tipb::ExecType::TypeTopN: + // todo support top n + throw Exception("TopN is not supported", ErrorCodes::NOT_IMPLEMENTED); + case tipb::ExecType::TypeLimit: + return buildLimitString(executor.limit(), ss); + } +} + +bool isProject(const tipb::Executor &) +{ + // currently, project is not pushed so always return false + return false; +} +DAGStringConverter::DAGStringConverter(Context & context_, const tipb::DAGRequest & dag_request_) + : context(context_), dag_request(dag_request_) +{ + afterAgg = false; +} + +String DAGStringConverter::buildSqlString() +{ + std::stringstream query_buf; + std::stringstream project; + for (const tipb::Executor & executor : dag_request.executors()) + { + buildString(executor, query_buf); + } + if (!isProject(dag_request.executors(dag_request.executors_size() - 1))) + { + //append final project + project << "SELECT "; + bool first = true; + for (UInt32 index : dag_request.output_offsets()) + { + if (first) + { + first = false; + } + else + { + project << ", "; + } + project << getCurrentOutputColumns()[index]; + } + project << " "; + } + return project.str() + query_buf.str(); +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStringConverter.h b/dbms/src/Flash/Coprocessor/DAGStringConverter.h new file mode 100644 index 00000000000..f90396be281 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGStringConverter.h @@ -0,0 +1,61 @@ +#pragma once + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop + +#include + +namespace DB +{ + +class Context; + +class DAGStringConverter +{ +public: + DAGStringConverter(Context & context_, const tipb::DAGRequest & dag_request_); + + ~DAGStringConverter() = default; + + String buildSqlString(); + + const std::vector & getCurrentColumns() + { + if (afterAgg) + { + return columns_from_agg; + } + return columns_from_ts; + } + + const Names & getCurrentOutputColumns() + { + if (afterAgg) + { + return output_from_agg; + } + return output_from_ts; + } + +protected: + void buildTSString(const tipb::TableScan & ts, std::stringstream & ss); + void buildSelString(const tipb::Selection & sel, std::stringstream & ss); + void buildLimitString(const tipb::Limit & limit, std::stringstream & ss); + void buildString(const tipb::Executor & executor, std::stringstream & ss); + +protected: + Context & context; + const tipb::DAGRequest & dag_request; + // used by columnRef, which starts with 1, and refs column index in the original ts/agg output + std::vector columns_from_ts; + std::vector columns_from_agg; + // used by output_offset, which starts with 0, and refs the index in the selected output of ts/agg operater + Names output_from_ts; + Names output_from_agg; + bool afterAgg; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp new file mode 100644 index 00000000000..0f196df8db1 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -0,0 +1,860 @@ +#include + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int COP_BAD_DAG_REQUEST; +extern const int UNSUPPORTED_METHOD; +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes + +bool isFunctionExpr(const tipb::Expr & expr) { return expr.tp() == tipb::ExprType::ScalarFunc || isAggFunctionExpr(expr); } + +const String & getAggFunctionName(const tipb::Expr & expr) +{ + if (agg_func_map.find(expr.tp()) == agg_func_map.end()) + { + throw Exception(tipb::ExprType_Name(expr.tp()) + " is not supported.", ErrorCodes::UNSUPPORTED_METHOD); + } + return agg_func_map[expr.tp()]; +} + +const String & getFunctionName(const tipb::Expr & expr) +{ + if (isAggFunctionExpr(expr)) + { + if (agg_func_map.find(expr.tp()) == agg_func_map.end()) + { + throw Exception(tipb::ExprType_Name(expr.tp()) + " is not supported.", ErrorCodes::UNSUPPORTED_METHOD); + } + return agg_func_map[expr.tp()]; + } + else + { + if (scalar_func_map.find(expr.sig()) == scalar_func_map.end()) + { + throw Exception(tipb::ScalarFuncSig_Name(expr.sig()) + " is not supported.", ErrorCodes::UNSUPPORTED_METHOD); + } + return scalar_func_map[expr.sig()]; + } +} + +String exprToString(const tipb::Expr & expr, const std::vector & input_col) +{ + std::stringstream ss; + String func_name; + Field f; + switch (expr.tp()) + { + case tipb::ExprType::Null: + return "NULL"; + case tipb::ExprType::Int64: + return std::to_string(decodeDAGInt64(expr.val())); + case tipb::ExprType::Uint64: + return std::to_string(decodeDAGUInt64(expr.val())); + case tipb::ExprType::Float32: + return std::to_string(decodeDAGFloat32(expr.val())); + case tipb::ExprType::Float64: + return std::to_string(decodeDAGFloat64(expr.val())); + case tipb::ExprType::String: + return decodeDAGString(expr.val()); + case tipb::ExprType::Bytes: + return decodeDAGBytes(expr.val()); + case tipb::ExprType::MysqlDecimal: + { + auto field = decodeDAGDecimal(expr.val()); + if (field.getType() == Field::Types::Decimal32) + return field.get>().toString(); + else if (field.getType() == Field::Types::Decimal64) + return field.get>().toString(); + else if (field.getType() == Field::Types::Decimal128) + return field.get>().toString(); + else if (field.getType() == Field::Types::Decimal256) + return field.get>().toString(); + else + throw Exception("Not decimal literal" + expr.DebugString(), ErrorCodes::COP_BAD_DAG_REQUEST); + } + case tipb::ExprType::MysqlTime: + { + if (!expr.has_field_type() || (expr.field_type().tp() != TiDB::TypeDate && expr.field_type().tp() != TiDB::TypeDatetime)) + throw Exception("Invalid MySQL Time literal " + expr.DebugString(), ErrorCodes::COP_BAD_DAG_REQUEST); + auto t = decodeDAGUInt64(expr.val()); + // TODO: Use timezone in DAG request. + return std::to_string(TiDB::DatumFlat(t, static_cast(expr.field_type().tp())).field().get()); + } + case tipb::ExprType::ColumnRef: + return getColumnNameForColumnExpr(expr, input_col); + case tipb::ExprType::Count: + case tipb::ExprType::Sum: + case tipb::ExprType::Avg: + case tipb::ExprType::Min: + case tipb::ExprType::Max: + case tipb::ExprType::First: + if (agg_func_map.find(expr.tp()) == agg_func_map.end()) + { + throw Exception(tipb::ExprType_Name(expr.tp()) + " not supported", ErrorCodes::UNSUPPORTED_METHOD); + } + func_name = agg_func_map.find(expr.tp())->second; + break; + case tipb::ExprType::ScalarFunc: + if (scalar_func_map.find(expr.sig()) == scalar_func_map.end()) + { + throw Exception(tipb::ScalarFuncSig_Name(expr.sig()) + " not supported", ErrorCodes::UNSUPPORTED_METHOD); + } + func_name = scalar_func_map.find(expr.sig())->second; + break; + default: + throw Exception(tipb::ExprType_Name(expr.tp()) + " not supported", ErrorCodes::UNSUPPORTED_METHOD); + } + // build function expr + if (isInOrGlobalInOperator(func_name)) + { + // for in, we could not represent the function expr using func_name(param1, param2, ...) + throw Exception("Function " + func_name + " not supported", ErrorCodes::UNSUPPORTED_METHOD); + } + ss << func_name << "("; + bool first = true; + for (const tipb::Expr & child : expr.children()) + { + String s = exprToString(child, input_col); + if (first) + { + first = false; + } + else + { + ss << ", "; + } + ss << s; + } + ss << ") "; + return ss.str(); +} + +const String & getTypeName(const tipb::Expr & expr) { return tipb::ExprType_Name(expr.tp()); } + +bool isAggFunctionExpr(const tipb::Expr & expr) +{ + switch (expr.tp()) + { + case tipb::ExprType::Count: + case tipb::ExprType::Sum: + case tipb::ExprType::Avg: + case tipb::ExprType::Min: + case tipb::ExprType::Max: + case tipb::ExprType::First: + case tipb::ExprType::GroupConcat: + case tipb::ExprType::Agg_BitAnd: + case tipb::ExprType::Agg_BitOr: + case tipb::ExprType::Agg_BitXor: + case tipb::ExprType::Std: + case tipb::ExprType::Stddev: + case tipb::ExprType::StddevPop: + case tipb::ExprType::StddevSamp: + case tipb::ExprType::VarPop: + case tipb::ExprType::VarSamp: + case tipb::ExprType::Variance: + case tipb::ExprType::JsonArrayAgg: + case tipb::ExprType::JsonObjectAgg: + return true; + default: + return false; + } +} + +bool isLiteralExpr(const tipb::Expr & expr) +{ + switch (expr.tp()) + { + case tipb::ExprType::Null: + case tipb::ExprType::Int64: + case tipb::ExprType::Uint64: + case tipb::ExprType::Float32: + case tipb::ExprType::Float64: + case tipb::ExprType::String: + case tipb::ExprType::Bytes: + case tipb::ExprType::MysqlBit: + case tipb::ExprType::MysqlDecimal: + case tipb::ExprType::MysqlDuration: + case tipb::ExprType::MysqlEnum: + case tipb::ExprType::MysqlHex: + case tipb::ExprType::MysqlSet: + case tipb::ExprType::MysqlTime: + case tipb::ExprType::MysqlJson: + case tipb::ExprType::ValueList: + return true; + default: + return false; + } +} + +bool isColumnExpr(const tipb::Expr & expr) { return expr.tp() == tipb::ExprType::ColumnRef; } + +Field decodeLiteral(const tipb::Expr & expr) +{ + switch (expr.tp()) + { + case tipb::ExprType::Null: + return Field(); + case tipb::ExprType::Int64: + return decodeDAGInt64(expr.val()); + case tipb::ExprType::Uint64: + return decodeDAGUInt64(expr.val()); + case tipb::ExprType::Float32: + return Float64(decodeDAGFloat32(expr.val())); + case tipb::ExprType::Float64: + return decodeDAGFloat64(expr.val()); + case tipb::ExprType::String: + return decodeDAGString(expr.val()); + case tipb::ExprType::Bytes: + return decodeDAGBytes(expr.val()); + case tipb::ExprType::MysqlDecimal: + return decodeDAGDecimal(expr.val()); + case tipb::ExprType::MysqlTime: + { + if (!expr.has_field_type() || (expr.field_type().tp() != TiDB::TypeDate && expr.field_type().tp() != TiDB::TypeDatetime)) + throw Exception("Invalid MySQL Time literal " + expr.DebugString(), ErrorCodes::COP_BAD_DAG_REQUEST); + auto t = decodeDAGUInt64(expr.val()); + // TODO: Use timezone in DAG request. + return TiDB::DatumFlat(t, static_cast(expr.field_type().tp())).field(); + } + case tipb::ExprType::MysqlBit: + case tipb::ExprType::MysqlDuration: + case tipb::ExprType::MysqlEnum: + case tipb::ExprType::MysqlHex: + case tipb::ExprType::MysqlSet: + case tipb::ExprType::MysqlJson: + case tipb::ExprType::ValueList: + throw Exception(tipb::ExprType_Name(expr.tp()) + " is not supported yet", ErrorCodes::UNSUPPORTED_METHOD); + default: + throw Exception("Should not reach here: not a literal expression", ErrorCodes::LOGICAL_ERROR); + } +} + +String getColumnNameForColumnExpr(const tipb::Expr & expr, const std::vector & input_col) +{ + auto column_index = decodeDAGInt64(expr.val()); + if (column_index < 0 || column_index >= (Int64)input_col.size()) + { + throw Exception("Column index out of bound", ErrorCodes::COP_BAD_DAG_REQUEST); + } + return input_col[column_index].name; +} + +bool isInOrGlobalInOperator(const String & name) { return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn"; } + +// for some historical or unknown reasons, TiDB might set a invalid +// field type. This function checks if the expr has a valid field type +// so far the known invalid field types are: +// 1. decimal type with scale -1 +bool exprHasValidFieldType(const tipb::Expr & expr) +{ + return expr.has_field_type() && !(expr.field_type().tp() == TiDB::TP::TypeNewDecimal && expr.field_type().decimal() == -1); +} + +std::unordered_map agg_func_map({ + {tipb::ExprType::Count, "count"}, {tipb::ExprType::Sum, "sum"}, {tipb::ExprType::Min, "min"}, {tipb::ExprType::Max, "max"}, + {tipb::ExprType::First, "any"}, + //{tipb::ExprType::Avg, ""}, + //{tipb::ExprType::GroupConcat, ""}, + //{tipb::ExprType::Agg_BitAnd, ""}, + //{tipb::ExprType::Agg_BitOr, ""}, + //{tipb::ExprType::Agg_BitXor, ""}, + //{tipb::ExprType::Std, ""}, + //{tipb::ExprType::Stddev, ""}, + //{tipb::ExprType::StddevPop, ""}, + //{tipb::ExprType::StddevSamp, ""}, + //{tipb::ExprType::VarPop, ""}, + //{tipb::ExprType::VarSamp, ""}, + //{tipb::ExprType::Variance, ""}, + //{tipb::ExprType::JsonArrayAgg, ""}, + //{tipb::ExprType::JsonObjectAgg, ""}, +}); + +std::unordered_map scalar_func_map({ + /* + {tipb::ScalarFuncSig::CastIntAsInt, "cast"}, + {tipb::ScalarFuncSig::CastIntAsReal, "cast"}, + {tipb::ScalarFuncSig::CastIntAsString, "cast"}, + {tipb::ScalarFuncSig::CastIntAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastIntAsTime, "cast"}, + {tipb::ScalarFuncSig::CastIntAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastIntAsJson, "cast"}, + + {tipb::ScalarFuncSig::CastRealAsInt, "cast"}, + {tipb::ScalarFuncSig::CastRealAsReal, "cast"}, + {tipb::ScalarFuncSig::CastRealAsString, "cast"}, + {tipb::ScalarFuncSig::CastRealAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastRealAsTime, "cast"}, + {tipb::ScalarFuncSig::CastRealAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastRealAsJson, "cast"}, + + {tipb::ScalarFuncSig::CastDecimalAsInt, "cast"}, + {tipb::ScalarFuncSig::CastDecimalAsReal, "cast"}, + {tipb::ScalarFuncSig::CastDecimalAsString, "cast"}, + {tipb::ScalarFuncSig::CastDecimalAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastDecimalAsTime, "cast"}, + {tipb::ScalarFuncSig::CastDecimalAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastDecimalAsJson, "cast"}, + + {tipb::ScalarFuncSig::CastStringAsInt, "cast"}, + {tipb::ScalarFuncSig::CastStringAsReal, "cast"}, + {tipb::ScalarFuncSig::CastStringAsString, "cast"}, + {tipb::ScalarFuncSig::CastStringAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastStringAsTime, "cast"}, + {tipb::ScalarFuncSig::CastStringAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastStringAsJson, "cast"}, + + {tipb::ScalarFuncSig::CastTimeAsInt, "cast"}, + {tipb::ScalarFuncSig::CastTimeAsReal, "cast"}, + {tipb::ScalarFuncSig::CastTimeAsString, "cast"}, + {tipb::ScalarFuncSig::CastTimeAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastTimeAsTime, "cast"}, + {tipb::ScalarFuncSig::CastTimeAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastTimeAsJson, "cast"}, + + {tipb::ScalarFuncSig::CastDurationAsInt, "cast"}, + {tipb::ScalarFuncSig::CastDurationAsReal, "cast"}, + {tipb::ScalarFuncSig::CastDurationAsString, "cast"}, + {tipb::ScalarFuncSig::CastDurationAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastDurationAsTime, "cast"}, + {tipb::ScalarFuncSig::CastDurationAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastDurationAsJson, "cast"}, + + {tipb::ScalarFuncSig::CastJsonAsInt, "cast"}, + {tipb::ScalarFuncSig::CastJsonAsReal, "cast"}, + {tipb::ScalarFuncSig::CastJsonAsString, "cast"}, + {tipb::ScalarFuncSig::CastJsonAsDecimal, "cast"}, + {tipb::ScalarFuncSig::CastJsonAsTime, "cast"}, + {tipb::ScalarFuncSig::CastJsonAsDuration, "cast"}, + {tipb::ScalarFuncSig::CastJsonAsJson, "cast"}, + */ + + {tipb::ScalarFuncSig::CoalesceInt, "coalesce"}, + {tipb::ScalarFuncSig::CoalesceReal, "coalesce"}, + {tipb::ScalarFuncSig::CoalesceString, "coalesce"}, + {tipb::ScalarFuncSig::CoalesceDecimal, "coalesce"}, + {tipb::ScalarFuncSig::CoalesceTime, "coalesce"}, + {tipb::ScalarFuncSig::CoalesceDuration, "coalesce"}, + {tipb::ScalarFuncSig::CoalesceJson, "coalesce"}, + + {tipb::ScalarFuncSig::LTInt, "less"}, + {tipb::ScalarFuncSig::LTReal, "less"}, + {tipb::ScalarFuncSig::LTString, "less"}, + {tipb::ScalarFuncSig::LTDecimal, "less"}, + {tipb::ScalarFuncSig::LTTime, "less"}, + {tipb::ScalarFuncSig::LTDuration, "less"}, + {tipb::ScalarFuncSig::LTJson, "less"}, + + {tipb::ScalarFuncSig::LEInt, "lessOrEquals"}, + {tipb::ScalarFuncSig::LEReal, "lessOrEquals"}, + {tipb::ScalarFuncSig::LEString, "lessOrEquals"}, + {tipb::ScalarFuncSig::LEDecimal, "lessOrEquals"}, + {tipb::ScalarFuncSig::LETime, "lessOrEquals"}, + {tipb::ScalarFuncSig::LEDuration, "lessOrEquals"}, + {tipb::ScalarFuncSig::LEJson, "lessOrEquals"}, + + {tipb::ScalarFuncSig::GTInt, "greater"}, + {tipb::ScalarFuncSig::GTReal, "greater"}, + {tipb::ScalarFuncSig::GTString, "greater"}, + {tipb::ScalarFuncSig::GTDecimal, "greater"}, + {tipb::ScalarFuncSig::GTTime, "greater"}, + {tipb::ScalarFuncSig::GTDuration, "greater"}, + {tipb::ScalarFuncSig::GTJson, "greater"}, + + {tipb::ScalarFuncSig::GreatestInt, "greatest"}, + {tipb::ScalarFuncSig::GreatestReal, "greatest"}, + {tipb::ScalarFuncSig::GreatestString, "greatest"}, + {tipb::ScalarFuncSig::GreatestDecimal, "greatest"}, + {tipb::ScalarFuncSig::GreatestTime, "greatest"}, + + {tipb::ScalarFuncSig::LeastInt, "least"}, + {tipb::ScalarFuncSig::LeastReal, "least"}, + {tipb::ScalarFuncSig::LeastString, "least"}, + {tipb::ScalarFuncSig::LeastDecimal, "least"}, + {tipb::ScalarFuncSig::LeastTime, "least"}, + + //{tipb::ScalarFuncSig::IntervalInt, "cast"}, + //{tipb::ScalarFuncSig::IntervalReal, "cast"}, + + {tipb::ScalarFuncSig::GEInt, "greaterOrEquals"}, + {tipb::ScalarFuncSig::GEReal, "greaterOrEquals"}, + {tipb::ScalarFuncSig::GEString, "greaterOrEquals"}, + {tipb::ScalarFuncSig::GEDecimal, "greaterOrEquals"}, + {tipb::ScalarFuncSig::GETime, "greaterOrEquals"}, + {tipb::ScalarFuncSig::GEDuration, "greaterOrEquals"}, + {tipb::ScalarFuncSig::GEJson, "greaterOrEquals"}, + + {tipb::ScalarFuncSig::EQInt, "equals"}, + {tipb::ScalarFuncSig::EQReal, "equals"}, + {tipb::ScalarFuncSig::EQString, "equals"}, + {tipb::ScalarFuncSig::EQDecimal, "equals"}, + {tipb::ScalarFuncSig::EQTime, "equals"}, + {tipb::ScalarFuncSig::EQDuration, "equals"}, + {tipb::ScalarFuncSig::EQJson, "equals"}, + + {tipb::ScalarFuncSig::NEInt, "notEquals"}, + {tipb::ScalarFuncSig::NEReal, "notEquals"}, + {tipb::ScalarFuncSig::NEString, "notEquals"}, + {tipb::ScalarFuncSig::NEDecimal, "notEquals"}, + {tipb::ScalarFuncSig::NETime, "notEquals"}, + {tipb::ScalarFuncSig::NEDuration, "notEquals"}, + {tipb::ScalarFuncSig::NEJson, "notEquals"}, + + //{tipb::ScalarFuncSig::NullEQInt, "cast"}, + //{tipb::ScalarFuncSig::NullEQReal, "cast"}, + //{tipb::ScalarFuncSig::NullEQString, "cast"}, + //{tipb::ScalarFuncSig::NullEQDecimal, "cast"}, + //{tipb::ScalarFuncSig::NullEQTime, "cast"}, + //{tipb::ScalarFuncSig::NullEQDuration, "cast"}, + //{tipb::ScalarFuncSig::NullEQJson, "cast"}, + + {tipb::ScalarFuncSig::PlusReal, "plus"}, + {tipb::ScalarFuncSig::PlusDecimal, "plus"}, + {tipb::ScalarFuncSig::PlusInt, "plus"}, + + {tipb::ScalarFuncSig::MinusReal, "minus"}, + {tipb::ScalarFuncSig::MinusDecimal, "minus"}, + {tipb::ScalarFuncSig::MinusInt, "minus"}, + + {tipb::ScalarFuncSig::MultiplyReal, "multiply"}, + {tipb::ScalarFuncSig::MultiplyDecimal, "multiply"}, + {tipb::ScalarFuncSig::MultiplyInt, "multiply"}, + + {tipb::ScalarFuncSig::DivideReal, "divide"}, + {tipb::ScalarFuncSig::DivideDecimal, "divide"}, + {tipb::ScalarFuncSig::IntDivideInt, "intDiv"}, + {tipb::ScalarFuncSig::IntDivideDecimal, "divide"}, + + {tipb::ScalarFuncSig::ModReal, "modulo"}, + {tipb::ScalarFuncSig::ModDecimal, "modulo"}, + {tipb::ScalarFuncSig::ModInt, "modulo"}, + + {tipb::ScalarFuncSig::MultiplyIntUnsigned, "multiply"}, + + {tipb::ScalarFuncSig::AbsInt, "abs"}, + {tipb::ScalarFuncSig::AbsUInt, "abs"}, + {tipb::ScalarFuncSig::AbsReal, "abs"}, + {tipb::ScalarFuncSig::AbsDecimal, "abs"}, + + {tipb::ScalarFuncSig::CeilIntToDec, "ceil"}, + {tipb::ScalarFuncSig::CeilIntToInt, "ceil"}, + {tipb::ScalarFuncSig::CeilDecToInt, "ceil"}, + {tipb::ScalarFuncSig::CeilDecToDec, "ceil"}, + {tipb::ScalarFuncSig::CeilReal, "ceil"}, + + {tipb::ScalarFuncSig::FloorIntToDec, "floor"}, + {tipb::ScalarFuncSig::FloorIntToInt, "floor"}, + {tipb::ScalarFuncSig::FloorDecToInt, "floor"}, + {tipb::ScalarFuncSig::FloorDecToDec, "floor"}, + {tipb::ScalarFuncSig::FloorReal, "floor"}, + + {tipb::ScalarFuncSig::RoundReal, "round"}, + {tipb::ScalarFuncSig::RoundInt, "round"}, + {tipb::ScalarFuncSig::RoundDec, "round"}, + //{tipb::ScalarFuncSig::RoundWithFracReal, "cast"}, + //{tipb::ScalarFuncSig::RoundWithFracInt, "cast"}, + //{tipb::ScalarFuncSig::RoundWithFracDec, "cast"}, + + {tipb::ScalarFuncSig::Log1Arg, "log"}, + //{tipb::ScalarFuncSig::Log2Args, "cast"}, + {tipb::ScalarFuncSig::Log2, "log2"}, + {tipb::ScalarFuncSig::Log10, "log10"}, + + {tipb::ScalarFuncSig::Rand, "rand"}, + //{tipb::ScalarFuncSig::RandWithSeed, "cast"}, + + {tipb::ScalarFuncSig::Pow, "pow"}, + //{tipb::ScalarFuncSig::Conv, "cast"}, + //{tipb::ScalarFuncSig::CRC32, "cast"}, + //{tipb::ScalarFuncSig::Sign, "cast"}, + + {tipb::ScalarFuncSig::Sqrt, "sqrt"}, + {tipb::ScalarFuncSig::Acos, "acos"}, + {tipb::ScalarFuncSig::Asin, "asin"}, + {tipb::ScalarFuncSig::Atan1Arg, "atan"}, + //{tipb::ScalarFuncSig::Atan2Args, "cast"}, + {tipb::ScalarFuncSig::Cos, "cos"}, + //{tipb::ScalarFuncSig::Cot, "cast"}, + //{tipb::ScalarFuncSig::Degrees, "cast"}, + {tipb::ScalarFuncSig::Exp, "exp"}, + //{tipb::ScalarFuncSig::PI, "cast"}, + //{tipb::ScalarFuncSig::Radians, "cast"}, + {tipb::ScalarFuncSig::Sin, "sin"}, + {tipb::ScalarFuncSig::Tan, "tan"}, + {tipb::ScalarFuncSig::TruncateInt, "trunc"}, + {tipb::ScalarFuncSig::TruncateReal, "trunc"}, + //{tipb::ScalarFuncSig::TruncateDecimal, "cast"}, + + {tipb::ScalarFuncSig::LogicalAnd, "and"}, + {tipb::ScalarFuncSig::LogicalOr, "or"}, + {tipb::ScalarFuncSig::LogicalXor, "xor"}, + {tipb::ScalarFuncSig::UnaryNot, "not"}, + {tipb::ScalarFuncSig::UnaryMinusInt, "negate"}, + {tipb::ScalarFuncSig::UnaryMinusReal, "negate"}, + {tipb::ScalarFuncSig::UnaryMinusDecimal, "negate"}, + {tipb::ScalarFuncSig::DecimalIsNull, "isNull"}, + {tipb::ScalarFuncSig::DurationIsNull, "isNull"}, + {tipb::ScalarFuncSig::RealIsNull, "isNull"}, + {tipb::ScalarFuncSig::StringIsNull, "isNull"}, + {tipb::ScalarFuncSig::TimeIsNull, "isNull"}, + {tipb::ScalarFuncSig::IntIsNull, "isNull"}, + {tipb::ScalarFuncSig::JsonIsNull, "isNull"}, + + //{tipb::ScalarFuncSig::BitAndSig, "cast"}, + //{tipb::ScalarFuncSig::BitOrSig, "cast"}, + //{tipb::ScalarFuncSig::BitXorSig, "cast"}, + //{tipb::ScalarFuncSig::BitNegSig, "cast"}, + //{tipb::ScalarFuncSig::IntIsTrue, "cast"}, + //{tipb::ScalarFuncSig::RealIsTrue, "cast"}, + //{tipb::ScalarFuncSig::DecimalIsTrue, "cast"}, + //{tipb::ScalarFuncSig::IntIsFalse, "cast"}, + //{tipb::ScalarFuncSig::RealIsFalse, "cast"}, + //{tipb::ScalarFuncSig::DecimalIsFalse, "cast"}, + + //{tipb::ScalarFuncSig::LeftShift, "cast"}, + //{tipb::ScalarFuncSig::RightShift, "cast"}, + + //{tipb::ScalarFuncSig::BitCount, "cast"}, + //{tipb::ScalarFuncSig::GetParamString, "cast"}, + //{tipb::ScalarFuncSig::GetVar, "cast"}, + //{tipb::ScalarFuncSig::RowSig, "cast"}, + //{tipb::ScalarFuncSig::SetVar, "cast"}, + //{tipb::ScalarFuncSig::ValuesDecimal, "cast"}, + //{tipb::ScalarFuncSig::ValuesDuration, "cast"}, + //{tipb::ScalarFuncSig::ValuesInt, "cast"}, + //{tipb::ScalarFuncSig::ValuesJSON, "cast"}, + //{tipb::ScalarFuncSig::ValuesReal, "cast"}, + //{tipb::ScalarFuncSig::ValuesString, "cast"}, + //{tipb::ScalarFuncSig::ValuesTime, "cast"}, + + {tipb::ScalarFuncSig::InInt, "in"}, + {tipb::ScalarFuncSig::InReal, "in"}, + {tipb::ScalarFuncSig::InString, "in"}, + {tipb::ScalarFuncSig::InDecimal, "in"}, + {tipb::ScalarFuncSig::InTime, "in"}, + {tipb::ScalarFuncSig::InDuration, "in"}, + {tipb::ScalarFuncSig::InJson, "in"}, + + {tipb::ScalarFuncSig::IfNullInt, "ifNull"}, + {tipb::ScalarFuncSig::IfNullReal, "ifNull"}, + {tipb::ScalarFuncSig::IfNullString, "ifNull"}, + {tipb::ScalarFuncSig::IfNullDecimal, "ifNull"}, + {tipb::ScalarFuncSig::IfNullTime, "ifNull"}, + {tipb::ScalarFuncSig::IfNullDuration, "ifNull"}, + {tipb::ScalarFuncSig::IfNullJson, "ifNull"}, + + {tipb::ScalarFuncSig::IfInt, "if"}, + {tipb::ScalarFuncSig::IfReal, "if"}, + {tipb::ScalarFuncSig::IfString, "if"}, + {tipb::ScalarFuncSig::IfDecimal, "if"}, + {tipb::ScalarFuncSig::IfTime, "if"}, + {tipb::ScalarFuncSig::IfDuration, "if"}, + {tipb::ScalarFuncSig::IfJson, "if"}, + + //todo need further check for caseWithExpression and multiIf + //{tipb::ScalarFuncSig::CaseWhenInt, "caseWithExpression"}, + //{tipb::ScalarFuncSig::CaseWhenReal, "caseWithExpression"}, + //{tipb::ScalarFuncSig::CaseWhenString, "caseWithExpression"}, + //{tipb::ScalarFuncSig::CaseWhenDecimal, "caseWithExpression"}, + //{tipb::ScalarFuncSig::CaseWhenTime, "caseWithExpression"}, + //{tipb::ScalarFuncSig::CaseWhenDuration, "caseWithExpression"}, + //{tipb::ScalarFuncSig::CaseWhenJson, "caseWithExpression"}, + + //{tipb::ScalarFuncSig::AesDecrypt, "cast"}, + //{tipb::ScalarFuncSig::AesEncrypt, "cast"}, + //{tipb::ScalarFuncSig::Compress, "cast"}, + //{tipb::ScalarFuncSig::MD5, "cast"}, + //{tipb::ScalarFuncSig::Password, "cast"}, + //{tipb::ScalarFuncSig::RandomBytes, "cast"}, + //{tipb::ScalarFuncSig::SHA1, "cast"}, + //{tipb::ScalarFuncSig::SHA2, "cast"}, + //{tipb::ScalarFuncSig::Uncompress, "cast"}, + //{tipb::ScalarFuncSig::UncompressedLength, "cast"}, + + //{tipb::ScalarFuncSig::Database, "cast"}, + //{tipb::ScalarFuncSig::FoundRows, "cast"}, + //{tipb::ScalarFuncSig::CurrentUser, "cast"}, + //{tipb::ScalarFuncSig::User, "cast"}, + //{tipb::ScalarFuncSig::ConnectionID, "cast"}, + //{tipb::ScalarFuncSig::LastInsertID, "cast"}, + //{tipb::ScalarFuncSig::LastInsertIDWithID, "cast"}, + //{tipb::ScalarFuncSig::Version, "cast"}, + //{tipb::ScalarFuncSig::TiDBVersion, "cast"}, + //{tipb::ScalarFuncSig::RowCount, "cast"}, + + //{tipb::ScalarFuncSig::Sleep, "cast"}, + //{tipb::ScalarFuncSig::Lock, "cast"}, + //{tipb::ScalarFuncSig::ReleaseLock, "cast"}, + //{tipb::ScalarFuncSig::DecimalAnyValue, "cast"}, + //{tipb::ScalarFuncSig::DurationAnyValue, "cast"}, + //{tipb::ScalarFuncSig::IntAnyValue, "cast"}, + //{tipb::ScalarFuncSig::JSONAnyValue, "cast"}, + //{tipb::ScalarFuncSig::RealAnyValue, "cast"}, + //{tipb::ScalarFuncSig::StringAnyValue, "cast"}, + //{tipb::ScalarFuncSig::TimeAnyValue, "cast"}, + //{tipb::ScalarFuncSig::InetAton, "cast"}, + //{tipb::ScalarFuncSig::InetNtoa, "cast"}, + //{tipb::ScalarFuncSig::Inet6Aton, "cast"}, + //{tipb::ScalarFuncSig::Inet6Ntoa, "cast"}, + //{tipb::ScalarFuncSig::IsIPv4, "cast"}, + //{tipb::ScalarFuncSig::IsIPv4Compat, "cast"}, + //{tipb::ScalarFuncSig::IsIPv4Mapped, "cast"}, + //{tipb::ScalarFuncSig::IsIPv6, "cast"}, + //{tipb::ScalarFuncSig::UUID, "cast"}, + + {tipb::ScalarFuncSig::LikeSig, "like3Args"}, + //{tipb::ScalarFuncSig::RegexpBinarySig, "cast"}, + //{tipb::ScalarFuncSig::RegexpSig, "cast"}, + + //{tipb::ScalarFuncSig::JsonExtractSig, "cast"}, + //{tipb::ScalarFuncSig::JsonUnquoteSig, "cast"}, + //{tipb::ScalarFuncSig::JsonTypeSig, "cast"}, + //{tipb::ScalarFuncSig::JsonSetSig, "cast"}, + //{tipb::ScalarFuncSig::JsonInsertSig, "cast"}, + //{tipb::ScalarFuncSig::JsonReplaceSig, "cast"}, + //{tipb::ScalarFuncSig::JsonRemoveSig, "cast"}, + //{tipb::ScalarFuncSig::JsonMergeSig, "cast"}, + //{tipb::ScalarFuncSig::JsonObjectSig, "cast"}, + //{tipb::ScalarFuncSig::JsonArraySig, "cast"}, + //{tipb::ScalarFuncSig::JsonValidJsonSig, "cast"}, + //{tipb::ScalarFuncSig::JsonContainsSig, "cast"}, + //{tipb::ScalarFuncSig::JsonArrayAppendSig, "cast"}, + //{tipb::ScalarFuncSig::JsonArrayInsertSig, "cast"}, + //{tipb::ScalarFuncSig::JsonMergePatchSig, "cast"}, + //{tipb::ScalarFuncSig::JsonMergePreserveSig, "cast"}, + //{tipb::ScalarFuncSig::JsonContainsPathSig, "cast"}, + //{tipb::ScalarFuncSig::JsonPrettySig, "cast"}, + //{tipb::ScalarFuncSig::JsonQuoteSig, "cast"}, + //{tipb::ScalarFuncSig::JsonSearchSig, "cast"}, + //{tipb::ScalarFuncSig::JsonStorageSizeSig, "cast"}, + //{tipb::ScalarFuncSig::JsonDepthSig, "cast"}, + //{tipb::ScalarFuncSig::JsonKeysSig, "cast"}, + //{tipb::ScalarFuncSig::JsonLengthSig, "cast"}, + //{tipb::ScalarFuncSig::JsonKeys2ArgsSig, "cast"}, + //{tipb::ScalarFuncSig::JsonValidStringSig, "cast"}, + + //{tipb::ScalarFuncSig::DateFormatSig, "cast"}, + //{tipb::ScalarFuncSig::DateLiteral, "cast"}, + //{tipb::ScalarFuncSig::DateDiff, "cast"}, + //{tipb::ScalarFuncSig::NullTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::TimeStringTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::DurationDurationTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::DurationDurationTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::StringTimeTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::StringDurationTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::StringStringTimeDiff, "cast"}, + //{tipb::ScalarFuncSig::TimeTimeTimeDiff, "cast"}, + + //{tipb::ScalarFuncSig::Date, "cast"}, + //{tipb::ScalarFuncSig::Hour, "cast"}, + //{tipb::ScalarFuncSig::Minute, "cast"}, + //{tipb::ScalarFuncSig::Second, "cast"}, + //{tipb::ScalarFuncSig::MicroSecond, "cast"}, + //{tipb::ScalarFuncSig::Month, "cast"}, + //{tipb::ScalarFuncSig::MonthName, "cast"}, + + //{tipb::ScalarFuncSig::NowWithArg, "cast"}, + //{tipb::ScalarFuncSig::NowWithoutArg, "cast"}, + + //{tipb::ScalarFuncSig::DayName, "cast"}, + //{tipb::ScalarFuncSig::DayOfMonth, "cast"}, + //{tipb::ScalarFuncSig::DayOfWeek, "cast"}, + //{tipb::ScalarFuncSig::DayOfYear, "cast"}, + + //{tipb::ScalarFuncSig::WeekWithMode, "cast"}, + //{tipb::ScalarFuncSig::WeekWithoutMode, "cast"}, + //{tipb::ScalarFuncSig::WeekDay, "cast"}, + //{tipb::ScalarFuncSig::WeekOfYear, "cast"}, + + //{tipb::ScalarFuncSig::Year, "cast"}, + //{tipb::ScalarFuncSig::YearWeekWithMode, "cast"}, + //{tipb::ScalarFuncSig::YearWeekWithoutMode, "cast"}, + + //{tipb::ScalarFuncSig::GetFormat, "cast"}, + //{tipb::ScalarFuncSig::SysDateWithFsp, "cast"}, + //{tipb::ScalarFuncSig::SysDateWithoutFsp, "cast"}, + //{tipb::ScalarFuncSig::CurrentDate, "cast"}, + //{tipb::ScalarFuncSig::CurrentTime0Arg, "cast"}, + //{tipb::ScalarFuncSig::CurrentTime1Arg, "cast"}, + + //{tipb::ScalarFuncSig::Time, "cast"}, + //{tipb::ScalarFuncSig::TimeLiteral, "cast"}, + //{tipb::ScalarFuncSig::UTCDate, "cast"}, + //{tipb::ScalarFuncSig::UTCTimestampWithArg, "cast"}, + //{tipb::ScalarFuncSig::UTCTimestampWithoutArg, "cast"}, + + //{tipb::ScalarFuncSig::AddDatetimeAndDuration, "cast"}, + //{tipb::ScalarFuncSig::AddDatetimeAndString, "cast"}, + //{tipb::ScalarFuncSig::AddTimeDateTimeNull, "cast"}, + //{tipb::ScalarFuncSig::AddStringAndDuration, "cast"}, + //{tipb::ScalarFuncSig::AddStringAndString, "cast"}, + //{tipb::ScalarFuncSig::AddTimeStringNull, "cast"}, + //{tipb::ScalarFuncSig::AddDurationAndDuration, "cast"}, + //{tipb::ScalarFuncSig::AddDurationAndString, "cast"}, + //{tipb::ScalarFuncSig::AddTimeDurationNull, "cast"}, + //{tipb::ScalarFuncSig::AddDateAndDuration, "cast"}, + //{tipb::ScalarFuncSig::AddDateAndString, "cast"}, + + //{tipb::ScalarFuncSig::SubDateAndDuration, "cast"}, + //{tipb::ScalarFuncSig::SubDateAndString, "cast"}, + //{tipb::ScalarFuncSig::SubTimeDateTimeNull, "cast"}, + //{tipb::ScalarFuncSig::SubStringAndDuration, "cast"}, + //{tipb::ScalarFuncSig::SubStringAndString, "cast"}, + //{tipb::ScalarFuncSig::SubTimeStringNull, "cast"}, + //{tipb::ScalarFuncSig::SubDurationAndDuration, "cast"}, + //{tipb::ScalarFuncSig::SubDurationAndString, "cast"}, + //{tipb::ScalarFuncSig::SubDateAndDuration, "cast"}, + //{tipb::ScalarFuncSig::SubDateAndString, "cast"}, + + //{tipb::ScalarFuncSig::UnixTimestampCurrent, "cast"}, + //{tipb::ScalarFuncSig::UnixTimestampInt, "cast"}, + //{tipb::ScalarFuncSig::UnixTimestampDec, "cast"}, + + //{tipb::ScalarFuncSig::ConvertTz, "cast"}, + //{tipb::ScalarFuncSig::MakeDate, "cast"}, + //{tipb::ScalarFuncSig::MakeTime, "cast"}, + //{tipb::ScalarFuncSig::PeriodAdd, "cast"}, + //{tipb::ScalarFuncSig::PeriodDiff, "cast"}, + //{tipb::ScalarFuncSig::Quarter, "cast"}, + + //{tipb::ScalarFuncSig::SecToTime, "cast"}, + //{tipb::ScalarFuncSig::TimeToSec, "cast"}, + //{tipb::ScalarFuncSig::TimestampAdd, "cast"}, + //{tipb::ScalarFuncSig::ToDays, "cast"}, + //{tipb::ScalarFuncSig::ToSeconds, "cast"}, + //{tipb::ScalarFuncSig::UTCTimeWithArg, "cast"}, + //{tipb::ScalarFuncSig::UTCTimestampWithoutArg, "cast"}, + //{tipb::ScalarFuncSig::Timestamp1Arg, "cast"}, + //{tipb::ScalarFuncSig::Timestamp2Args, "cast"}, + //{tipb::ScalarFuncSig::TimestampLiteral, "cast"}, + + //{tipb::ScalarFuncSig::LastDay, "cast"}, + //{tipb::ScalarFuncSig::StrToDateDate, "cast"}, + //{tipb::ScalarFuncSig::StrToDateDatetime, "cast"}, + //{tipb::ScalarFuncSig::StrToDateDuration, "cast"}, + //{tipb::ScalarFuncSig::FromUnixTime1Arg, "cast"}, + //{tipb::ScalarFuncSig::FromUnixTime2Arg, "cast"}, + //{tipb::ScalarFuncSig::ExtractDatetime, "cast"}, + //{tipb::ScalarFuncSig::ExtractDuration, "cast"}, + + //{tipb::ScalarFuncSig::AddDateStringString, "cast"}, + //{tipb::ScalarFuncSig::AddDateStringInt, "cast"}, + //{tipb::ScalarFuncSig::AddDateStringDecimal, "cast"}, + //{tipb::ScalarFuncSig::AddDateIntString, "cast"}, + //{tipb::ScalarFuncSig::AddDateIntInt, "cast"}, + //{tipb::ScalarFuncSig::AddDateDatetimeString, "cast"}, + //{tipb::ScalarFuncSig::AddDateDatetimeInt, "cast"}, + + //{tipb::ScalarFuncSig::SubDateStringString, "cast"}, + //{tipb::ScalarFuncSig::SubDateStringInt, "cast"}, + //{tipb::ScalarFuncSig::SubDateStringDecimal, "cast"}, + //{tipb::ScalarFuncSig::SubDateIntString, "cast"}, + //{tipb::ScalarFuncSig::SubDateIntInt, "cast"}, + //{tipb::ScalarFuncSig::SubDateDatetimeString, "cast"}, + //{tipb::ScalarFuncSig::SubDateDatetimeInt, "cast"}, + + //{tipb::ScalarFuncSig::FromDays, "cast"}, + //{tipb::ScalarFuncSig::TimeFormat, "cast"}, + //{tipb::ScalarFuncSig::TimestampDiff, "cast"}, + + //{tipb::ScalarFuncSig::BitLength, "cast"}, + //{tipb::ScalarFuncSig::Bin, "cast"}, + //{tipb::ScalarFuncSig::ASCII, "cast"}, + //{tipb::ScalarFuncSig::Char, "cast"}, + {tipb::ScalarFuncSig::CharLength, "lengthUTF8"}, + //{tipb::ScalarFuncSig::Concat, "cast"}, + //{tipb::ScalarFuncSig::ConcatWS, "cast"}, + //{tipb::ScalarFuncSig::Convert, "cast"}, + //{tipb::ScalarFuncSig::Elt, "cast"}, + //{tipb::ScalarFuncSig::ExportSet3Arg, "cast"}, + //{tipb::ScalarFuncSig::ExportSet4Arg, "cast"}, + //{tipb::ScalarFuncSig::ExportSet5Arg, "cast"}, + //{tipb::ScalarFuncSig::FieldInt, "cast"}, + //{tipb::ScalarFuncSig::FieldReal, "cast"}, + //{tipb::ScalarFuncSig::FieldString, "cast"}, + + //{tipb::ScalarFuncSig::FindInSet, "cast"}, + //{tipb::ScalarFuncSig::Format, "cast"}, + //{tipb::ScalarFuncSig::FormatWithLocale, "cast"}, + //{tipb::ScalarFuncSig::FromBase64, "cast"}, + //{tipb::ScalarFuncSig::HexIntArg, "cast"}, + //{tipb::ScalarFuncSig::HexStrArg, "cast"}, + //{tipb::ScalarFuncSig::Insert, "cast"}, + //{tipb::ScalarFuncSig::InsertBinary, "cast"}, + //{tipb::ScalarFuncSig::Instr, "cast"}, + //{tipb::ScalarFuncSig::InstrBinary, "cast"}, + + {tipb::ScalarFuncSig::LTrim, "ltrim"}, + //{tipb::ScalarFuncSig::Left, "cast"}, + //{tipb::ScalarFuncSig::LeftBinary, "cast"}, + {tipb::ScalarFuncSig::Length, "length"}, + //{tipb::ScalarFuncSig::Locate2Args, "cast"}, + //{tipb::ScalarFuncSig::Locate3Args, "cast"}, + //{tipb::ScalarFuncSig::LocateBinary2Args, "cast"}, + //{tipb::ScalarFuncSig::LocateBinary3Args, "cast"}, + + {tipb::ScalarFuncSig::Lower, "lower"}, + //{tipb::ScalarFuncSig::Lpad, "cast"}, + //{tipb::ScalarFuncSig::LpadBinary, "cast"}, + //{tipb::ScalarFuncSig::MakeSet, "cast"}, + //{tipb::ScalarFuncSig::OctInt, "cast"}, + //{tipb::ScalarFuncSig::OctString, "cast"}, + //{tipb::ScalarFuncSig::Ord, "cast"}, + //{tipb::ScalarFuncSig::Quote, "cast"}, + {tipb::ScalarFuncSig::RTrim, "rtrim"}, + //{tipb::ScalarFuncSig::Repeat, "cast"}, + //{tipb::ScalarFuncSig::Replace, "cast"}, + //{tipb::ScalarFuncSig::Reverse, "cast"}, + //{tipb::ScalarFuncSig::ReverseBinary, "cast"}, + //{tipb::ScalarFuncSig::Right, "cast"}, + //{tipb::ScalarFuncSig::RightBinary, "cast"}, + //{tipb::ScalarFuncSig::Rpad, "cast"}, + //{tipb::ScalarFuncSig::RpadBinary, "cast"}, + //{tipb::ScalarFuncSig::Space, "cast"}, + //{tipb::ScalarFuncSig::Strcmp, "cast"}, + //{tipb::ScalarFuncSig::Substring2Args, "cast"}, + //{tipb::ScalarFuncSig::Substring3Args, "cast"}, + //{tipb::ScalarFuncSig::SubstringBinary2Args, "cast"}, + //{tipb::ScalarFuncSig::SubstringBinary3Args, "cast"}, + //{tipb::ScalarFuncSig::SubstringIndex, "cast"}, + + //{tipb::ScalarFuncSig::ToBase64, "cast"}, + //{tipb::ScalarFuncSig::Trim1Arg, "cast"}, + //{tipb::ScalarFuncSig::Trim2Args, "cast"}, + //{tipb::ScalarFuncSig::Trim3Args, "cast"}, + //{tipb::ScalarFuncSig::UnHex, "cast"}, + {tipb::ScalarFuncSig::Upper, "upper"}, +}); + +tipb::FieldType columnInfoToFieldType(const TiDB::ColumnInfo & ci) +{ + tipb::FieldType ret; + ret.set_tp(ci.tp); + ret.set_flag(ci.flag); + ret.set_flen(ci.flen); + ret.set_decimal(ci.decimal); + return ret; +} + +TiDB::ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type) +{ + TiDB::ColumnInfo ret; + ret.tp = static_cast(field_type.tp()); + ret.flag = field_type.flag(); + ret.flen = field_type.flen(); + ret.decimal = field_type.decimal(); + return ret; +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h new file mode 100644 index 00000000000..b45c12680c3 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -0,0 +1,36 @@ +#pragma once + +#include + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +#include +#include +#include +#include + +namespace DB +{ + +bool isLiteralExpr(const tipb::Expr & expr); +Field decodeLiteral(const tipb::Expr & expr); +bool isFunctionExpr(const tipb::Expr & expr); +bool isAggFunctionExpr(const tipb::Expr & expr); +const String & getFunctionName(const tipb::Expr & expr); +const String & getAggFunctionName(const tipb::Expr & expr); +bool isColumnExpr(const tipb::Expr & expr); +String getColumnNameForColumnExpr(const tipb::Expr & expr, const std::vector & input_col); +const String & getTypeName(const tipb::Expr & expr); +String exprToString(const tipb::Expr & expr, const std::vector & input_col); +bool isInOrGlobalInOperator(const String & name); +bool exprHasValidFieldType(const tipb::Expr & expr); +extern std::unordered_map agg_func_map; +extern std::unordered_map scalar_func_map; + +tipb::FieldType columnInfoToFieldType(const TiDB::ColumnInfo & ci); +TiDB::ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp new file mode 100644 index 00000000000..f0321b3bd7b --- /dev/null +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -0,0 +1,666 @@ +#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 + +InterpreterDAG::InterpreterDAG(Context & context_, const DAGQuerySource & dag_) + : context(context_), dag(dag_), log(&Logger::get("InterpreterDAG")) +{} + +template +bool isAllValueCoveredByRanges(std::vector> & ranges, const std::vector> & region_ranges) +{ + if (ranges.empty()) + 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 + { + 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; + } + } + merged_ranges.emplace_back(std::make_pair(merged_range.first, merged_range.second)); + + 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) + return false; + } + return true; +} + +bool checkKeyRanges(const std::vector> & key_ranges, TableID table_id, bool pk_is_uint64, + const ImutRegionRangePtr & region_key_range) +{ + if (key_ranges.empty()) + return true; + + 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 isAllValueCoveredByRanges(update_handle_ranges, update_region_handle_ranges); + } + else + return isAllValueCoveredByRanges(handle_ranges, region_handle_ranges); +} +// the flow is the same as executeFetchcolumns +void InterpreterDAG::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(); + // TODO: Get schema version from DAG request. + if (context.getSettingsRef().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, DEFAULT_UNSPECIFIED_SCHEMA_VERSION); + } + + Names required_columns; + std::vector source_columns; + std::vector is_ts_column; + for (const tipb::ColumnInfo & ci : ts.columns()) + { + ColumnID cid = ci.column_id(); + + if (cid == -1) + // Column ID -1 means TiDB expects no specific column, mostly it is for cases like `select count(*)`. + // This means we can return whatever column, we'll choose it later if no other columns are specified either. + continue; + + String name = storage->getTableInfo().getColumnName(cid); + required_columns.push_back(name); + auto pair = storage->getColumns().getPhysical(name); + source_columns.emplace_back(std::move(pair)); + is_ts_column.push_back(ci.tp() == TiDB::TypeTimestamp); + } + if (required_columns.empty()) + { + // No column specified, we choose the handle column as it will be emitted by storage read anyhow. + // Set `void` column field type correspondingly for further needs, i.e. encoding results. + if (auto pk_handle_col = storage->getTableInfo().getPKHandleColumn()) + { + required_columns.push_back(pk_handle_col->get().name); + auto pair = storage->getColumns().getPhysical(pk_handle_col->get().name); + source_columns.push_back(pair); + is_ts_column.push_back(false); + // For PK handle, use original column info of itself. + dag.getDAGContext().void_result_ft = columnInfoToFieldType(pk_handle_col->get()); + } + else + { + required_columns.push_back(MutableSupport::tidb_pk_column_name); + auto pair = storage->getColumns().getPhysical(MutableSupport::tidb_pk_column_name); + source_columns.push_back(pair); + is_ts_column.push_back(false); + // For implicit handle, reverse get a column info. + auto column_info = reverseGetColumnInfo(pair, -1, Field()); + dag.getDAGContext().void_result_ft = columnInfoToFieldType(column_info); + } + } + + analyzer = std::make_unique(std::move(source_columns), context); + + if (!dag.hasAggregation()) + { + // if the dag request does not contain agg, then the final output is + // based on the output of table scan + for (auto i : dag.getDAGRequest().output_offsets()) + { + if (i >= required_columns.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 + const Settings & settings = context.getSettingsRef(); + + 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 (dag.hasSelection()) + { + for (auto & condition : dag.getSelection().conditions()) + { + analyzer->makeExplicitSetForIndex(condition, storage); + } + } + //todo support index in + SelectQueryInfo query_info; + // set query to avoid unexpected NPE + query_info.query = dag.getAST(); + query_info.dag_query = std::make_unique(dag, 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 = dag.getRegionID(); + info.version = dag.getRegionVersion(); + info.conf_version = dag.getRegionConfVersion(); + auto current_region = context.getTMTContext().getKVStore()->getRegion(info.region_id); + if (!current_region) + { + std::vector region_ids; + region_ids.push_back(info.region_id); + throw RegionException(std::move(region_ids), RegionTable::RegionReadStatus::NOT_FOUND); + } + if (!checkKeyRanges(dag.getKeyRanges(), table_id, storage->pkIsUInt64(), current_region->getRange())) + 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 = 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); + } + }); + } + + addTimeZoneCastAfterTS(is_ts_column, pipeline); +} + +// add timezone cast for timestamp type, this is used to support session level timezone +void InterpreterDAG::addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline) +{ + bool hasTSColumn = false; + for (auto b : is_ts_column) + hasTSColumn |= b; + if (!hasTSColumn) + return; + + ExpressionActionsChain chain; + if (analyzer->appendTimeZoneCastsAfterTS(chain, is_ts_column, dag.getDAGRequest())) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, chain.getLastActions()); }); +} + +InterpreterDAG::AnalysisResult InterpreterDAG::analyzeExpressions() +{ + AnalysisResult res; + ExpressionActionsChain chain; + if (dag.hasSelection()) + { + analyzer->appendWhere(chain, dag.getSelection(), res.filter_column_name); + res.has_where = true; + res.before_where = chain.getLastActions(); + chain.addStep(); + } + // There will be either Agg... + if (dag.hasAggregation()) + { + analyzer->appendAggregation(chain, dag.getAggregation(), 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, dag.getAggregation(), dag.getDAGRequest()); + //todo use output_offset to reconstruct the final project columns + for (auto element : analyzer->getCurrentInputColumns()) + { + final_project.emplace_back(element.name, ""); + } + } + // Or TopN, not both. + if (dag.hasTopN()) + { + res.has_order_by = true; + analyzer->appendOrderBy(chain, dag.getTopN(), res.order_column_names); + } + // Append final project results if needed. + analyzer->appendFinalProject(chain, final_project); + res.before_order_and_select = chain.getLastActions(); + chain.finalize(); + chain.clear(); + //todo need call prependProjectInput?? + return res; +} + +void InterpreterDAG::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expr, String & filter_column) +{ + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expr, filter_column); }); +} + +void InterpreterDAG::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 InterpreterDAG::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr) +{ + if (!expressionActionsPtr->getActions().empty()) + { + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expressionActionsPtr); }); + } +} + +void InterpreterDAG::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_->getData().merging_params.mode != MergeTreeData::MergingParams::Txn) + throw Exception("Specifying schema_version for non-TMT storage: " + storage_->getName() + ", table: " + std::to_string(table_id) + + " is not allowed", + ErrorCodes::LOGICAL_ERROR); + + /// Lock storage. + auto lock = storage_->lockStructure(false, __PRETTY_FUNCTION__); + + /// Check schema version. + auto storage_schema_version = storage_->getTableInfo().schema_version; + // Not allow storage schema version greater than query schema version in any case. + 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); + + // If schema synced, we must be very recent so we are good as long as storage schema version is no greater than query schema version. + // If schema not synced, we are good if storage schema version is right on query schema version. + // Otherwise we are at the risk of out-of-date schema, but we still have a chance to be sure that we are good, if global schema version is greater than query schema version. + if ((schema_synced && storage_schema_version <= query_schema_version) + || (!schema_synced && (storage_schema_version == query_schema_version || global_schema_version > query_schema_version))) + return std::make_tuple(storage_, lock, storage_schema_version, true); + + return std::make_tuple(nullptr, nullptr, storage_schema_version, false); + }; + + /// Try get storage and lock once. + TMTStoragePtr 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 InterpreterDAG::getSortDescription(Strings & order_column_names) +{ + // construct SortDescription + SortDescription order_descr; + const tipb::TopN & topn = dag.getTopN(); + 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 InterpreterDAG::executeUnion(Pipeline & pipeline) +{ + if (pipeline.hasMoreThanOneStream()) + { + pipeline.firstStream() = std::make_shared>(pipeline.streams, nullptr, max_streams); + pipeline.streams.resize(1); + } +} + +void InterpreterDAG::executeOrder(Pipeline & pipeline, Strings & order_column_names) +{ + SortDescription order_descr = getSortDescription(order_column_names); + const Settings & settings = context.getSettingsRef(); + Int64 limit = dag.getTopN().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 InterpreterDAG::recordProfileStreams(Pipeline & pipeline, Int32 index) +{ + for (auto & stream : pipeline.streams) + { + dag.getDAGContext().profile_streams_list[index].push_back(stream); + } +} + +void InterpreterDAG::executeImpl(Pipeline & pipeline) +{ + executeTS(dag.getTS(), pipeline); + recordProfileStreams(pipeline, dag.getTSIndex()); + + auto res = analyzeExpressions(); + // execute selection + if (res.has_where) + { + executeWhere(pipeline, res.before_where, res.filter_column_name); + 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()); + } + + // execute projection + executeFinalProject(pipeline); + + // execute limit + if (dag.hasLimit() && !dag.hasTopN()) + { + executeLimit(pipeline); + recordProfileStreams(pipeline, dag.getLimitIndex()); + } +} + +void InterpreterDAG::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 InterpreterDAG::executeLimit(Pipeline & pipeline) +{ + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, dag.getLimit().limit(), 0, false); }); + if (pipeline.hasMoreThanOneStream()) + { + executeUnion(pipeline); + pipeline.transform( + [&](auto & stream) { stream = std::make_shared(stream, dag.getLimit().limit(), 0, false); }); + } +} + +BlockIO InterpreterDAG::execute() +{ + Pipeline pipeline; + executeImpl(pipeline); + executeUnion(pipeline); + + BlockIO res; + res.in = pipeline.firstStream(); + + LOG_DEBUG( + log, __PRETTY_FUNCTION__ << " Convert DAG request to BlockIO, adding " << analyzer->getImplicitCastCount() << " implicit cast"); + return res; +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h new file mode 100644 index 00000000000..8a5b7dfe76a --- /dev/null +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -0,0 +1,106 @@ +#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 + +namespace DB +{ + +class Context; + +/** build ch plan from dag request: dag executors -> ch plan + */ +class InterpreterDAG : public IInterpreter +{ +public: + InterpreterDAG(Context & context_, const DAGQuerySource & dag_); + + ~InterpreterDAG() = default; + + 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; + }; + + 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); + void addTimeZoneCastAfterTS(std::vector & is_ts_column, Pipeline & pipeline); + +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. + size_t max_streams = 1; + + /// Table from where to read data, if not subquery. + TMTStoragePtr storage; + TableStructureReadLockPtr table_lock; + + std::unique_ptr analyzer; + + Poco::Logger * log; +}; +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/tests/CMakeLists.txt b/dbms/src/Flash/Coprocessor/tests/CMakeLists.txt new file mode 100644 index 00000000000..b8e4b57cbca --- /dev/null +++ b/dbms/src/Flash/Coprocessor/tests/CMakeLists.txt @@ -0,0 +1 @@ +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp new file mode 100644 index 00000000000..c45890e29ce --- /dev/null +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -0,0 +1,124 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} + +CoprocessorHandler::CoprocessorHandler( + CoprocessorContext & cop_context_, const coprocessor::Request * cop_request_, coprocessor::Response * cop_response_) + : cop_context(cop_context_), cop_request(cop_request_), cop_response(cop_response_), log(&Logger::get("CoprocessorHandler")) +{} + +grpc::Status CoprocessorHandler::execute() +try +{ + switch (cop_request->tp()) + { + case COP_REQ_TYPE_DAG: + { + std::vector> key_ranges; + for (auto & range : cop_request->ranges()) + { + 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))); + } + tipb::DAGRequest dag_request; + dag_request.ParseFromString(cop_request->data()); + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); + 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::move(key_ranges), + dag_response); + driver.execute(); + cop_response->set_data(dag_response.SerializeAsString()); + 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: " << e.getStackTrace().toString()); + cop_response->Clear(); + kvrpcpb::LockInfo * lock_info = cop_response->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); + // 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: " << e.getStackTrace().toString()); + cop_response->Clear(); + errorpb::Error * region_err; + switch (e.status) + { + case RegionTable::RegionReadStatus::NOT_FOUND: + case RegionTable::RegionReadStatus::PENDING_REMOVE: + region_err = cop_response->mutable_region_error(); + region_err->mutable_region_not_found()->set_region_id(cop_request->context().region_id()); + break; + case RegionTable::RegionReadStatus::VERSION_ERROR: + region_err = cop_response->mutable_region_error(); + region_err->mutable_epoch_not_match(); + break; + default: + // should not happen + break; + } + // return ok so TiDB has the chance to see the LockException + return grpc::Status::OK; +} +catch (const Exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": Exception: " << e.getStackTrace().toString()); + cop_response->Clear(); + cop_response->set_other_error(e.message()); + + if (e.code() == ErrorCodes::NOT_IMPLEMENTED) + return ::grpc::Status(::grpc::StatusCode::UNIMPLEMENTED, e.message()); + + // TODO: Map other DB error codes to grpc codes. + + return grpc::Status(grpc::StatusCode::INTERNAL, e.message()); +} +catch (const std::exception & e) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": Exception: " << e.what()); + cop_response->Clear(); + cop_response->set_other_error(e.what()); + return grpc::Status(grpc::StatusCode::INTERNAL, e.what()); +} +catch (...) +{ + LOG_ERROR(log, __PRETTY_FUNCTION__ << ": catch other exception."); + cop_response->Clear(); + cop_response->set_other_error("other exception"); + return grpc::Status(grpc::StatusCode::INTERNAL, "other exception"); +} + +} // namespace DB diff --git a/dbms/src/Flash/CoprocessorHandler.h b/dbms/src/Flash/CoprocessorHandler.h new file mode 100644 index 00000000000..900d9d77fbe --- /dev/null +++ b/dbms/src/Flash/CoprocessorHandler.h @@ -0,0 +1,55 @@ +#pragma once + +#include + +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop +#include + +namespace DB +{ + +struct CoprocessorContext +{ + Context & db_context; + const kvrpcpb::Context & kv_context; + const grpc::ServerContext & grpc_server_context; + + CoprocessorContext(Context & db_context_, const kvrpcpb::Context & kv_context_, const grpc::ServerContext & grpc_server_context_) + : db_context(db_context_), kv_context(kv_context_), grpc_server_context(grpc_server_context_) + {} +}; + +/// Coprocessor request handler, deals with: +/// 1. DAG request: WIP; +/// 2. Analyze request: NOT IMPLEMENTED; +/// 3. Checksum request: NOT IMPLEMENTED; +class CoprocessorHandler +{ +public: + CoprocessorHandler(CoprocessorContext & cop_context_, const coprocessor::Request * cop_request_, coprocessor::Response * response_); + + ~CoprocessorHandler() = default; + + grpc::Status execute(); + +protected: + enum + { + COP_REQ_TYPE_DAG = 103, + COP_REQ_TYPE_ANALYZE = 104, + COP_REQ_TYPE_CHECKSUM = 105, + }; + + CoprocessorContext & cop_context; + const coprocessor::Request * cop_request; + coprocessor::Response * cop_response; + + Logger * log; +}; + +} // namespace DB diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp new file mode 100644 index 00000000000..5a404b0d5f2 --- /dev/null +++ b/dbms/src/Flash/FlashService.cpp @@ -0,0 +1,123 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} + +FlashService::FlashService(IServer & server_) : server(server_), log(&Logger::get("FlashService")) {} + +grpc::Status FlashService::Coprocessor( + grpc::ServerContext * grpc_context, const coprocessor::Request * request, coprocessor::Response * response) +{ + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling coprocessor request: " << request->DebugString()); + + auto [context, status] = createDBContext(grpc_context); + if (!status.ok()) + { + return status; + } + + CoprocessorContext cop_context(context, request->context(), *grpc_context); + CoprocessorHandler cop_handler(cop_context, request, response); + + 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) +{ + auto [context, status] = createDBContext(grpc_context); + if (!status.ok()) + { + return status; + } + + tikvpb::BatchCommandsRequest request; + while (stream->Read(&request)) + { + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling batch commands: " << request.DebugString()); + + tikvpb::BatchCommandsResponse response; + BatchCommandsContext batch_commands_context( + context, [this](const grpc::ServerContext * grpc_server_context) { return createDBContext(grpc_server_context); }, + *grpc_context); + BatchCommandsHandler batch_commands_handler(batch_commands_context, request, response); + auto ret = batch_commands_handler.execute(); + if (!ret.ok()) + { + LOG_DEBUG( + log, __PRETTY_FUNCTION__ << ": Handle batch commands request done: " << ret.error_code() << ", " << ret.error_message()); + return ret; + } + + if (!stream->Write(response)) + { + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Write response failed for unknown reason."); + return grpc::Status(grpc::StatusCode::UNKNOWN, "Write response failed for unknown reason."); + } + + LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handle batch commands request done: " << ret.error_code() << ", " << ret.error_message()); + } + + return grpc::Status::OK; +} + +String getClientMetaVarWithDefault(const grpc::ServerContext * grpc_context, const String & name, const String & default_val) +{ + if (auto it = grpc_context->client_metadata().find(name); it != grpc_context->client_metadata().end()) + return it->second.data(); + return default_val; +} + +std::tuple FlashService::createDBContext(const grpc::ServerContext * grpc_context) const +{ + /// Create DB context. + Context context = server.context(); + context.setGlobalContext(server.context()); + + /// Set a bunch of client information. + String query_id = getClientMetaVarWithDefault(grpc_context, "query_id", ""); + context.setCurrentQueryId(query_id); + ClientInfo & client_info = context.getClientInfo(); + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::GRPC; + std::string peer = grpc_context->peer(); + Int64 pos = peer.find(':'); + if (pos == -1) + { + return std::make_tuple(context, ::grpc::Status(::grpc::StatusCode::INVALID_ARGUMENT, "Invalid peer address: " + peer)); + } + std::string client_ip = peer.substr(pos + 1); + Poco::Net::SocketAddress client_address(client_ip); + client_info.current_address = client_address; + client_info.current_user = getClientMetaVarWithDefault(grpc_context, "user", ""); + + /// Set DAG parameters. + std::string dag_records_per_chunk_str = getClientMetaVarWithDefault(grpc_context, "dag_records_per_chunk", ""); + if (!dag_records_per_chunk_str.empty()) + { + context.setSetting("dag_records_per_chunk", dag_records_per_chunk_str); + } + std::string planner = getClientMetaVarWithDefault(grpc_context, "dag_planner", "optree"); + context.setSetting("dag_planner", planner); + std::string expr_field_type_check = getClientMetaVarWithDefault(grpc_context, "dag_expr_field_type_strict_check", "1"); + context.setSetting("dag_expr_field_type_strict_check", expr_field_type_check); + + return std::make_tuple(context, grpc::Status::OK); +} + +} // namespace DB diff --git a/dbms/src/Flash/FlashService.h b/dbms/src/Flash/FlashService.h new file mode 100644 index 00000000000..09e1640ab23 --- /dev/null +++ b/dbms/src/Flash/FlashService.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +namespace DB +{ + +class IServer; + +class FlashService final : public tikvpb::Tikv::Service, public std::enable_shared_from_this, private boost::noncopyable +{ +public: + explicit FlashService(IServer & server_); + + grpc::Status Coprocessor( + grpc::ServerContext * grpc_context, const coprocessor::Request * request, coprocessor::Response * response) override; + + grpc::Status BatchCommands(grpc::ServerContext * grpc_context, + grpc::ServerReaderWriter * stream) override; + +private: + std::tuple createDBContext(const grpc::ServerContext * grpc_contex) const; + +private: + IServer & server; + + Logger * log; +}; + +} // namespace DB diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index a12af86890a..67762101bf4 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -130,6 +130,151 @@ inline int memcmp16(const void * a, const void * b) } +inline time_t dateToDateTime(UInt32 date_data) +{ + DayNum_t day_num(date_data); + LocalDate local_date(day_num); + // todo use timezone info + return DateLUT::instance().makeDateTime(local_date.year(), local_date.month(), local_date.day(), 0, 0, 0); +} + +inline std::tuple dateTimeToDate(time_t time_data) +{ + // todo use timezone info + auto & date_lut = DateLUT::instance(); + auto truncated = date_lut.toHour(time_data) != 0 || date_lut.toMinute(time_data) != 0 || date_lut.toSecond(time_data) != 0; + auto values = date_lut.getValues(time_data); + auto day_num = date_lut.makeDayNum(values.year, values.month, values.day_of_month); + return std::make_tuple(day_num, truncated); +} + + +template class Op, bool is_left_date> +struct DateDateTimeComparisonImpl +{ + static void NO_INLINE vector_vector(const PaddedPODArray & a, const PaddedPODArray & b, PaddedPODArray & c) + { + size_t size = a.size(); + const A * a_pos = &a[0]; + const B * b_pos = &b[0]; + UInt8 * c_pos = &c[0]; + const A * a_end = a_pos + size; + while (a_pos < a_end) + { + if (is_left_date) + { + using OpType = B; + time_t date_time = dateToDateTime(*a_pos); + *c_pos = Op::apply((OpType)date_time, *b_pos); + } + else + { + using OpType = A; + time_t date_time = dateToDateTime(*b_pos); + *c_pos = Op::apply(*a_pos, (OpType)date_time); + } + ++a_pos; + ++b_pos; + ++c_pos; + } + } + + static void NO_INLINE vector_constant(const PaddedPODArray & a, B b, PaddedPODArray & c) + { + if (!is_left_date) + { + // datetime vector with date constant + using OpType = A; + time_t date_time = dateToDateTime(b); + NumComparisonImpl>::vector_constant(a, (OpType) date_time, c); + } + else + { + // date vector with datetime constant + // first check if datetime constant can be convert to date constant + bool truncated; + DayNum_t date_num; + std::tie(date_num, truncated) = dateTimeToDate((time_t) b); + if (!truncated) + { + using OpType = A; + NumComparisonImpl>::vector_constant(a, (OpType) date_num, c); + } + else + { + using OpType = B; + size_t size = a.size(); + const A *a_pos = &a[0]; + UInt8 *c_pos = &c[0]; + const A *a_end = a_pos + size; + + while (a_pos < a_end) + { + time_t date_time = dateToDateTime(*a_pos); + *c_pos = Op::apply((OpType) date_time, b); + ++a_pos; + ++c_pos; + } + } + } + } + + static void constant_vector(A a, const PaddedPODArray & b, PaddedPODArray & c) + { + if (is_left_date) + { + // date constant with datetime vector + using OpType = B; + time_t date_time = dateToDateTime(a); + NumComparisonImpl>::constant_vector((OpType)date_time, b, c); + } + else + { + // datetime constant with date vector + bool truncated; + DayNum_t date_num; + std::tie(date_num, truncated) = dateTimeToDate((time_t) a); + if (!truncated) + { + using OpType = B; + NumComparisonImpl>::vector_constant((OpType)a, date_num, c); + } + else + { + using OpType = A; + size_t size = b.size(); + const B *b_pos = &b[0]; + UInt8 *c_pos = &c[0]; + const B *b_end = b_pos + size; + + while (b_pos < b_end) + { + time_t date_time = dateToDateTime(*b_pos); + *c_pos = Op::apply(a, (OpType) date_time); + ++b_pos; + ++c_pos; + } + } + } + } + + static void constant_constant(A a, B b, UInt8 & c) { + if (is_left_date) + { + using OpType = B; + time_t date_time = dateToDateTime(a); + NumComparisonImpl>::constant_constant((OpType) date_time, b, c); + } + else + { + using OpType = A; + time_t date_time = dateToDateTime(b); + NumComparisonImpl>::constant_constant(a, (OpType) date_time, c); + } + } +}; + + template struct StringComparisonImpl { @@ -985,6 +1130,69 @@ class FunctionComparison : public IFunction } } + bool executeDateWithDateTimeOrDateTimeWithDate( + Block &block, size_t result, + const IColumn *col_left_untyped, const IColumn *col_right_untyped, + const DataTypePtr &left_type, const DataTypePtr &right_type) + { + if ((checkDataType(left_type.get()) && checkDataType(right_type.get())) + || (checkDataType(left_type.get()) && checkDataType(right_type.get()))) + { + bool is_left_date = checkDataType(left_type.get()); + if (is_left_date) + { + return executeDateAndDateTimeCompare(block, result, col_left_untyped, col_right_untyped); + } + else + { + return executeDateAndDateTimeCompare(block, result, col_left_untyped, col_right_untyped); + } + } + return false; + } + + template + bool executeDateAndDateTimeCompare(Block & block, size_t result, const IColumn * c0, const IColumn * c1) + { + bool c0_const = c0->isColumnConst(); + bool c1_const = c1->isColumnConst(); + + if (c0_const && c1_const) + { + UInt8 res = 0; + DateDateTimeComparisonImpl::constant_constant( + checkAndGetColumnConst>(c0)->template getValue(), + checkAndGetColumnConst>(c1)-> template getValue(), res); + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(c0->size(), toField(res)); + } + else + { + auto c_res = ColumnUInt8::create(); + ColumnUInt8::Container & vec_res = c_res->getData(); + vec_res.resize(c0->size()); + if (c0_const) + { + DateDateTimeComparisonImpl::constant_vector( + checkAndGetColumnConst>(c0)-> template getValue(), + checkAndGetColumn>(c1)->getData(), vec_res); + } + else if (c1_const) + { + DateDateTimeComparisonImpl::vector_constant( + checkAndGetColumn>(c0)->getData(), + checkAndGetColumnConst>(c1)-> template getValue(), vec_res); + } + else + { + DateDateTimeComparisonImpl::vector_vector( + checkAndGetColumn>(c0)->getData(), + checkAndGetColumn>(c1)->getData(), vec_res); + } + block.getByPosition(result).column = std::move(c_res); + } + return true; + } + public: String getName() const override { @@ -1103,7 +1311,8 @@ class FunctionComparison : public IFunction if (left_is_num && right_is_num) { - if (!( executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + if (!(executeDateWithDateTimeOrDateTimeWithDate(block, result, col_left_untyped, col_right_untyped, col_with_type_and_name_left.type, col_with_type_and_name_right.type) + || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) diff --git a/dbms/src/Functions/FunctionsDateTime.cpp b/dbms/src/Functions/FunctionsDateTime.cpp index 58a52188af7..e9e5e065dd1 100644 --- a/dbms/src/Functions/FunctionsDateTime.cpp +++ b/dbms/src/Functions/FunctionsDateTime.cpp @@ -59,6 +59,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) { factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index c23b5c1fa7d..1a6fbf3e028 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -1276,6 +1276,72 @@ class FunctionYesterday : public IFunction } }; +class FunctionMyTimeZoneConvertByOffset : public IFunction +{ + using FromFieldType = typename DataTypeMyDateTime::FieldType; + using ToFieldType = typename DataTypeMyDateTime::FieldType; +public: + static FunctionPtr create(const Context &) { return std::make_shared(); }; + static constexpr auto name = "ConvertTimeZoneByOffset"; + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override {return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!checkDataType(arguments[0].type.get())) + throw Exception{ + "Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + + ". Should be MyDateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + if (!arguments[1].type->isInteger()) + throw Exception{ + "Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + + ". Should be Integer type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return arguments[0].type; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { + static const DateLUTImpl & UTC = DateLUT::instance("UTC"); + if (const ColumnVector *col_from + = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) { + auto col_to = ColumnVector::create(); + const typename ColumnVector::Container &vec_from = col_from->getData(); + typename ColumnVector::Container &vec_to = col_to->getData(); + size_t size = vec_from.size(); + vec_to.resize(size); + + const auto offset_col = block.getByPosition(arguments.back()).column.get(); + if (!offset_col->isColumnConst()) + throw Exception{ + "Second argument of function " + getName() + " must be an integral constant", + ErrorCodes::ILLEGAL_COLUMN}; + + const auto offset = offset_col->getInt(0); + for (size_t i = 0; i < size; ++i) { + UInt64 result_time = vec_from[i] + offset; + // todo maybe affected by daytime saving, need double check + convertTimeZoneByOffset(vec_from[i], result_time, offset, UTC); + vec_to[i] = result_time; + } + + block.getByPosition(result).column = std::move(col_to); + } else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + name, + ErrorCodes::ILLEGAL_COLUMN); + } + +}; template class FunctionMyTimeZoneConverter : public IFunction { @@ -1318,7 +1384,7 @@ class FunctionMyTimeZoneConverter : public IFunction size_t size = vec_from.size(); vec_to.resize(size); - const auto & time_zone_utc = DateLUT::instance("UTC"); + static const auto & time_zone_utc = DateLUT::instance("UTC"); const auto & time_zone_other = extractTimeZoneFromFunctionArguments(block, arguments, 1, 0); for (size_t i = 0; i < size; ++i) { diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 2b356923152..37f905f6606 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -1025,6 +1025,10 @@ struct NameLike { static constexpr auto name = "like"; }; +struct NameLike3Args +{ + static constexpr auto name = "like3Args"; +}; struct NameNotLike { static constexpr auto name = "notLike"; @@ -1058,6 +1062,7 @@ using FunctionPositionCaseInsensitiveUTF8 using FunctionMatch = FunctionsStringSearch, NameMatch>; using FunctionLike = FunctionsStringSearch, NameLike>; +using FunctionLike3Args = FunctionsStringSearch, NameLike3Args, 3>; using FunctionNotLike = FunctionsStringSearch, NameNotLike>; using FunctionExtract = FunctionsStringSearchToString; using FunctionReplaceOne = FunctionStringReplace, NameReplaceOne>; @@ -1078,6 +1083,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); } diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index 9de117464a2..c132d2a1bd3 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -38,12 +38,14 @@ namespace DB * Warning! At this point, the arguments needle, pattern, n, replacement must be constants. */ +static const UInt8 CH_ESCAPE_CHAR = '\\'; -template +template class FunctionsStringSearch : public IFunction { public: static constexpr auto name = Name::name; + static constexpr auto has_3_args = (num_args == 3); static FunctionPtr create(const Context &) { return std::make_shared(); @@ -56,7 +58,7 @@ class FunctionsStringSearch : public IFunction size_t getNumberOfArguments() const override { - return 2; + return num_args; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -68,10 +70,60 @@ class FunctionsStringSearch : public IFunction if (!arguments[1]->isString()) throw Exception( "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (has_3_args && !arguments[2]->isInteger()) + throw Exception( + "Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared>(); } + // replace the escape_char in orig_string with '\\' + // this function does not check the validation of the orig_string + // for example, for string "abcd" and escape char 'd', it will + // return "abc\\" + String replaceEscapeChar(String & orig_string, UInt8 escape_char) + { + std::stringstream ss; + for (size_t i = 0; i < orig_string.size(); i++) + { + auto c = orig_string[i]; + if (c == escape_char) + { + if (i+1 != orig_string.size() && orig_string[i+1] == escape_char) + { + // two successive escape char, which means it is trying to escape itself, just remove one + i++; + ss << escape_char; + } + else + { + // https://github.com/pingcap/tidb/blob/master/util/stringutil/string_util.go#L154 + // if any char following escape char that is not [escape_char,'_','%'], it is invalid escape. + // mysql will treat escape character as the origin value even + // the escape sequence is invalid in Go or C. + // e.g., \m is invalid in Go, but in MySQL we will get "m" for select '\m'. + // Following case is correct just for escape \, not for others like +. + // TODO: Add more checks for other escapes. + if (i+1 != orig_string.size() && orig_string[i+1] == CH_ESCAPE_CHAR) + { + continue; + } + ss << CH_ESCAPE_CHAR; + } + } + else if (c == CH_ESCAPE_CHAR) + { + // need to escape this '\\' + ss << CH_ESCAPE_CHAR << CH_ESCAPE_CHAR; + } + else + { + ss << c; + } + } + return ss.str(); + } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { using ResultType = typename Impl::ResultType; @@ -82,10 +134,44 @@ class FunctionsStringSearch : public IFunction const ColumnConst * col_haystack_const = typeid_cast(&*column_haystack); const ColumnConst * col_needle_const = typeid_cast(&*column_needle); + UInt8 escape_char = CH_ESCAPE_CHAR; + if (has_3_args) + { + auto * col_escape_const = typeid_cast(&*block.getByPosition(arguments[2]).column); + bool valid_args = true; + if (col_needle_const == nullptr || col_escape_const == nullptr) + { + valid_args = false; + } + else + { + auto c = col_escape_const->getValue(); + if (c < 0 || c > 255) + { + // todo maybe use more strict constraint + valid_args = false; + } + else + { + escape_char = (UInt8) c; + } + } + if (!valid_args) + { + throw Exception("2nd and 3rd arguments of function " + getName() + " must " + "be constants, and the 3rd argument must between 0 and 255."); + } + } + if (col_haystack_const && col_needle_const) { ResultType res{}; - Impl::constant_constant(col_haystack_const->getValue(), col_needle_const->getValue(), res); + String needle_string = col_needle_const->getValue(); + if (has_3_args && escape_char != CH_ESCAPE_CHAR) + { + needle_string = replaceEscapeChar(needle_string, escape_char); + } + Impl::constant_constant(col_haystack_const->getValue(), needle_string, res); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res)); return; } @@ -105,7 +191,15 @@ class FunctionsStringSearch : public IFunction col_needle_vector->getOffsets(), vec_res); else if (col_haystack_vector && col_needle_const) - Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue(), vec_res); + { + String needle_string = col_needle_const->getValue(); + if (has_3_args && escape_char != CH_ESCAPE_CHAR) + { + needle_string = replaceEscapeChar(needle_string, escape_char); + } + Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), + needle_string, vec_res); + } else if (col_haystack_const && col_needle_vector) Impl::constant_vector(col_haystack_const->getValue(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res); else diff --git a/dbms/src/Interpreters/ClientInfo.h b/dbms/src/Interpreters/ClientInfo.h index 58a6c250b55..bf4562bec8f 100644 --- a/dbms/src/Interpreters/ClientInfo.h +++ b/dbms/src/Interpreters/ClientInfo.h @@ -24,6 +24,7 @@ class ClientInfo { TCP = 1, HTTP = 2, + GRPC = 3, }; enum class HTTPMethod : UInt8 diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 74ae3b798f5..07f5bba77b4 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1389,12 +1389,12 @@ DDLWorker & Context::getDDLWorker() const return *shared->ddl_worker; } -void Context::initializeRaftService(const std::string & service_addr) +void Context::initializeRaftService() { auto lock = getLock(); if (shared->raft_service) throw Exception("Raft Service has already been initialized.", ErrorCodes::LOGICAL_ERROR); - shared->raft_service = std::make_shared(service_addr, *this); + shared->raft_service = std::make_shared(*this); } void Context::shutdownRaftService() diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 9bcfc440332..af3f1ec5067 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -358,7 +358,7 @@ class Context void setDDLWorker(std::shared_ptr ddl_worker); DDLWorker & getDDLWorker() const; - void initializeRaftService(const std::string & service_addr); + void initializeRaftService(); void shutdownRaftService(); void createTMTContext(const std::vector & pd_addrs, const std::string & learner_key, diff --git a/dbms/src/Interpreters/IQuerySource.h b/dbms/src/Interpreters/IQuerySource.h new file mode 100644 index 00000000000..0738f47f836 --- /dev/null +++ b/dbms/src/Interpreters/IQuerySource.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// A tiny abstraction of different sources a query comes from, i.e. SQL string or DAG request. +class IQuerySource +{ +public: + virtual ~IQuerySource() = default; + + virtual std::tuple parse(size_t max_query_size) = 0; + virtual String str(size_t max_query_size) = 0; + virtual std::unique_ptr interpreter(Context & context, QueryProcessingStage::Enum stage) = 0; +}; + +} // namespace DB diff --git a/dbms/src/Interpreters/SQLQuerySource.cpp b/dbms/src/Interpreters/SQLQuerySource.cpp new file mode 100644 index 00000000000..aeb1a64bcad --- /dev/null +++ b/dbms/src/Interpreters/SQLQuerySource.cpp @@ -0,0 +1,36 @@ +#include +#include +#include +#include + + +namespace DB +{ + +SQLQuerySource::SQLQuerySource(const char * begin_, const char * end_) : begin(begin_), end(end_) {} + +std::tuple SQLQuerySource::parse(size_t max_query_size) +{ + ParserQuery parser(end); + size_t query_size; + /// TODO Parser should fail early when max_query_size limit is reached. + ast = parseQuery(parser, begin, end, "", max_query_size); + + /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. + if (!(begin <= ast->range.first && ast->range.second <= end)) + throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR); + query_size = ast->range.second - begin; + query = String(begin, begin + query_size); + return std::make_tuple(query, ast); +} + +String SQLQuerySource::str(size_t max_query_size) +{ + return String(begin, begin + std::min(end - begin, static_cast(max_query_size))); +} + +std::unique_ptr SQLQuerySource::interpreter(Context & context, QueryProcessingStage::Enum stage) +{ + return InterpreterFactory::get(ast, context, stage); +} +} // namespace DB diff --git a/dbms/src/Interpreters/SQLQuerySource.h b/dbms/src/Interpreters/SQLQuerySource.h new file mode 100644 index 00000000000..cdffe438c66 --- /dev/null +++ b/dbms/src/Interpreters/SQLQuerySource.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace DB +{ + +/// Regular query source of a SQL string. +class SQLQuerySource : public IQuerySource +{ +public: + SQLQuerySource(const char * begin_, const char * end_); + + std::tuple parse(size_t max_query_size) override; + String str(size_t max_query_size) override; + std::unique_ptr interpreter(Context & context, QueryProcessingStage::Enum stage) override; + +private: + const char * begin; + const char * end; + String query; + ASTPtr ast; +}; + +} // namespace DB diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 925479e05e1..27e8757c658 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -12,6 +12,8 @@ #include #include +#include + #include #include #include @@ -22,6 +24,7 @@ #include #include +#include namespace DB @@ -34,6 +37,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int INCORRECT_ELEMENT_OF_SET; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int COP_BAD_DAG_REQUEST; } @@ -256,6 +260,41 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co insertFromBlock(block, fill_set_elements); } +void Set::createFromDAGExpr(const DataTypes & types, const tipb::Expr & expr, bool fill_set_elements) +{ + /// Will form a block with values from the set. + + Block header; + size_t num_columns = types.size(); + if (num_columns != 1) + { + throw Exception("Incorrect element of set, tuple in is not supported yet", ErrorCodes::INCORRECT_ELEMENT_OF_SET); + } + for (size_t i = 0; i < num_columns; ++i) + header.insert(ColumnWithTypeAndName(types[i]->createColumn(), types[i], "_" + toString(i))); + setHeader(header); + + MutableColumns columns = header.cloneEmptyColumns(); + + for (int i = 1; i < expr.children_size(); i++) + { + auto & child = expr.children(i); + // todo support constant expression by constant folding + if (!isLiteralExpr(child)) + { + throw Exception("Only literal is supported in children of expr `in`", ErrorCodes::COP_BAD_DAG_REQUEST); + } + Field value = decodeLiteral(child); + DataTypePtr type = child.has_field_type() ? getDataTypeByFieldType(child.field_type()) : types[0]; + value = convertFieldToType(value, *type); + + if (!value.isNull()) + columns[0]->insert(value); + } + + Block block = header.cloneWithColumns(std::move(columns)); + insertFromBlock(block, fill_set_elements); +} ColumnPtr Set::execute(const Block & block, bool negative) const { diff --git a/dbms/src/Interpreters/Set.h b/dbms/src/Interpreters/Set.h index e27bdf58ec6..9600ed2065f 100644 --- a/dbms/src/Interpreters/Set.h +++ b/dbms/src/Interpreters/Set.h @@ -1,5 +1,10 @@ #pragma once +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + #include #include #include @@ -48,6 +53,11 @@ class Set */ void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements); + /** + * Create a Set from DAG Expr, used when processing DAG Request + */ + void createFromDAGExpr(const DataTypes & types, const tipb::Expr & expr, bool fill_set_elements); + /** Create a Set from stream. * Call setHeader, then call insertFromBlock for each block. */ diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 8c80ed0f990..83c55d51089 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -29,7 +29,11 @@ struct Settings M(SettingString, regions, "", "the region need to be read.") \ M(SettingBool, resolve_locks, false, "tmt resolve locks.") \ M(SettingUInt64, read_tso, DEFAULT_MAX_READ_TSO, "tmt read tso.") \ + M(SettingInt64, dag_records_per_chunk, DEFAULT_DAG_RECORDS_PER_CHUNK, "default chunk size of a DAG response.") \ + M(SettingString, dag_planner, "optree", "planner for DAG query, sql builds the SQL string, optree builds the internal operator(stream) tree.") \ + M(SettingBool, dag_expr_field_type_strict_check, true, "when set to true, every expr in the dag request must provide field type, otherwise only the result expr will be checked.") \ M(SettingInt64, schema_version, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, "tmt schema version.") \ + M(SettingUInt64, batch_commands_threads, 0, "Number of threads to use for handling batch commands concurrently. 0 means - same as 'max_threads'.") \ M(SettingUInt64, min_compress_block_size, DEFAULT_MIN_COMPRESS_BLOCK_SIZE, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.") \ M(SettingUInt64, max_compress_block_size, DEFAULT_MAX_COMPRESS_BLOCK_SIZE, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \ M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading") \ diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index bac8cef33c5..6ebe2f72b50 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -18,11 +18,15 @@ #include #include -#include +#include #include #include #include +#include +#include #include +#include +#include namespace ProfileEvents @@ -132,8 +136,7 @@ static void onExceptionBeforeStart(const String & query, Context & context, time static std::tuple executeQueryImpl( - const char * begin, - const char * end, + IQuerySource & query_src, Context & context, bool internal, QueryProcessingStage::Enum stage) @@ -145,9 +148,8 @@ static std::tuple executeQueryImpl( const Settings & settings = context.getSettingsRef(); - ParserQuery parser(end); ASTPtr ast; - size_t query_size; + String query; /// Don't limit the size of internal queries. size_t max_query_size = 0; @@ -156,28 +158,21 @@ static std::tuple executeQueryImpl( try { - /// TODO Parser should fail early when max_query_size limit is reached. - ast = parseQuery(parser, begin, end, "", max_query_size); - - /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. - if (!(begin <= ast->range.first && ast->range.second <= end)) - throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR); - query_size = ast->range.second - begin; + std::tie(query, ast) = query_src.parse(max_query_size); } catch (...) { if (!internal) { /// Anyway log the query. - String query = String(begin, begin + std::min(end - begin, static_cast(max_query_size))); - logQuery(query.substr(0, settings.log_queries_cut_to_length), context); - onExceptionBeforeStart(query, context, current_time); + String str = query_src.str(max_query_size); + logQuery(str.substr(0, settings.log_queries_cut_to_length), context); + onExceptionBeforeStart(str, context, current_time); } throw; } - String query(begin, query_size); BlockIO res; try @@ -206,7 +201,7 @@ static std::tuple executeQueryImpl( context.setProcessListElement(&process_list_entry->get()); } - auto interpreter = InterpreterFactory::get(ast, context, stage); + auto interpreter = query_src.interpreter(context, stage); res = interpreter->execute(); /// Delayed initialization of query streams (required for KILL QUERY purposes) @@ -391,7 +386,16 @@ BlockIO executeQuery( QueryProcessingStage::Enum stage) { BlockIO streams; - std::tie(std::ignore, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context, internal, stage); + SQLQuerySource query_src(query.data(), query.data() + query.size()); + std::tie(std::ignore, streams) = executeQueryImpl(query_src, context, internal, stage); + return streams; +} + + +BlockIO executeQuery(DAGQuerySource & dag, Context & context, bool internal, QueryProcessingStage::Enum stage) +{ + BlockIO streams; + std::tie(std::ignore, streams) = executeQueryImpl(dag, context, internal, stage); return streams; } @@ -432,7 +436,8 @@ void executeQuery( ASTPtr ast; BlockIO streams; - std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete); + SQLQuerySource query_info(begin, end); + std::tie(ast, streams) = executeQueryImpl(query_info, context, false, QueryProcessingStage::Complete); try { diff --git a/dbms/src/Interpreters/executeQuery.h b/dbms/src/Interpreters/executeQuery.h index cc333ea8cb9..4f4ef136ed2 100644 --- a/dbms/src/Interpreters/executeQuery.h +++ b/dbms/src/Interpreters/executeQuery.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -39,4 +40,7 @@ BlockIO executeQuery( QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// To which stage the query must be executed. ); + +BlockIO executeQuery(DAGQuerySource & dag, Context & context, bool internal, QueryProcessingStage::Enum stage); + } diff --git a/dbms/src/Raft/RaftService.cpp b/dbms/src/Raft/RaftService.cpp index 599111cab7c..525ae068afe 100644 --- a/dbms/src/Raft/RaftService.cpp +++ b/dbms/src/Raft/RaftService.cpp @@ -9,9 +9,8 @@ namespace DB { -RaftService::RaftService(const std::string & address_, DB::Context & db_context_) - : address(address_), - db_context(db_context_), +RaftService::RaftService(DB::Context & db_context_) + : db_context(db_context_), kvstore(db_context.getTMTContext().getKVStore()), background_pool(db_context.getBackgroundPool()), log(&Logger::get("RaftService")) @@ -19,16 +18,6 @@ RaftService::RaftService(const std::string & address_, DB::Context & db_context_ if (!db_context.getTMTContext().isInitialized()) throw Exception("TMTContext is not initialized", ErrorCodes::LOGICAL_ERROR); - grpc::ServerBuilder builder; - builder.AddListeningPort(address, grpc::InsecureServerCredentials()); - builder.RegisterService(this); - - // Prevent TiKV from throwing "Received message larger than max (4404462 vs. 4194304)" error. - builder.SetMaxReceiveMessageSize(-1); - builder.SetMaxSendMessageSize(-1); - - grpc_server = builder.BuildAndStart(); - persist_handle = background_pool.addTask([this] { return kvstore->tryPersist(); }, false); table_flush_handle = background_pool.addTask([this] { @@ -72,8 +61,6 @@ RaftService::RaftService(const std::string & address_, DB::Context & db_context_ return true; }); - LOG_INFO(log, "Raft service listening on [" << address << "]"); - { std::vector regions; kvstore->traverseRegions([®ions](RegionID, const RegionPtr & region) { @@ -128,12 +115,6 @@ RaftService::~RaftService() background_pool.removeTask(region_decode_handle); region_decode_handle = nullptr; } - - // wait 5 seconds for pending rpcs to gracefully stop - gpr_timespec deadline{5, 0, GPR_TIMESPAN}; - LOG_DEBUG(log, "Begin to shutting down grpc server"); - grpc_server->Shutdown(deadline); - grpc_server->Wait(); } grpc::Status RaftService::ApplyCommandBatch(grpc::ServerContext * grpc_context, CommandServerReaderWriter * stream) @@ -152,7 +133,7 @@ grpc::Status RaftService::ApplyCommandBatch(grpc::ServerContext * grpc_context, } catch (...) { - tryLogCurrentException(log, "gRPC ApplyCommandBatch on " + address + " error"); + tryLogCurrentException(log, "gRPC ApplyCommandBatch error"); } return grpc::Status::CANCELLED; @@ -167,7 +148,7 @@ grpc::Status RaftService::ApplySnapshot(grpc::ServerContext *, CommandServerRead } catch (...) { - tryLogCurrentException(log, "gRPC ApplyCommandBatch on " + address + " error"); + tryLogCurrentException(log, "gRPC ApplyCommandBatch error"); return grpc::Status(grpc::StatusCode::UNKNOWN, "Runtime error, check theflash log for detail."); } } diff --git a/dbms/src/Raft/RaftService.h b/dbms/src/Raft/RaftService.h index 787c5f3e035..38421f0c727 100644 --- a/dbms/src/Raft/RaftService.h +++ b/dbms/src/Raft/RaftService.h @@ -23,7 +23,7 @@ using RegionMap = std::unordered_map; class RaftService final : public enginepb::Engine::Service, public std::enable_shared_from_this, private boost::noncopyable { public: - RaftService(const std::string & address_, Context & db_context); + RaftService(Context & db_context); ~RaftService() final; @@ -37,10 +37,6 @@ class RaftService final : public enginepb::Engine::Service, public std::enable_s grpc::ServerContext * grpc_context, CommandServerReader * reader, enginepb::SnapshotDone * response) override; private: - std::string address; - - GRPCServerPtr grpc_server; - Context & db_context; KVStorePtr kvstore; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 7ddfe01aeb0..c94b6febbfa 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -1,16 +1,8 @@ #include "Server.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include #include #include @@ -20,20 +12,30 @@ #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 #include "HTTPHandlerFactory.h" #include "MetricsTransmitter.h" #include "StatusFile.h" @@ -443,30 +445,71 @@ int Server::main(const std::vector & /*args*/) LOG_DEBUG(log, "Shutted down storages."); }); - if (has_zookeeper && config().has("distributed_ddl")) - { - /// DDL worker should be started after all tables were loaded - String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/"); - global_context->setDDLWorker(std::make_shared(ddl_zookeeper_path, *global_context, &config(), "distributed_ddl")); - } - { /// initialize TMTContext global_context->getTMTContext().restore(); } - if (need_raft_service) + /// Then, startup grpc server to serve raft and/or flash services. + String flash_server_addr = config().getString("flash.service_addr", "0.0.0.0:3930"); + std::unique_ptr flash_service = nullptr; + std::unique_ptr flash_grpc_server = nullptr; { - String raft_service_addr = config().getString("raft.service_addr"); - global_context->initializeRaftService(raft_service_addr); + grpc::ServerBuilder builder; + builder.AddListeningPort(flash_server_addr, grpc::InsecureServerCredentials()); + + /// Init and register raft service if necessary. + if (need_raft_service) + { + global_context->initializeRaftService(); + builder.RegisterService(&(global_context->getRaftService())); + LOG_INFO(log, "Raft service registered"); + } + + /// Init and register flash service. + flash_service = std::make_unique(*this); + builder.RegisterService(flash_service.get()); + LOG_INFO(log, "Flash service registered"); + + /// Kick off grpc server. + // Prevent TiKV from throwing "Received message larger than max (4404462 vs. 4194304)" error. + builder.SetMaxReceiveMessageSize(-1); + builder.SetMaxSendMessageSize(-1); + flash_grpc_server = builder.BuildAndStart(); + LOG_INFO(log, "Flash grpc server listening on [" << flash_server_addr << "]"); } SCOPE_EXIT({ - LOG_INFO(log, "Shutting down raft service."); - global_context->shutdownRaftService(); - LOG_INFO(log, "Shutted down raft service."); + /// Shut down grpc server. + // wait 5 seconds for pending rpcs to gracefully stop + gpr_timespec deadline{5, 0, GPR_TIMESPAN}; + LOG_INFO(log, "Begin to shut down flash grpc server"); + flash_grpc_server->Shutdown(deadline); + flash_grpc_server->Wait(); + flash_grpc_server.reset(); + LOG_INFO(log, "Shut down flash grpc server"); + + /// Close flash service. + LOG_INFO(log, "Begin to shut down flash service"); + flash_service.reset(); + LOG_INFO(log, "Shut down flash service"); + + /// Close raft service if necessary. + if (need_raft_service) + { + LOG_INFO(log, "Begin to shut down raft service"); + global_context->shutdownRaftService(); + LOG_INFO(log, "Shut down raft service"); + } }); + if (has_zookeeper && config().has("distributed_ddl")) + { + /// DDL worker should be started after all tables were loaded + String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/"); + global_context->setDDLWorker(std::make_shared(ddl_zookeeper_path, *global_context, &config(), "distributed_ddl")); + } + { Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); diff --git a/dbms/src/Server/config.xml b/dbms/src/Server/config.xml index dc6fc34f507..ced2c54b2c1 100644 --- a/dbms/src/Server/config.xml +++ b/dbms/src/Server/config.xml @@ -313,12 +313,15 @@ - 0.0.0.0:20021 /var/lib/clickhouse/kvstore /var/lib/clickhouse/regmap http://127.0.0.1:13579 + + 0.0.0.0:3930 + + system + + 0.0.0.0:3930 + + 8123 9000 9009 diff --git a/tests/docker/config/tiflash.xml b/tests/docker/config/tiflash.xml index 8de552eab6e..afc6ea001e4 100644 --- a/tests/docker/config/tiflash.xml +++ b/tests/docker/config/tiflash.xml @@ -16,7 +16,6 @@ the flash - 0.0.0.0:3930 /data/kvstore pd0:2379 @@ -26,6 +25,10 @@ system + + 0.0.0.0:3930 + + 8123 9000 9009 diff --git a/tests/mutable-test/txn_dag/aggregation.test b/tests/mutable-test/txn_dag/aggregation.test new file mode 100644 index 00000000000..ef1905a5dc4 --- /dev/null +++ b/tests/mutable-test/txn_dag/aggregation.test @@ -0,0 +1,38 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 52, 'test3', 777) + +# DAG read by not specifying region id, group by. +=> DBGInvoke dag('select count(col_1) from default.test group by col_2') +┌─count(col_1)─┬─col_2─┐ +│ 2 │ 666 │ +│ 1 │ 777 │ +└──────────────┴───────┘ + +=> DBGInvoke dag('select count(col_1),count(col_1) from default.test group by col_2') +┌─count(col_1)─┬─count(col_1)─┬─col_2─┐ +│ 2 │ 2 │ 666 │ +│ 1 │ 1 │ 777 │ +└──────────────┴──────────────┴───────┘ + +# DAG read by explicitly specifying region id, where + group by. +=> DBGInvoke dag('select count(col_1) from default.test where col_2 = 666 group by col_2', 4) +┌─count(col_1)─┬─col_2─┐ +│ 2 │ 666 │ +└──────────────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/data_type_number.test b/tests/mutable-test/txn_dag/data_type_number.test new file mode 100644 index 00000000000..95ec49b10f6 --- /dev/null +++ b/tests/mutable-test/txn_dag/data_type_number.test @@ -0,0 +1,91 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +#=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int8, col_2 UInt8, col_3 Int16, col_4 UInt16, col_5 Int32, col_6 UInt32, col_7 Int64, col_8 UInt64, col_9 Float32, col_10 Float64, col_11 Decimal(10,2)') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int8, col_2 UInt8, col_3 Int16, col_4 UInt16, col_5 Int32, col_6 UInt32, col_7 Int64, col_8 UInt64, col_9 Float32, col_10 Float64') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +#=> DBGInvoke __raft_insert_row(default, test, 4, 50, -128, 255, -32768, 65535, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 12345.6789, 1234567.890123, 666.88) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, -128, 255, -32768, 65535, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 12345.6789, 1234567.890123) + +# DAG read full table scan +=> DBGInvoke dag('select * from default.test') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Int8 column +=> DBGInvoke dag('select * from default.test where col_1 = -128') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by UInt8 column +=> DBGInvoke dag('select * from default.test where col_2 = 255') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Int16 column +=> DBGInvoke dag('select * from default.test where col_3 = -32768') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by UInt16 column +=> DBGInvoke dag('select * from default.test where col_4 = 65535') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Int32 column +=> DBGInvoke dag('select * from default.test where col_5 = -2147483648') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by UInt32 column +=> DBGInvoke dag('select * from default.test where col_6 = 4294967295') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Int64 column +=> DBGInvoke dag('select * from default.test where col_7 = -9223372036854775808') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by UInt64 column +=> DBGInvoke dag('select * from default.test where col_8 = 18446744073709551615') " --dag_planner="optree +┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Float32 column +#=> DBGInvoke dag('select * from default.test where col_9 = 12345.679') " --dag_planner="optree +#┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +#│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +#└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Float64 column +#=> DBGInvoke dag('select * from default.test where col_10 = 1234567.890123') " --dag_planner="optree +#┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┐ +#│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ +#└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┘ + +# DAG read filter by Decimal column +#=> DBGInvoke dag('select * from default.test where col_11 = 666.88') " --dag_planner="optree +#┌─col_1─┬─col_2─┬──col_3─┬─col_4─┬───────col_5─┬──────col_6─┬────────────────col_7─┬────────────────col_8─┬─────col_9─┬─────────col_10─┬─col_11─┐ +#│ -128 │ 255 │ -32768 │ 65535 │ -2147483648 │ 4294967295 │ -9223372036854775808 │ 18446744073709551615 │ 12345.679 │ 1234567.890123 │ 666.88 │ +#└───────┴───────┴────────┴───────┴─────────────┴────────────┴──────────────────────┴──────────────────────┴───────────┴────────────────┴────────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/data_type_others.test b/tests/mutable-test/txn_dag/data_type_others.test new file mode 100644 index 00000000000..3194f8e1ba9 --- /dev/null +++ b/tests/mutable-test/txn_dag/data_type_others.test @@ -0,0 +1,41 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Enum16(\'male\' = 1, \'female\' = 2, \'both\' = 3, \'unknown\' = 4)') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'data type test', 2) + +# DAG read full table scan +=> DBGInvoke dag('select * from default.test') " --dag_planner="optree +┌─col_1──────────┬─col_2──┐ +│ data type test │ female │ +└────────────────┴────────┘ + +# DAG read filter string column +=> DBGInvoke dag('select * from default.test where col_1 = \'data type test\'') " --dag_planner="optree +┌─col_1──────────┬─col_2──┐ +│ data type test │ female │ +└────────────────┴────────┘ + +# DAG read filter enum column +=> DBGInvoke dag('select * from default.test where col_2 = \'female\'') " --dag_planner="optree +┌─col_1──────────┬─col_2──┐ +│ data type test │ female │ +└────────────────┴────────┘ + +# DAG read filter enum column +=> DBGInvoke dag('select * from default.test where col_2 = 2') " --dag_planner="optree +┌─col_1──────────┬─col_2──┐ +│ data type test │ female │ +└────────────────┴────────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/data_type_time.test b/tests/mutable-test/txn_dag/data_type_time.test new file mode 100644 index 00000000000..de813bbb71a --- /dev/null +++ b/tests/mutable-test/txn_dag/data_type_time.test @@ -0,0 +1,28 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDate, col_2 MyDateTime') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, '2019-06-10', '2019-06-10 09:00:00') + +=> DBGInvoke dag('select * from default.test') " --dag_planner="optree +┌──────col_1─┬───────────────col_2─┐ +│ 2019-06-10 │ 2019-06-10 09:00:00 │ +└────────────┴─────────────────────┘ + +# Mock DAG doesn't support date/datetime comparison with string, may need type inference and do implicit conversion to literal. +# => DBGInvoke dag('select * from default.test where col_1 = \'2019-06-06\' and col_2 = \'2019-06-10 09:00:00\'') " --dag_planner="optree +# ┌──────col_1─┬───────────────col_2─┐ +# │ 2019-06-10 │ 2019-06-10 09:00:00 │ +# └────────────┴─────────────────────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/filter.test b/tests/mutable-test/txn_dag/filter.test new file mode 100644 index 00000000000..dd31a653c28 --- /dev/null +++ b/tests/mutable-test/txn_dag/filter.test @@ -0,0 +1,49 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2', 777) + +# DAG read by not specifying region id, where col_1 = 666. +=> DBGInvoke dag('select * from default.test where col_2 = 666') +┌─col_1─┬─col_2─┐ +│ test1 │ 666 │ +└───────┴───────┘ + +# DAG read by explicitly specifying region id, where col_2 = 'test2'. +=> DBGInvoke dag('select col_2 from default.test where col_1 = \'test2\'', 4) +┌─col_2─┐ +│ 777 │ +└───────┘ + +# Mock DAG read, where or. +=> DBGInvoke mock_dag('select col_2, col_1, col_2 from default.test where col_1 = \'test2\' or col_2 = 666', 4) +┌─col_2─┬─col_1─┬─col_2─┐ +│ 666 │ test1 │ 666 │ +│ 777 │ test2 │ 777 │ +└───────┴───────┴───────┘ + +# DAG read, col used multiple times in the query +=> DBGInvoke dag('select * from default.test where col_2 < 777 or col_2 > 888') +┌─col_1─┬─col_2─┐ +│ test1 │ 666 │ +└───────┴───────┘ + +# Mock DAG read, where and. +=> DBGInvoke mock_dag('select col_2, col_1, col_2 from default.test where col_1 = \'test2\' and col_2 = 777', 4) +┌─col_2─┬─col_1─┬─col_2─┐ +│ 777 │ test2 │ 777 │ +└───────┴───────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/key_condition.test b/tests/mutable-test/txn_dag/key_condition.test new file mode 100644 index 00000000000..3c30fcb413a --- /dev/null +++ b/tests/mutable-test/txn_dag/key_condition.test @@ -0,0 +1,35 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64','col_2') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 66, 'test1') +=> DBGInvoke __raft_insert_row(default, test, 4, 77, 'test2') + +# DAG read by not specifying region id, where col_1 = 666. +=> DBGInvoke dag('select * from default.test where col_2 = 66') +┌─col_1─┬─col_2─┐ +│ test1 │ 66 │ +└───────┴───────┘ + +=> DBGInvoke dag('select * from default.test where col_2 > 66') +┌─col_1─┬─col_2─┐ +│ test2 │ 77 │ +└───────┴───────┘ + +=> DBGInvoke dag('select * from default.test where col_2 >= 66') +┌─col_1─┬─col_2─┐ +│ test1 │ 66 │ +│ test2 │ 77 │ +└───────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/key_range.test b/tests/mutable-test/txn_dag/key_range.test new file mode 100644 index 00000000000..5a3860c3118 --- /dev/null +++ b/tests/mutable-test/txn_dag/key_range.test @@ -0,0 +1,24 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test_uint) +=> drop table if exists default.test_uint + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test_uint, 'col_1 String, col_2 UInt64','col_2') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test_uint) +=> DBGInvoke __raft_insert_row(default, test_uint, 4, 88, 'test1') +=> DBGInvoke __raft_insert_row(default, test_uint, 4, 99, 'test2') + +=> DBGInvoke dag('select * from default.test_uint where col_2 >= 66') +┌─col_1─┬─col_2─┐ +│ test1 │ 88 │ +│ test2 │ 99 │ +└───────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test_uint) +=> drop table if exists default.test_uint diff --git a/tests/mutable-test/txn_dag/limit.test b/tests/mutable-test/txn_dag/limit.test new file mode 100644 index 00000000000..ee8d97f75a7 --- /dev/null +++ b/tests/mutable-test/txn_dag/limit.test @@ -0,0 +1,31 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test1', 666) + +# DAG read by not specifying region id, order by col_2 limit 1. +=> DBGInvoke dag('select * from default.test') +┌─col_1─┬─col_2─┐ +│ test1 │ 666 │ +│ test1 │ 666 │ +└───────┴───────┘ + +# Mock DAG read, where + topn. +=> DBGInvoke mock_dag('select col_2, col_1, col_2 from default.test where col_2 = 666 limit 1', 4) +┌─col_2─┬─col_1─┬─col_2─┐ +│ 666 │ test1 │ 666 │ +└───────┴───────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/project.test b/tests/mutable-test/txn_dag/project.test new file mode 100644 index 00000000000..8b29b4a7a08 --- /dev/null +++ b/tests/mutable-test/txn_dag/project.test @@ -0,0 +1,41 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 666) + +# DAG read by not specifying region id, select *. +=> DBGInvoke dag('select * from default.test') " --dag_planner="optree +┌─col_1─┬─col_2─┐ +│ test1 │ 666 │ +└───────┴───────┘ + +# DAG read by not specifying region id, select col_1. +=> DBGInvoke dag('select col_1 from default.test') " --dag_planner="optree +┌─col_1─┐ +│ test1 │ +└───────┘ + +# DAG read by explicitly specifying region id, select col_2. +=> DBGInvoke dag('select col_2 from default.test', 4) " --dag_planner="optree +┌─col_2─┐ +│ 666 │ +└───────┘ + +# Mock DAG read, select col_2, col_1, col_2. +=> DBGInvoke mock_dag('select col_2, col_1, col_2 from default.test', 4) " --dag_planner="optree +┌─col_2─┬─col_1─┬─col_2─┐ +│ 666 │ test1 │ 666 │ +└───────┴───────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/table_scan.test b/tests/mutable-test/txn_dag/table_scan.test new file mode 100644 index 00000000000..ab824845d71 --- /dev/null +++ b/tests/mutable-test/txn_dag/table_scan.test @@ -0,0 +1,41 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1') + +# DAG read by not specifying region id. +=> DBGInvoke dag('select * from default.test') " --dag_planner="optree +┌─col_1─┐ +│ test1 │ +└───────┘ + +# DAG read by explicitly specifying region id. +=> DBGInvoke dag('select * from default.test', 4) " --dag_planner="optree +┌─col_1─┐ +│ test1 │ +└───────┘ + +# Mock DAG read. +=> DBGInvoke mock_dag('select * from default.test', 4) " --dag_planner="optree +┌─col_1─┐ +│ test1 │ +└───────┘ + +# TiDB may push down table scan with -1 column, use keyword _void testing this case. +=> DBGInvoke dag('select _void from default.test') " --dag_planner="optree +┌─_void─┐ +│ 50 │ +└───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/time_zone.test b/tests/mutable-test/txn_dag/time_zone.test new file mode 100644 index 00000000000..9806a8bcae5 --- /dev/null +++ b/tests/mutable-test/txn_dag/time_zone.test @@ -0,0 +1,64 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDate, col_2 MyDatetime(5), col_3 MyDatetime') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, '2019-06-10', '2019-06-10 09:00:00', '2019-06-10 09:00:00') +=> DBGInvoke __raft_insert_row(default, test, 4, 51, '2019-06-11', '2019-06-11 07:00:00', '2019-06-11 09:00:00') +=> DBGInvoke __raft_insert_row(default, test, 4, 52, '2019-06-11', '2019-06-11 08:00:00', '2019-06-11 09:00:00') +=> DBGInvoke __raft_insert_row(default, test, 4, 53, '2019-06-12', '2019-06-11 08:00:00', '2019-06-11 09:00:00') + +=> DBGInvoke dag('select * from default.test') " --dag_planner="optree +┌──────col_1─┬─────────────────────col_2─┬───────────────col_3─┐ +│ 2019-06-10 │ 2019-06-10 09:00:00.00000 │ 2019-06-10 09:00:00 │ +│ 2019-06-11 │ 2019-06-11 07:00:00.00000 │ 2019-06-11 09:00:00 │ +│ 2019-06-11 │ 2019-06-11 08:00:00.00000 │ 2019-06-11 09:00:00 │ +│ 2019-06-12 │ 2019-06-11 08:00:00.00000 │ 2019-06-11 09:00:00 │ +└────────────┴───────────────────────────┴─────────────────────┘ + +# use tz_offset, result is the same since cop will convert the timestamp value to utc timestamp when returing to tidb +=> DBGInvoke dag('select * from default.test',4,28800) " --dag_planner="optree +┌──────col_1─┬─────────────────────col_2─┬───────────────col_3─┐ +│ 2019-06-10 │ 2019-06-10 09:00:00.00000 │ 2019-06-10 09:00:00 │ +│ 2019-06-11 │ 2019-06-11 07:00:00.00000 │ 2019-06-11 09:00:00 │ +│ 2019-06-11 │ 2019-06-11 08:00:00.00000 │ 2019-06-11 09:00:00 │ +│ 2019-06-12 │ 2019-06-11 08:00:00.00000 │ 2019-06-11 09:00:00 │ +└────────────┴───────────────────────────┴─────────────────────┘ + +=> DBGInvoke dag('select * from default.test where col_2 > col_3') " --dag_planner="optree + +=> DBGInvoke dag('select * from default.test where col_2 > col_3',4,28800) " --dag_planner="optree +┌──────col_1─┬─────────────────────col_2─┬───────────────col_3─┐ +│ 2019-06-10 │ 2019-06-10 09:00:00.00000 │ 2019-06-10 09:00:00 │ +│ 2019-06-11 │ 2019-06-11 07:00:00.00000 │ 2019-06-11 09:00:00 │ +│ 2019-06-11 │ 2019-06-11 08:00:00.00000 │ 2019-06-11 09:00:00 │ +│ 2019-06-12 │ 2019-06-11 08:00:00.00000 │ 2019-06-11 09:00:00 │ +└────────────┴───────────────────────────┴─────────────────────┘ + +# tz_name overwrite tz_offset +=> DBGInvoke dag('select * from default.test where col_2 > col_3',4,28800,'UTC') " --dag_planner="optree + +# ts_col in group by clause +=> DBGInvoke dag('select count(1) from default.test where col_2 > \'2019-06-11 15:00:00\' group by col_2',4,28800) " --dag_planner="optree +┌─count(1)─┬─────────────────────col_2─┐ +│ 2 │ 2019-06-11 08:00:00.00000 │ +└──────────┴───────────────────────────┘ + +# ts_col in agg clause +=> DBGInvoke dag('select max(col_2) from default.test group by col_1',4,28800) " --dag_planner="optree +┌──────────max(col_2)─┬──────col_1─┐ +│ 2019-06-11 08:00:00 │ 2019-06-12 │ +│ 2019-06-11 08:00:00 │ 2019-06-11 │ +│ 2019-06-10 09:00:00 │ 2019-06-10 │ +└─────────────────────┴────────────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test diff --git a/tests/mutable-test/txn_dag/topn.test b/tests/mutable-test/txn_dag/topn.test new file mode 100644 index 00000000000..02cdf79c86f --- /dev/null +++ b/tests/mutable-test/txn_dag/topn.test @@ -0,0 +1,33 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('true') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Nullable(String), col_2 Nullable(Int64)') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __put_region(4, 0, 100, default, test) +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2', 777) +=> DBGInvoke __raft_insert_row(default, test, 4, 52, NULL, NULL) + +# DAG read by not specifying region id, order by col_2 limit 1. +=> DBGInvoke dag('select * from default.test order by col_2 limit 2') +┌─col_1─┬─col_2─┐ +│ \N │ \N │ +│ test1 │ 666 │ +└───────┴───────┘ + +# Mock DAG read, where + topn. +=> DBGInvoke mock_dag('select col_2, col_1, col_2 from default.test where col_1 = \'test2\' or col_2 = 666 order by col_1 desc limit 2', 4) +┌─col_2─┬─col_1─┬─col_2─┐ +│ 777 │ test2 │ 777 │ +│ 666 │ test1 │ 666 │ +└───────┴───────┴───────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test