提交 66dd568e 编写于 作者: Z zhang2014

ISSUES-957 update condition & test

上级 6f28e0a6
......@@ -572,7 +572,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
out = std::make_shared<AddingDefaultBlockOutputStream>(
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<ProhibitColumnsBlockOutputStream>(out, columns.materialized_columns);
BlockIO io;
......
......@@ -61,7 +61,8 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
else if (typeid_cast<ASTInsertQuery *>(query.get()))
{
/// readonly is checked inside InterpreterInsertQuery
return std::make_unique<InterpreterInsertQuery>(query, context);
bool allow_materialized = static_cast<bool>(context.getSettingsRef().insert_allow_materialized_columns);
return std::make_unique<InterpreterInsertQuery>(query, context, allow_materialized);
}
else if (typeid_cast<ASTCreateQuery *>(query.get()))
{
......
......@@ -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<AddingDefaultBlockOutputStream>(
out, required_columns, table->column_defaults, context, static_cast<bool>(context.getSettingsRef().strict_insert_defaults));
if (context.getSettingsRef().insert_allow_materialized_columns)
if (!allow_materialized)
out = std::make_shared<ProhibitColumnsBlockOutputStream>(out, table->materialized_columns);
out = std::make_shared<SquashingBlockOutputStream>(
......
......@@ -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;
};
......
......@@ -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<bool>(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<ASTIdentifier>(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<bool>(args.local_context.getSettingsRef().insert_allow_materialized_columns));
});
}
......
......@@ -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_);
};
}
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;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册