Merge pull request #26639 from ClickHouse/fix-bad-cast

Fix bad cast
This commit is contained in:
Nikolai Kochetov 2021-08-19 11:17:26 +03:00 committed by GitHub
commit 3e44ef437e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 264 additions and 189 deletions

View File

@ -5,6 +5,7 @@
#include <Common/SipHash.h>
#include <Common/assert_cast.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/IDataType.h>
namespace DB

View File

@ -1,6 +1,6 @@
#pragma once
#include <Core/Block.h>
#include <Columns/IColumn.h>
namespace DB

View File

@ -22,6 +22,85 @@ namespace ErrorCodes
extern const int POSITION_OUT_OF_BOUND;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int AMBIGUOUS_COLUMN_NAME;
}
template <typename ReturnType>
static ReturnType onError(const std::string & message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception(message, code);
else
return false;
};
template <typename ReturnType>
static ReturnType checkColumnStructure(const ColumnWithTypeAndName & actual, const ColumnWithTypeAndName & expected,
const std::string & context_description, bool allow_remove_constants, int code)
{
if (actual.name != expected.name)
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different names of columns:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
if (!actual.type->equals(*expected.type))
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different types:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
if (!actual.column || !expected.column)
return ReturnType(true);
const IColumn * actual_column = actual.column.get();
/// If we allow to remove constants, and expected column is not const, then unwrap actual constant column.
if (allow_remove_constants && !isColumnConst(*expected.column))
{
if (const auto * column_const = typeid_cast<const ColumnConst *>(actual_column))
actual_column = &column_const->getDataColumn();
}
if (actual_column->getName() != expected.column->getName())
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different columns:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
if (isColumnConst(*actual.column) && isColumnConst(*expected.column))
{
Field actual_value = assert_cast<const ColumnConst &>(*actual.column).getField();
Field expected_value = assert_cast<const ColumnConst &>(*expected.column).getField();
if (actual_value != expected_value)
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different values of constants, actual: "
+ applyVisitor(FieldVisitorToString(), actual_value) + ", expected: " + applyVisitor(FieldVisitorToString(), expected_value),
code);
}
return ReturnType(true);
}
template <typename ReturnType>
static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description, bool allow_remove_constants)
{
size_t columns = rhs.columns();
if (lhs.columns() != columns)
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different number of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < columns; ++i)
{
const auto & actual = lhs.getByPosition(i);
const auto & expected = rhs.getByPosition(i);
if constexpr (std::is_same_v<ReturnType, bool>)
{
if (!checkColumnStructure<ReturnType>(actual, expected, context_description, allow_remove_constants, ErrorCodes::LOGICAL_ERROR))
return false;
}
else
checkColumnStructure<ReturnType>(actual, expected, context_description, allow_remove_constants, ErrorCodes::LOGICAL_ERROR);
}
return ReturnType(true);
}
@ -57,24 +136,41 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem)
throw Exception("Position out of bound in Block::insert(), max position = "
+ toString(data.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
if (elem.name.empty())
throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
for (auto & name_pos : index_by_name)
if (name_pos.second >= position)
++name_pos.second;
index_by_name.emplace(elem.name, position);
auto [it, inserted] = index_by_name.emplace(elem.name, position);
if (!inserted)
checkColumnStructure<void>(data[it->second], elem,
"(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME);
data.emplace(data.begin() + position, std::move(elem));
}
void Block::insert(ColumnWithTypeAndName elem)
{
index_by_name.emplace(elem.name, data.size());
if (elem.name.empty())
throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
auto [it, inserted] = index_by_name.emplace(elem.name, data.size());
if (!inserted)
checkColumnStructure<void>(data[it->second], elem,
"(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME);
data.emplace_back(std::move(elem));
}
void Block::insertUnique(ColumnWithTypeAndName elem)
{
if (elem.name.empty())
throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
if (index_by_name.end() == index_by_name.find(elem.name))
insert(std::move(elem));
}
@ -487,67 +583,6 @@ DataTypes Block::getDataTypes() const
}
template <typename ReturnType>
static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description, bool allow_remove_constants)
{
auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception(message, code);
else
return false;
};
size_t columns = rhs.columns();
if (lhs.columns() != columns)
return on_error("Block structure mismatch in " + context_description + " stream: different number of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < columns; ++i)
{
const auto & expected = rhs.getByPosition(i);
const auto & actual = lhs.getByPosition(i);
if (actual.name != expected.name)
return on_error("Block structure mismatch in " + context_description + " stream: different names of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
if (!actual.type->equals(*expected.type))
return on_error("Block structure mismatch in " + context_description + " stream: different types:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
if (!actual.column || !expected.column)
continue;
const IColumn * actual_column = actual.column.get();
/// If we allow to remove constants, and expected column is not const, then unwrap actual constant column.
if (allow_remove_constants && !isColumnConst(*expected.column))
{
if (const auto * column_const = typeid_cast<const ColumnConst *>(actual_column))
actual_column = &column_const->getDataColumn();
}
if (actual_column->getName() != expected.column->getName())
return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
if (isColumnConst(*actual.column) && isColumnConst(*expected.column))
{
Field actual_value = assert_cast<const ColumnConst &>(*actual.column).getField();
Field expected_value = assert_cast<const ColumnConst &>(*expected.column).getField();
if (actual_value != expected_value)
return on_error("Block structure mismatch in " + context_description + " stream: different values of constants, actual: "
+ applyVisitor(FieldVisitorToString(), actual_value) + ", expected: " + applyVisitor(FieldVisitorToString(), expected_value),
ErrorCodes::LOGICAL_ERROR);
}
}
return ReturnType(true);
}
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs)
{
return checkBlockStructure<bool>(lhs, rhs, {}, false);

View File

@ -104,7 +104,7 @@ public:
throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(),
ErrorCodes::ILLEGAL_COLUMN);
DB::Block columns_of_key_columns;
ColumnsWithTypeAndName columns_of_key_columns;
/// First argument may be a tuple or a single column.
const ColumnWithTypeAndName & left_arg = arguments[0];
@ -127,16 +127,16 @@ public:
const DataTypes & tuple_types = type_tuple->getElements();
size_t tuple_size = tuple_columns.size();
for (size_t i = 0; i < tuple_size; ++i)
columns_of_key_columns.insert({ tuple_columns[i], tuple_types[i], "" });
columns_of_key_columns.emplace_back(tuple_columns[i], tuple_types[i], "_" + toString(i));
}
else
columns_of_key_columns.insert(left_arg);
columns_of_key_columns.emplace_back(left_arg);
/// Replace single LowCardinality column to it's dictionary if possible.
ColumnPtr lc_indexes = nullptr;
if (columns_of_key_columns.columns() == 1)
if (columns_of_key_columns.size() == 1)
{
auto & arg = columns_of_key_columns.safeGetByPosition(0);
auto & arg = columns_of_key_columns.at(0);
const auto * col = arg.column.get();
if (const auto * const_col = typeid_cast<const ColumnConst *>(col))
col = &const_col->getDataColumn();

View File

@ -374,8 +374,8 @@ SetPtr makeExplicitSet(
SetPtr set
= std::make_shared<Set>(size_limits, create_ordered_set, context->getSettingsRef().transform_null_in);
set->setHeader(block.cloneEmpty());
set->insertFromBlock(block);
set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName());
set->insertFromBlock(block.getColumnsWithTypeAndName());
set->finishInsert();
prepared_sets[set_key] = set;

View File

@ -80,9 +80,13 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
Block scalar;
if (data.getContext()->hasQueryContext() && data.getContext()->getQueryContext()->hasScalar(scalar_query_hash_str))
{
scalar = data.getContext()->getQueryContext()->getScalar(scalar_query_hash_str);
}
else if (data.scalars.count(scalar_query_hash_str))
{
scalar = data.scalars[scalar_query_hash_str];
}
else
{
auto subquery_context = Context::createCopy(data.getContext());
@ -149,7 +153,8 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
Block tmp_block;
while (tmp_block.rows() == 0 && executor.pull(tmp_block));
while (tmp_block.rows() == 0 && executor.pull(tmp_block))
;
if (tmp_block.rows() != 0)
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
@ -173,10 +178,10 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
}
else
{
ColumnWithTypeAndName ctn;
ctn.type = std::make_shared<DataTypeTuple>(block.getDataTypes());
ctn.column = ColumnTuple::create(block.getColumns());
scalar.insert(ctn);
scalar.insert({
ColumnTuple::create(block.getColumns()),
std::make_shared<DataTypeTuple>(block.getDataTypes()),
"tuple"});
}
}

View File

@ -1037,12 +1037,15 @@ ExpressionActionsChain::JoinStep::JoinStep(
: Step({})
, analyzed_join(std::move(analyzed_join_))
, join(std::move(join_))
, result_columns(std::move(required_columns_))
{
for (const auto & column : result_columns)
for (const auto & column : required_columns_)
required_columns.emplace_back(column.name, column.type);
analyzed_join->addJoinedColumnsAndCorrectTypes(result_columns);
NamesAndTypesList result_names_and_types = required_columns;
analyzed_join->addJoinedColumnsAndCorrectTypes(result_names_and_types);
for (const auto & [name, type] : result_names_and_types)
/// `column` is `nullptr` because we don't care on constness here, it may be changed in join
result_columns.emplace_back(nullptr, type, name);
}
void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_)

View File

@ -216,7 +216,7 @@ void ExpressionAnalyzer::analyzeAggregation()
if (join)
{
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), true, temp_actions, false);
auto sample_columns = temp_actions->getResultColumns();
auto sample_columns = temp_actions->getNamesAndTypesList();
analyzedJoin().addJoinedColumnsAndCorrectTypes(sample_columns);
temp_actions = std::make_shared<ActionsDAG>(sample_columns);
}
@ -337,7 +337,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
PullingAsyncPipelineExecutor executor(io.pipeline);
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set, true, getContext()->getSettingsRef().transform_null_in);
set->setHeader(executor.getHeader());
set->setHeader(executor.getHeader().getColumnsWithTypeAndName());
Block block;
while (executor.pull(block))
@ -346,7 +346,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
continue;
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
if (!set->insertFromBlock(block))
if (!set->insertFromBlock(block.getColumnsWithTypeAndName()))
return;
}
@ -1206,7 +1206,7 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain,
}
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order,
ManyExpressionActions & order_by_elements_actions)
ManyExpressionActions & order_by_elements_actions)
{
const auto * select_query = getSelectQuery();

View File

@ -99,14 +99,14 @@ void NO_INLINE Set::insertFromBlockImplCase(
}
void Set::setHeader(const Block & header)
void Set::setHeader(const ColumnsWithTypeAndName & header)
{
std::unique_lock lock(rwlock);
if (!data.empty())
return;
keys_size = header.columns();
keys_size = header.size();
ColumnRawPtrs key_columns;
key_columns.reserve(keys_size);
data_types.reserve(keys_size);
@ -118,10 +118,10 @@ void Set::setHeader(const Block & header)
/// Remember the columns we will work with
for (size_t i = 0; i < keys_size; ++i)
{
materialized_columns.emplace_back(header.safeGetByPosition(i).column->convertToFullColumnIfConst());
materialized_columns.emplace_back(header.at(i).column->convertToFullColumnIfConst());
key_columns.emplace_back(materialized_columns.back().get());
data_types.emplace_back(header.safeGetByPosition(i).type);
set_elements_types.emplace_back(header.safeGetByPosition(i).type);
data_types.emplace_back(header.at(i).type);
set_elements_types.emplace_back(header.at(i).type);
/// Convert low cardinality column to full.
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(data_types.back().get()))
@ -161,7 +161,7 @@ void Set::setHeader(const Block & header)
}
bool Set::insertFromBlock(const Block & block)
bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns)
{
std::unique_lock lock(rwlock);
@ -177,11 +177,11 @@ bool Set::insertFromBlock(const Block & block)
/// Remember the columns we will work with
for (size_t i = 0; i < keys_size; ++i)
{
materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality());
materialized_columns.emplace_back(columns.at(i).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality());
key_columns.emplace_back(materialized_columns.back().get());
}
size_t rows = block.rows();
size_t rows = columns.at(0).column->size();
/// We will insert to the Set only keys, where all components are not NULL.
ConstNullMapPtr null_map{};
@ -192,7 +192,7 @@ bool Set::insertFromBlock(const Block & block)
/// Filter to extract distinct values from the block.
ColumnUInt8::MutablePtr filter;
if (fill_set_elements)
filter = ColumnUInt8::create(block.rows());
filter = ColumnUInt8::create(rows);
switch (data.type)
{
@ -224,16 +224,16 @@ bool Set::insertFromBlock(const Block & block)
}
ColumnPtr Set::execute(const Block & block, bool negative) const
ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const
{
size_t num_key_columns = block.columns();
size_t num_key_columns = columns.size();
if (0 == num_key_columns)
throw Exception("Logical error: no columns passed to Set::execute method.", ErrorCodes::LOGICAL_ERROR);
auto res = ColumnUInt8::create();
ColumnUInt8::Container & vec_res = res->getData();
vec_res.resize(block.safeGetByPosition(0).column->size());
vec_res.resize(columns.at(0).column->size());
if (vec_res.empty())
return res;
@ -264,7 +264,7 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
{
ColumnPtr result;
const auto & column_before_cast = block.safeGetByPosition(i);
const auto & column_before_cast = columns.at(i);
ColumnWithTypeAndName column_to_cast
= {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name};
@ -428,7 +428,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<K
SortDescription sort_description;
for (size_t i = 0; i < tuple_size; ++i)
{
block_to_sort.insert({ ordered_set[i], nullptr, "" });
block_to_sort.insert({ ordered_set[i], nullptr, "_" + toString(i) });
sort_description.emplace_back(i, 1, 1);
}

View File

@ -42,10 +42,10 @@ public:
/** Create a Set from stream.
* Call setHeader, then call insertFromBlock for each block.
*/
void setHeader(const Block & header);
void setHeader(const ColumnsWithTypeAndName & header);
/// Returns false, if some limit was exceeded and no need to insert more data.
bool insertFromBlock(const Block & block);
bool insertFromBlock(const ColumnsWithTypeAndName & columns);
/// Call after all blocks were inserted. To get the information that set is already created.
void finishInsert() { is_created = true; }
@ -54,7 +54,7 @@ public:
/** For columns of 'block', check belonging of corresponding rows to the set.
* Return UInt8 column with the result.
*/
ColumnPtr execute(const Block & block, bool negative) const;
ColumnPtr execute(const ColumnsWithTypeAndName & columns, bool negative) const;
bool empty() const;
size_t getTotalRowCount() const;

View File

@ -231,20 +231,7 @@ void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column)
void TableJoin::addJoinedColumnsAndCorrectTypes(NamesAndTypesList & names_and_types, bool correct_nullability) const
{
ColumnsWithTypeAndName columns;
for (auto & pair : names_and_types)
columns.emplace_back(nullptr, std::move(pair.type), std::move(pair.name));
names_and_types.clear();
addJoinedColumnsAndCorrectTypes(columns, correct_nullability);
for (auto & col : columns)
names_and_types.emplace_back(std::move(col.name), std::move(col.type));
}
void TableJoin::addJoinedColumnsAndCorrectTypes(ColumnsWithTypeAndName & columns, bool correct_nullability) const
{
for (auto & col : columns)
for (auto & col : names_and_types)
{
if (hasUsing())
{
@ -252,17 +239,12 @@ void TableJoin::addJoinedColumnsAndCorrectTypes(ColumnsWithTypeAndName & columns
col.type = it->second;
}
if (correct_nullability && leftBecomeNullable(col.type))
{
/// No need to nullify constants
bool is_column_const = col.column && isColumnConst(*col.column);
if (!is_column_const)
col.type = JoinCommon::convertTypeToNullable(col.type);
}
col.type = JoinCommon::convertTypeToNullable(col.type);
}
/// Types in columns_added_by_join already converted and set nullable if needed
for (const auto & col : columns_added_by_join)
columns.emplace_back(nullptr, col.type, col.name);
names_and_types.emplace_back(col.name, col.type);
}
bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTTableJoin::Kind kind_) const

View File

@ -191,7 +191,6 @@ public:
void addJoinedColumn(const NameAndTypePair & joined_column);
void addJoinedColumnsAndCorrectTypes(NamesAndTypesList & names_and_types, bool correct_nullability = true) const;
void addJoinedColumnsAndCorrectTypes(ColumnsWithTypeAndName & columns, bool correct_nullability = true) const;
/// Calculates common supertypes for corresponding join key columns.
bool inferJoinKeyCommonType(const NamesAndTypesList & left, const NamesAndTypesList & right);

View File

@ -18,6 +18,7 @@
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <unordered_map>
namespace DB
{
@ -121,6 +122,7 @@ std::tuple<bool, ASTPtr> evaluateDatabaseNameForMergeEngine(const ASTPtr & node,
return std::tuple{false, ast};
}
namespace
{
using Conjunction = ColumnsWithTypeAndName;
@ -213,7 +215,7 @@ namespace
Disjunction result;
auto add_dnf = [&](const auto &dnf)
auto add_dnf = [&](const auto & dnf)
{
if (dnf.size() > limit)
{
@ -338,6 +340,7 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
if (const auto * fn = node->as<ASTFunction>())
{
std::unordered_map<std::string, bool> always_false_map;
const auto dnf = analyzeFunction(fn, target_expr, limit);
if (dnf.empty() || !limit)
@ -368,7 +371,41 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
for (const auto & conjunct : dnf)
{
Block block(conjunct);
Block block;
for (const auto & elem : conjunct)
{
if (!block.has(elem.name))
{
block.insert(elem);
}
else
{
/// Conjunction of condition on column equality to distinct values can never be satisfied.
const ColumnWithTypeAndName & prev = block.getByName(elem.name);
if (isColumnConst(*prev.column) && isColumnConst(*elem.column))
{
Field prev_value = assert_cast<const ColumnConst &>(*prev.column).getField();
Field curr_value = assert_cast<const ColumnConst &>(*elem.column).getField();
if (!always_false_map.count(elem.name))
{
always_false_map[elem.name] = prev_value != curr_value;
}
else
{
auto & always_false = always_false_map[elem.name];
/// If at least one of conjunct is not always false, we should preserve this.
if (always_false)
{
always_false = prev_value != curr_value;
}
}
}
}
}
// Block should contain all required columns from `target_expr`
if (!has_required_columns(block))
@ -393,6 +430,11 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
return {};
}
}
bool any_always_false = std::any_of(always_false_map.begin(), always_false_map.end(), [](const auto & v) { return v.second; });
if (any_always_false)
return Blocks{};
}
else if (const auto * literal = node->as<ASTLiteral>())
{

View File

@ -84,7 +84,7 @@ void CreatingSetsTransform::init()
is_initialized = true;
if (subquery.set)
subquery.set->setHeader(getInputPort().getHeader());
subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName());
watch.restart();
startSubquery();
@ -97,7 +97,7 @@ void CreatingSetsTransform::consume(Chunk chunk)
if (!done_with_set)
{
if (!subquery.set->insertFromBlock(block))
if (!subquery.set->insertFromBlock(block.getColumnsWithTypeAndName()))
done_with_set = true;
}

View File

@ -174,8 +174,9 @@ private:
/// - optimize_skip_unused_shards
/// - force_optimize_skip_unused_shards
ClusterPtr getOptimizedCluster(ContextPtr, const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query_ptr) const;
ClusterPtr
skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) const;
ClusterPtr skipUnusedShards(
ClusterPtr cluster, const ASTPtr & query_ptr, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) const;
/// This method returns optimal query processing stage.
///

View File

@ -435,11 +435,17 @@ Pipe StorageMerge::createSources(
if (!pipe.empty())
{
if (concat_streams && pipe.numOutputPorts() > 1)
{
// It's possible to have many tables read from merge, resize(1) might open too many files at the same time.
// Using concat instead.
pipe.addTransform(std::make_shared<ConcatProcessor>(pipe.getHeader(), pipe.numOutputPorts()));
}
if (has_database_virtual_column)
/// Add virtual columns if we don't already have them.
Block pipe_header = pipe.getHeader();
if (has_database_virtual_column && !pipe_header.has("_database"))
{
ColumnWithTypeAndName column;
column.name = "_database";
@ -457,7 +463,7 @@ Pipe StorageMerge::createSources(
});
}
if (has_table_virtual_column)
if (has_table_virtual_column && !pipe_header.has("_table"))
{
ColumnWithTypeAndName column;
column.name = "_table";

View File

@ -146,13 +146,13 @@ StorageSet::StorageSet(
Block header = getInMemoryMetadataPtr()->getSampleBlock();
header = header.sortColumns();
set->setHeader(header);
set->setHeader(header.getColumnsWithTypeAndName());
restore();
}
void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); }
void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block.getColumnsWithTypeAndName()); }
void StorageSet::finishInsert() { set->finishInsert(); }
size_t StorageSet::getSize() const { return set->getTotalRowCount(); }
@ -170,7 +170,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn
increment = 0;
set = std::make_shared<Set>(SizeLimits(), false, true);
set->setHeader(header);
set->setHeader(header.getColumnsWithTypeAndName());
}

View File

@ -97,12 +97,12 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context)
auto stream = interpreter_subquery->execute().getInputStream();
SizeLimits limites(context->getSettingsRef().max_rows_in_set, context->getSettingsRef().max_bytes_in_set, OverflowMode::THROW);
Set set(limites, true, context->getSettingsRef().transform_null_in);
set.setHeader(stream->getHeader());
set.setHeader(stream->getHeader().getColumnsWithTypeAndName());
stream->readPrefix();
while (Block block = stream->read())
{
set.insertFromBlock(block);
set.insertFromBlock(block.getColumnsWithTypeAndName());
}
set.finishInsert();
stream->readSuffix();

View File

@ -128,6 +128,7 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context)
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt64>();
col.name = "a";
sample.insert(std::move(col));
}

View File

@ -19,10 +19,10 @@
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<!--<type>UInt32</type>-->
</id>
<attribute>
<name>id</name>
<name>key</name>
<type>UInt32</type>
<null_value></null_value>
</attribute>
@ -65,10 +65,10 @@
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<!--<type>UInt32</type>-->
</id>
<attribute>
<name>id</name>
<name>key</name>
<type>UInt32</type>
<null_value></null_value>
</attribute>

View File

@ -13,11 +13,11 @@ node1 = cluster.add_instance('node1',
postgres_dict_table_template = """
CREATE TABLE IF NOT EXISTS {} (
id Integer NOT NULL, value Integer NOT NULL, PRIMARY KEY (id))
id Integer NOT NULL, key Integer NOT NULL, value Integer NOT NULL, PRIMARY KEY (id))
"""
click_dict_table_template = """
CREATE TABLE IF NOT EXISTS `test`.`dict_table_{}` (
`id` UInt64, `value` UInt32
`key` UInt32, `value` UInt32
) ENGINE = Dictionary({})
"""
@ -43,7 +43,7 @@ def create_and_fill_postgres_table(cursor, table_name, port, host):
create_postgres_table(cursor, table_name)
# Fill postgres table using clickhouse postgres table function and check
table_func = '''postgresql('{}:{}', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(host, port, table_name)
node1.query('''INSERT INTO TABLE FUNCTION {} SELECT number, number from numbers(10000)
node1.query('''INSERT INTO TABLE FUNCTION {} SELECT number, number, number from numbers(10000)
'''.format(table_func, table_name))
result = node1.query("SELECT count() FROM {}".format(table_func))
assert result.rstrip() == '10000'
@ -82,7 +82,7 @@ def test_load_dictionaries(started_cluster):
node1.query("SYSTEM RELOAD DICTIONARY {}".format(dict_name))
assert node1.query("SELECT count() FROM `test`.`dict_table_{}`".format(table_name)).rstrip() == '10000'
assert node1.query("SELECT dictGetUInt32('{}', 'id', toUInt64(0))".format(dict_name)) == '0\n'
assert node1.query("SELECT dictGetUInt32('{}', 'key', toUInt64(0))".format(dict_name)) == '0\n'
assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(9999))".format(dict_name)) == '9999\n'
cursor.execute("DROP TABLE IF EXISTS {}".format(table_name))
@ -252,11 +252,11 @@ def test_dictionary_with_replicas(started_cluster):
create_postgres_table(cursor1, 'test1')
create_postgres_table(cursor2, 'test1')
cursor1.execute('INSERT INTO test1 select i, i from generate_series(0, 99) as t(i);');
cursor2.execute('INSERT INTO test1 select i, i from generate_series(100, 199) as t(i);');
cursor1.execute('INSERT INTO test1 select i, i, i from generate_series(0, 99) as t(i);')
cursor2.execute('INSERT INTO test1 select i, i, i from generate_series(100, 199) as t(i);')
create_dict('test1', 1)
result = node1.query("SELECT * FROM `test`.`dict_table_test1` ORDER BY id")
result = node1.query("SELECT * FROM `test`.`dict_table_test1` ORDER BY key")
# priority 0 - non running port
assert node1.contains_in_log('PostgreSQLConnectionPool: Connection error*')

View File

@ -18,7 +18,7 @@
<structure>
<id>
<name>column1</name>
<name>id</name>
</id>
<attribute>

View File

@ -20,7 +20,7 @@
<structure>
<id>
<name>X</name>
<name>id</name>
</id>
<attribute>

View File

@ -20,7 +20,7 @@
<structure>
<id>
<name>X</name>
<name>id</name>
</id>
<attribute>

View File

@ -99,19 +99,19 @@ def started_cluster():
logging.debug(f"sqlite data received: {sqlite_db}")
node1.exec_in_container(
["sqlite3", sqlite_db, "CREATE TABLE t1(x INTEGER PRIMARY KEY ASC, y, z);"],
["sqlite3", sqlite_db, "CREATE TABLE t1(id INTEGER PRIMARY KEY ASC, x INTEGER, y, z);"],
privileged=True, user='root')
node1.exec_in_container(
["sqlite3", sqlite_db, "CREATE TABLE t2(X INTEGER PRIMARY KEY ASC, Y, Z);"],
["sqlite3", sqlite_db, "CREATE TABLE t2(id INTEGER PRIMARY KEY ASC, X INTEGER, Y, Z);"],
privileged=True, user='root')
node1.exec_in_container(
["sqlite3", sqlite_db, "CREATE TABLE t3(X INTEGER PRIMARY KEY ASC, Y, Z);"],
["sqlite3", sqlite_db, "CREATE TABLE t3(id INTEGER PRIMARY KEY ASC, X INTEGER, Y, Z);"],
privileged=True, user='root')
node1.exec_in_container(
["sqlite3", sqlite_db, "CREATE TABLE t4(X INTEGER PRIMARY KEY ASC, Y, Z);"],
["sqlite3", sqlite_db, "CREATE TABLE t4(id INTEGER PRIMARY KEY ASC, X INTEGER, Y, Z);"],
privileged=True, user='root')
node1.exec_in_container(
["sqlite3", sqlite_db, "CREATE TABLE tf1(x INTEGER PRIMARY KEY ASC, y, z);"],
["sqlite3", sqlite_db, "CREATE TABLE tf1(id INTEGER PRIMARY KEY ASC, x INTEGER, y, z);"],
privileged=True, user='root')
logging.debug("sqlite tables created")
mysql_conn = get_mysql_conn()
@ -128,7 +128,7 @@ def started_cluster():
cursor = postgres_conn.cursor()
cursor.execute(
"create table if not exists clickhouse.test_table (column1 int primary key, column2 varchar(40) not null)")
"create table if not exists clickhouse.test_table (id int primary key, column1 int not null, column2 varchar(40) not null)")
yield cluster
@ -210,9 +210,9 @@ def test_sqlite_simple_select_function_works(started_cluster):
sqlite_setup = node1.odbc_drivers["SQLite3"]
sqlite_db = sqlite_setup["Database"]
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t1 values(1, 2, 3);"],
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t1 values(1, 1, 2, 3);"],
privileged=True, user='root')
assert node1.query("select * from odbc('DSN={}', '{}')".format(sqlite_setup["DSN"], 't1')) == "1\t2\t3\n"
assert node1.query("select * from odbc('DSN={}', '{}')".format(sqlite_setup["DSN"], 't1')) == "1\t1\t2\t3\n"
assert node1.query("select y from odbc('DSN={}', '{}')".format(sqlite_setup["DSN"], 't1')) == "2\n"
assert node1.query("select z from odbc('DSN={}', '{}')".format(sqlite_setup["DSN"], 't1')) == "3\n"
@ -228,10 +228,10 @@ def test_sqlite_table_function(started_cluster):
sqlite_setup = node1.odbc_drivers["SQLite3"]
sqlite_db = sqlite_setup["Database"]
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO tf1 values(1, 2, 3);"],
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO tf1 values(1, 1, 2, 3);"],
privileged=True, user='root')
node1.query("create table odbc_tf as odbc('DSN={}', '{}')".format(sqlite_setup["DSN"], 'tf1'))
assert node1.query("select * from odbc_tf") == "1\t2\t3\n"
assert node1.query("select * from odbc_tf") == "1\t1\t2\t3\n"
assert node1.query("select y from odbc_tf") == "2\n"
assert node1.query("select z from odbc_tf") == "3\n"
@ -246,7 +246,7 @@ def test_sqlite_simple_select_storage_works(started_cluster):
sqlite_setup = node1.odbc_drivers["SQLite3"]
sqlite_db = sqlite_setup["Database"]
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t4 values(1, 2, 3);"],
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t4 values(1, 1, 2, 3);"],
privileged=True, user='root')
node1.query("create table SqliteODBC (x Int32, y String, z String) engine = ODBC('DSN={}', '', 't4')".format(
sqlite_setup["DSN"]))
@ -264,7 +264,7 @@ def test_sqlite_odbc_hashed_dictionary(started_cluster):
skip_test_msan(node1)
sqlite_db = node1.odbc_drivers["SQLite3"]["Database"]
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t2 values(1, 2, 3);"],
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t2 values(1, 1, 2, 3);"],
privileged=True, user='root')
node1.query("SYSTEM RELOAD DICTIONARY sqlite3_odbc_hashed")
@ -282,7 +282,7 @@ def test_sqlite_odbc_hashed_dictionary(started_cluster):
logging.debug("Waiting dictionary to update for the second time")
time.sleep(0.1)
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t2 values(200, 2, 7);"],
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t2 values(200, 200, 2, 7);"],
privileged=True, user='root')
# No reload because of invalidate query
@ -299,7 +299,7 @@ def test_sqlite_odbc_hashed_dictionary(started_cluster):
assert_eq_with_retry(node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(1))", "3")
assert_eq_with_retry(node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(200))", "1") # still default
node1.exec_in_container(["sqlite3", sqlite_db, "REPLACE INTO t2 values(1, 2, 5);"],
node1.exec_in_container(["sqlite3", sqlite_db, "REPLACE INTO t2 values(1, 1, 2, 5);"],
privileged=True, user='root')
assert_eq_with_retry(node1, "select dictGetUInt8('sqlite3_odbc_hashed', 'Z', toUInt64(1))", "5")
@ -310,7 +310,7 @@ def test_sqlite_odbc_cached_dictionary(started_cluster):
skip_test_msan(node1)
sqlite_db = node1.odbc_drivers["SQLite3"]["Database"]
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t3 values(1, 2, 3);"],
node1.exec_in_container(["sqlite3", sqlite_db, "INSERT INTO t3 values(1, 1, 2, 3);"],
privileged=True, user='root')
assert node1.query("select dictGetUInt8('sqlite3_odbc_cached', 'Z', toUInt64(1))") == "3\n"
@ -319,12 +319,12 @@ def test_sqlite_odbc_cached_dictionary(started_cluster):
node1.exec_in_container(["chmod", "a+rw", "/tmp"], privileged=True, user='root')
node1.exec_in_container(["chmod", "a+rw", sqlite_db], privileged=True, user='root')
node1.query("insert into table function odbc('DSN={};ReadOnly=0', '', 't3') values (200, 2, 7)".format(
node1.query("insert into table function odbc('DSN={};ReadOnly=0', '', 't3') values (200, 200, 2, 7)".format(
node1.odbc_drivers["SQLite3"]["DSN"]))
assert node1.query("select dictGetUInt8('sqlite3_odbc_cached', 'Z', toUInt64(200))") == "7\n" # new value
node1.exec_in_container(["sqlite3", sqlite_db, "REPLACE INTO t3 values(1, 2, 12);"],
node1.exec_in_container(["sqlite3", sqlite_db, "REPLACE INTO t3 values(1, 1, 2, 12);"],
privileged=True, user='root')
assert_eq_with_retry(node1, "select dictGetUInt8('sqlite3_odbc_cached', 'Z', toUInt64(1))", "12")
@ -336,7 +336,7 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster):
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute("truncate table clickhouse.test_table")
cursor.execute("insert into clickhouse.test_table values(1, 'hello'),(2, 'world')")
cursor.execute("insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')")
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed")
assert_eq_with_retry(node1, "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))", "hello")
assert_eq_with_retry(node1, "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", "world")
@ -348,7 +348,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster):
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute("truncate table clickhouse.test_table")
cursor.execute("insert into clickhouse.test_table values(3, 'xxx')")
cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')")
for i in range(100):
try:
node1.query("system reload dictionary postgres_odbc_hashed", timeout=15)
@ -369,13 +369,13 @@ def test_postgres_insert(started_cluster):
# reconstruction of connection string.
node1.query(
"create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')")
node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')")
assert node1.query("select * from pg_insert") == '1\thello\n2\tworld\n'
node1.query("insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,test")
"create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')")
node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')")
assert node1.query("select * from pg_insert") == '1\t1\thello\n2\t2\tworld\n'
node1.query("insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test")
node1.query(
"insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" \
" select number, 's' || toString(number) from numbers (4, 7)")
" select number, number, 's' || toString(number) from numbers (4, 7)")
assert node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n"
assert node1.query(
"select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))") == "55\t10\n"
@ -426,19 +426,19 @@ def test_odbc_postgres_date_data_type(started_cluster):
conn = get_postgres_conn(started_cluster);
cursor = conn.cursor()
cursor.execute("CREATE TABLE IF NOT EXISTS clickhouse.test_date (column1 integer, column2 date)")
cursor.execute("CREATE TABLE IF NOT EXISTS clickhouse.test_date (id integer, column1 integer, column2 date)")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, '2020-12-01')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, '2020-12-02')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, '2020-12-03')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')")
conn.commit()
node1.query(
'''
CREATE TABLE test_date (column1 UInt64, column2 Date)
CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date)
ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')''')
expected = '1\t2020-12-01\n2\t2020-12-02\n3\t2020-12-03\n'
expected = '1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n'
result = node1.query('SELECT * FROM test_date');
assert(result == expected)
cursor.execute("DROP TABLE IF EXISTS clickhouse.test_date")

View File

@ -4,7 +4,6 @@
11 12
12 11
10 10
10 11 11
12 11
10 12
11 12

View File

@ -5,7 +5,7 @@ select s.a + 1 as b, s.a + 2 as a from (select 10 as a) s;
select s.a + 2 as b, s.a + 1 as a from (select 10 as a) s;
select a, a as a from (select 10 as a);
select s.a, a, a + 1 as a from (select 10 as a) as s;
select s.a, a, a + 1 as a from (select 10 as a) as s; -- { serverError 352 }
select s.a + 2 as b, b - 1 as a from (select 10 as a) s;
select s.a as a, s.a + 2 as b from (select 10 as a) s;
select s.a + 1 as a, s.a + 2 as b from (select 10 as a) s;

View File

@ -3,7 +3,6 @@
7 0
7 1
xyzabc 2
1 3
1 2 0 0
1 0 0 3
\N 1 2 \N 0

View File

@ -7,7 +7,7 @@ join (select '1' as sid) as t2 on t2.sid = cast(t1.iid as String);
select cast(7 as String), * from (select 3 "'String'");
select cast(7 as String), * from (select number "'String'" FROM numbers(2));
SELECT concat('xyz', 'abc'), * FROM (SELECT 2 AS "'xyz'");
with 3 as "1" select 1, "1";
with 3 as "1" select 1, "1"; -- { serverError 352 }
-- https://github.com/ClickHouse/ClickHouse/issues/9953
select 1, * from (select 2 x) a left join (select 1, 3 y) b on y = x;

View File

@ -13,7 +13,7 @@ WITH dates AS
today - INTERVAL 3 day as older_date
)
-- Newer than 2 days are kept in windows of 600 seconds
select 1, 'sum_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all
select 1 AS key, 'sum_1' AS s, today - number * 60 - 30, number, 1, number from dates, numbers(300) union all
select 2, 'sum_1', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all
select 1, 'sum_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all
select 2, 'sum_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all
@ -23,7 +23,7 @@ WITH dates AS
select 2, 'max_2', today - number * 60 - 30, number, 1, number from dates, numbers(300) union all
-- Older than 2 days use 6000 second windows
select 1, 'sum_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all
select 1 AS key, 'sum_1' AS s, older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all
select 2, 'sum_1', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all
select 1, 'sum_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all
select 2, 'sum_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all

View File

@ -0,0 +1,2 @@
SELECT 1, * FROM (SELECT NULL AS `1`); -- { serverError 352 }
SELECT '7', 'xyz', * FROM (SELECT NULL AS `'xyz'`); -- { serverError 352 }

View File

@ -1,2 +1,2 @@
finished test_01948_tcp_default default SELECT * FROM\n (\n SELECT a.name as n\n FROM\n (\n SELECT \'Name\' as name, number FROM system.numbers LIMIT 2000000\n ) AS a,\n (\n SELECT \'Name\' as name, number FROM system.numbers LIMIT 2000000\n ) as b\n GROUP BY n\n )\n LIMIT 20\n FORMAT Null
finished test_01948_http_default default SELECT * FROM\n (\n SELECT a.name as n\n FROM\n (\n SELECT \'Name\' as name, number FROM system.numbers LIMIT 2000000\n ) AS a,\n (\n SELECT \'Name\' as name, number FROM system.numbers LIMIT 2000000\n ) as b\n GROUP BY n\n )\n LIMIT 20\n FORMAT Null
finished test_01948_tcp_default default SELECT * FROM\n (\n SELECT a.name as n\n FROM\n (\n SELECT \'Name\' as name, number FROM system.numbers LIMIT 2000000\n ) AS a,\n (\n SELECT \'Name\' as name2, number FROM system.numbers LIMIT 2000000\n ) as b\n GROUP BY n\n )\n LIMIT 20\n FORMAT Null
finished test_01948_http_default default SELECT * FROM\n (\n SELECT a.name as n\n FROM\n (\n SELECT \'Name\' as name, number FROM system.numbers LIMIT 2000000\n ) AS a,\n (\n SELECT \'Name\' as name2, number FROM system.numbers LIMIT 2000000\n ) as b\n GROUP BY n\n )\n LIMIT 20\n FORMAT Null

View File

@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT --max_execution_time 10 --query_id "test_01948_tcp_$CLICKHOUS
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
) AS a,
(
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000
) as b
GROUP BY n
)
@ -44,7 +44,7 @@ ${CLICKHOUSE_CURL_COMMAND} -q --max-time 10 -sS "$CLICKHOUSE_URL&query_id=test_0
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
) AS a,
(
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000
) as b
GROUP BY n
)