mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge branch 'ClickHouse:master' into group_sorted_array_function
This commit is contained in:
commit
c5226ff2f5
2
contrib/boost
vendored
2
contrib/boost
vendored
@ -1 +1 @@
|
||||
Subproject commit aec12eea7fc762721ae16943d1361340c66c9c17
|
||||
Subproject commit bb179652862b528d94a9032a784796c4db846c3f
|
@ -172,9 +172,9 @@ endif()
|
||||
# coroutine
|
||||
|
||||
set (SRCS_COROUTINE
|
||||
"${LIBRARY_DIR}/libs/coroutine/detail/coroutine_context.cpp"
|
||||
"${LIBRARY_DIR}/libs/coroutine/exceptions.cpp"
|
||||
"${LIBRARY_DIR}/libs/coroutine/posix/stack_traits.cpp"
|
||||
"${LIBRARY_DIR}/libs/coroutine/src/detail/coroutine_context.cpp"
|
||||
"${LIBRARY_DIR}/libs/coroutine/src/exceptions.cpp"
|
||||
"${LIBRARY_DIR}/libs/coroutine/src/posix/stack_traits.cpp"
|
||||
)
|
||||
add_library (_boost_coroutine ${SRCS_COROUTINE})
|
||||
add_library (boost::coroutine ALIAS _boost_coroutine)
|
||||
|
@ -26,9 +26,9 @@ SELECT p, toTypeName(p) FROM geo_point;
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─p─────┬─toTypeName(p)─┐
|
||||
┌─p───────┬─toTypeName(p)─┐
|
||||
│ (10,10) │ Point │
|
||||
└───────┴───────────────┘
|
||||
└─────────┴───────────────┘
|
||||
```
|
||||
|
||||
## Ring
|
||||
|
@ -32,6 +32,12 @@ contents:
|
||||
dst: /usr/bin/clickhouse-keeper
|
||||
- src: clickhouse-keeper.service
|
||||
dst: /lib/systemd/system/clickhouse-keeper.service
|
||||
- src: clickhouse
|
||||
dst: /usr/bin/clickhouse-keeper-client
|
||||
type: symlink
|
||||
- src: clickhouse
|
||||
dst: /usr/bin/clickhouse-keeper-converter
|
||||
type: symlink
|
||||
# docs
|
||||
- src: ../AUTHORS
|
||||
dst: /usr/share/doc/clickhouse-keeper/AUTHORS
|
||||
|
@ -303,6 +303,11 @@ ColumnPtr emptyNotNullableClone(const ColumnPtr & column)
|
||||
return column->cloneEmpty();
|
||||
}
|
||||
|
||||
ColumnPtr materializeColumn(const ColumnPtr & column)
|
||||
{
|
||||
return recursiveRemoveLowCardinality(recursiveRemoveSparse(column->convertToFullColumnIfConst()));
|
||||
}
|
||||
|
||||
ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names)
|
||||
{
|
||||
ColumnRawPtrs ptrs;
|
||||
@ -311,7 +316,7 @@ ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names)
|
||||
for (const auto & column_name : names)
|
||||
{
|
||||
auto & column = block.getByName(column_name).column;
|
||||
column = recursiveRemoveLowCardinality(recursiveRemoveSparse(column->convertToFullColumnIfConst()));
|
||||
column = materializeColumn(column);
|
||||
ptrs.push_back(column.get());
|
||||
}
|
||||
|
||||
@ -326,12 +331,7 @@ ColumnPtrMap materializeColumnsInplaceMap(const Block & block, const Names & nam
|
||||
for (const auto & column_name : names)
|
||||
{
|
||||
ColumnPtr column = block.getByName(column_name).column;
|
||||
|
||||
column = column->convertToFullColumnIfConst();
|
||||
column = recursiveRemoveLowCardinality(column);
|
||||
column = recursiveRemoveSparse(column);
|
||||
|
||||
ptrs[column_name] = column;
|
||||
ptrs[column_name] = materializeColumn(column);
|
||||
}
|
||||
|
||||
return ptrs;
|
||||
@ -340,8 +340,7 @@ ColumnPtrMap materializeColumnsInplaceMap(const Block & block, const Names & nam
|
||||
ColumnPtr materializeColumn(const Block & block, const String & column_name)
|
||||
{
|
||||
const auto & src_column = block.getByName(column_name).column;
|
||||
return recursiveRemoveLowCardinality(
|
||||
recursiveRemoveSparse(src_column->convertToFullColumnIfConst()));
|
||||
return materializeColumn(src_column);
|
||||
}
|
||||
|
||||
Columns materializeColumns(const Block & block, const Names & names)
|
||||
@ -539,7 +538,7 @@ JoinMask getColumnAsMask(const Block & block, const String & column_name)
|
||||
return JoinMask(const_cond->getBool(0), block.rows());
|
||||
}
|
||||
|
||||
ColumnPtr join_condition_col = recursiveRemoveLowCardinality(src_col.column->convertToFullColumnIfConst());
|
||||
ColumnPtr join_condition_col = materializeColumn(src_col.column);
|
||||
if (const auto * nullable_col = typeid_cast<const ColumnNullable *>(join_condition_col.get()))
|
||||
{
|
||||
if (isNothing(assert_cast<const DataTypeNullable &>(*col_type).getNestedType()))
|
||||
|
@ -198,7 +198,11 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context)
|
||||
CompletedPipelineExecutor executor(pipeline);
|
||||
executor.execute();
|
||||
|
||||
set_and_key->set->checkIsCreated();
|
||||
/// SET may not be created successfully at this step because of the sub-query timeout, but if we have
|
||||
/// timeout_overflow_mode set to `break`, no exception is thrown, and the executor just stops executing
|
||||
/// the pipeline without setting `set_and_key->set->is_created` to true.
|
||||
if (!set_and_key->set->isCreated())
|
||||
return nullptr;
|
||||
|
||||
return set_and_key->set;
|
||||
}
|
||||
|
@ -491,6 +491,10 @@ bool MergeTreeConditionInverted::traverseASTEquals(
|
||||
DataTypePtr const_type;
|
||||
if (argument.tryGetConstant(const_value, const_type))
|
||||
{
|
||||
auto const_data_type = WhichDataType(const_type);
|
||||
if (!const_data_type.isStringOrFixedString() && !const_data_type.isArray())
|
||||
return false;
|
||||
|
||||
key_column_num = header.getPositionByName(map_keys_index_column_name);
|
||||
key_exists = true;
|
||||
}
|
||||
|
@ -614,6 +614,9 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node)
|
||||
}
|
||||
else if (func->name == "and" || func->name == "indexHint")
|
||||
{
|
||||
if (args.size() < 2)
|
||||
return false;
|
||||
|
||||
auto last_arg = args.back();
|
||||
args.pop_back();
|
||||
|
||||
@ -633,6 +636,9 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node)
|
||||
}
|
||||
else if (func->name == "or")
|
||||
{
|
||||
if (args.size() < 2)
|
||||
return false;
|
||||
|
||||
auto last_arg = args.back();
|
||||
args.pop_back();
|
||||
|
||||
|
@ -0,0 +1,10 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
CREATE TABLE t (key UInt64, value UInt64, INDEX value_idx value TYPE bloom_filter GRANULARITY 1) ENGINE=MergeTree() ORDER BY key;
|
||||
|
||||
INSERT INTO t SELECT number, rand()%1000 FROM numbers(10000);
|
||||
|
||||
SET timeout_overflow_mode='break';
|
||||
SET max_execution_time=0.1;
|
||||
SELECT * FROM t WHERE value IN (SELECT number FROM numbers(1000000000));
|
||||
|
||||
DROP TABLE t;
|
@ -0,0 +1 @@
|
||||
0 1 0 0
|
21
tests/queries/0_stateless/02845_join_on_cond_sparse.sql
Normal file
21
tests/queries/0_stateless/02845_join_on_cond_sparse.sql
Normal file
@ -0,0 +1,21 @@
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
|
||||
CREATE TABLE t1 ( id UInt32, attr UInt32 ) ENGINE = MergeTree ORDER BY id
|
||||
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.1;
|
||||
|
||||
INSERT INTO t1 VALUES (0, 0);
|
||||
|
||||
CREATE TABLE t2 ( id UInt32, attr UInt32 ) ENGINE = MergeTree ORDER BY id
|
||||
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.1;
|
||||
|
||||
INSERT INTO t2 VALUES (0, 0);
|
||||
|
||||
SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND t1.attr != 0;
|
||||
|
||||
INSERT INTO t1 VALUES (0, 1);
|
||||
|
||||
SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND t1.attr != 0;
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
@ -0,0 +1,6 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/52019
|
||||
DROP TABLE IF EXISTS set_index__fuzz_41;
|
||||
CREATE TABLE set_index__fuzz_41 (`a` Date, `b` Nullable(DateTime64(3)), INDEX b_set b TYPE set(0) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO set_index__fuzz_41 (a) VALUES (today());
|
||||
SELECT b FROM set_index__fuzz_41 WHERE and(b = 256) SETTINGS force_data_skipping_indices = 'b_set', optimize_move_to_prewhere = 0, max_parallel_replicas=2, parallel_replicas_for_non_replicated_merge_tree=1, allow_experimental_parallel_reading_from_replicas=2, use_hedged_requests=0; -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
|
||||
DROP TABLE set_index__fuzz_41;
|
@ -0,0 +1,9 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/52019
|
||||
DROP TABLE IF EXISTS tab;
|
||||
SET allow_experimental_inverted_index=1;
|
||||
CREATE TABLE tab (`k` UInt64, `s` Map(String, String), INDEX af mapKeys(s) TYPE inverted(2) GRANULARITY 1) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO tab (k) VALUES (0);
|
||||
SELECT * FROM tab PREWHERE (s[NULL]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER }
|
||||
SELECT * FROM tab PREWHERE (s[1]) = 'Click a03' SETTINGS allow_experimental_analyzer=1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT * FROM tab PREWHERE (s['foo']) = 'Click a03' SETTINGS allow_experimental_analyzer=1;
|
||||
DROP TABLE tab;
|
Loading…
Reference in New Issue
Block a user