mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
ISSUES-957 update condition & test
This commit is contained in:
parent
6f28e0a66a
commit
66dd568e64
@ -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_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user