From 66dd568e64963040201836a2d36a537f189fa4d2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 12 Jan 2018 21:03:19 +0800 Subject: [PATCH] ISSUES-957 update condition & test --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 +- dbms/src/Interpreters/InterpreterFactory.cpp | 3 ++- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 7 ++++--- dbms/src/Interpreters/InterpreterInsertQuery.h | 3 ++- dbms/src/Storages/StorageBuffer.cpp | 10 +++++----- dbms/src/Storages/StorageBuffer.h | 3 ++- .../00553_buff_exists_materlized_column.sql | 5 ++--- 7 files changed, 18 insertions(+), 15 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 630d78dc06c..68de37b995e 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -572,7 +572,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) out = std::make_shared( out, columns.columns, columns.column_defaults, context, strict_insert_defaults); - if (context.getSettingsRef().insert_allow_materialized_columns) + if (!context.getSettingsRef().insert_allow_materialized_columns) out = std::make_shared(out, columns.materialized_columns); BlockIO io; diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 45bc9723317..d13c509053c 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -61,7 +61,8 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & else if (typeid_cast(query.get())) { /// readonly is checked inside InterpreterInsertQuery - return std::make_unique(query, context); + bool allow_materialized = static_cast(context.getSettingsRef().insert_allow_materialized_columns); + return std::make_unique(query, context, allow_materialized); } else if (typeid_cast(query.get())) { diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 7b2d0515d87..7ae0708fb05 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -37,8 +37,9 @@ namespace ErrorCodes } -InterpreterInsertQuery::InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_) - : query_ptr(query_ptr_), context(context_) +InterpreterInsertQuery::InterpreterInsertQuery( + const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_) + : query_ptr(query_ptr_), context(context_), allow_materialized(allow_materialized_) { ProfileEvents::increment(ProfileEvents::InsertQuery); } @@ -118,7 +119,7 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared( out, required_columns, table->column_defaults, context, static_cast(context.getSettingsRef().strict_insert_defaults)); - if (context.getSettingsRef().insert_allow_materialized_columns) + if (!allow_materialized) out = std::make_shared(out, table->materialized_columns); out = std::make_shared( diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.h b/dbms/src/Interpreters/InterpreterInsertQuery.h index 7901ea594ef..9bdc5cfcaba 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.h +++ b/dbms/src/Interpreters/InterpreterInsertQuery.h @@ -15,7 +15,7 @@ namespace DB class InterpreterInsertQuery : public IInterpreter { public: - InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_); + InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_ = false); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -37,6 +37,7 @@ private: ASTPtr query_ptr; const Context & context; + bool allow_materialized; }; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index d6bf45a28c6..b932324892d 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -55,14 +55,14 @@ StorageBuffer::StorageBuffer(const std::string & name_, const NamesAndTypesList const ColumnDefaults & column_defaults_, Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, - const String & destination_database_, const String & destination_table_) + const String & destination_database_, const String & destination_table_, bool allow_materialized_) : IStorage{materialized_columns_, alias_columns_, column_defaults_}, name(name_), columns(columns_), context(context_), num_shards(num_shards_), buffers(num_shards_), min_thresholds(min_thresholds_), max_thresholds(max_thresholds_), destination_database(destination_database_), destination_table(destination_table_), no_destination(destination_database.empty() && destination_table.empty()), - log(&Logger::get("StorageBuffer (" + name + ")")) + allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + name + ")")) { } @@ -527,7 +527,6 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl /** We will insert columns that are the intersection set of columns of the buffer table and the subordinate table. * This will support some of the cases (but not all) when the table structure does not match. */ - bool allow_materialized = static_cast(context.getSettingsRef().insert_allow_materialized_columns); Block structure_of_destination_table = allow_materialized ? table->getSampleBlock() : table->getSampleBlockNonMaterialized(); Names columns_intersection; columns_intersection.reserve(block.columns()); @@ -565,7 +564,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl for (const String & column : columns_intersection) list_of_columns->children.push_back(std::make_shared(StringRange(), column, ASTIdentifier::Column)); - InterpreterInsertQuery interpreter{insert, context}; + InterpreterInsertQuery interpreter{insert, context, allow_materialized}; auto block_io = interpreter.execute(); block_io.out->writePrefix(); @@ -651,7 +650,8 @@ void registerStorageBuffer(StorageFactory & factory) num_buckets, StorageBuffer::Thresholds{min_time, min_rows, min_bytes}, StorageBuffer::Thresholds{max_time, max_rows, max_bytes}, - destination_database, destination_table); + destination_database, destination_table, + static_cast(args.local_context.getSettingsRef().insert_allow_materialized_columns)); }); } diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 161e5d7ed10..98cbcff661c 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -103,6 +103,7 @@ private: const String destination_database; const String destination_table; bool no_destination; /// If set, do not write data from the buffer, but simply empty the buffer. + bool allow_materialized; Poco::Logger * log; @@ -131,7 +132,7 @@ protected: const ColumnDefaults & column_defaults_, Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, - const String & destination_database_, const String & destination_table_); + const String & destination_database_, const String & destination_table_, bool allow_materialized_); }; } diff --git a/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql b/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql index 8944015ff3d..49aff2aa184 100644 --- a/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql +++ b/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql @@ -1,6 +1,8 @@ DROP TABLE IF EXISTS test.nums; DROP TABLE IF EXISTS test.nums_buf; +SET insert_allow_materialized_columns = 1; + CREATE TABLE test.nums ( n UInt64, m UInt64 MATERIALIZED n+1 ) ENGINE = Log; CREATE TABLE test.nums_buf AS test.nums ENGINE = Buffer(test, nums, 1, 10, 100, 1, 3, 10000000, 100000000); @@ -11,10 +13,7 @@ INSERT INTO test.nums_buf (n) VALUES (4); INSERT INTO test.nums_buf (n) VALUES (5); SELECT n,m FROM test.nums ORDER BY n; - - SELECT n,m FROM test.nums_buf ORDER BY n; - DROP TABLE IF EXISTS test.nums; DROP TABLE IF EXISTS test.nums_buf;