mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 03:42:48 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into BLAKE3
This commit is contained in:
commit
def1dc0ae6
@ -839,6 +839,9 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
|
||||
if (node.name == "grouping")
|
||||
{
|
||||
if (data.only_consts)
|
||||
return; // Can not perform constant folding, because this function can be executed only after GROUP BY
|
||||
|
||||
size_t arguments_size = node.arguments->children.size();
|
||||
if (arguments_size == 0)
|
||||
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function GROUPING expects at least one argument");
|
||||
|
@ -538,6 +538,7 @@ void HashJoin::dataMapInit(MapsVariant & map)
|
||||
|
||||
bool HashJoin::overDictionary() const
|
||||
{
|
||||
assert(data->type != Type::DICT || table_join->getDictionaryReader());
|
||||
return data->type == Type::DICT;
|
||||
}
|
||||
|
||||
@ -917,6 +918,7 @@ public:
|
||||
bool is_join_get_)
|
||||
: join_on_keys(join_on_keys_)
|
||||
, rows_to_add(block.rows())
|
||||
, sample_block(saved_block_sample)
|
||||
, is_join_get(is_join_get_)
|
||||
{
|
||||
size_t num_columns_to_add = block_with_columns_to_add.columns();
|
||||
@ -956,12 +958,46 @@ public:
|
||||
return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name);
|
||||
}
|
||||
|
||||
static void assertBlockEqualsStructureUpToLowCard(const Block & lhs_block, const Block & rhs_block)
|
||||
{
|
||||
if (lhs_block.columns() != rhs_block.columns())
|
||||
throw Exception("Different number of columns in blocks", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (size_t i = 0; i < lhs_block.columns(); ++i)
|
||||
{
|
||||
const auto & lhs = lhs_block.getByPosition(i);
|
||||
const auto & rhs = rhs_block.getByPosition(i);
|
||||
if (lhs.name != rhs.name)
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]",
|
||||
lhs_block.dumpStructure(), rhs_block.dumpStructure());
|
||||
|
||||
const auto & ltype = recursiveRemoveLowCardinality(lhs.type);
|
||||
const auto & rtype = recursiveRemoveLowCardinality(rhs.type);
|
||||
if (!ltype->equals(*rtype))
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]",
|
||||
lhs_block.dumpStructure(), rhs_block.dumpStructure());
|
||||
|
||||
const auto & lcol = recursiveRemoveLowCardinality(lhs.column);
|
||||
const auto & rcol = recursiveRemoveLowCardinality(rhs.column);
|
||||
if (lcol->getDataType() != rcol->getDataType())
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]",
|
||||
lhs_block.dumpStructure(), rhs_block.dumpStructure());
|
||||
}
|
||||
}
|
||||
|
||||
template <bool has_defaults>
|
||||
void appendFromBlock(const Block & block, size_t row_num)
|
||||
{
|
||||
if constexpr (has_defaults)
|
||||
applyLazyDefaults();
|
||||
|
||||
#ifndef NDEBUG
|
||||
/// Like assertBlocksHaveEqualStructure but doesn't check low cardinality
|
||||
assertBlockEqualsStructureUpToLowCard(sample_block, block);
|
||||
#else
|
||||
UNUSED(assertBlockEqualsStructureUpToLowCard);
|
||||
#endif
|
||||
|
||||
if (is_join_get)
|
||||
{
|
||||
/// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin.
|
||||
@ -1024,6 +1060,7 @@ private:
|
||||
size_t lazy_defaults_count = 0;
|
||||
/// for ASOF
|
||||
const IColumn * left_asof_key = nullptr;
|
||||
Block sample_block;
|
||||
|
||||
bool is_join_get;
|
||||
|
||||
@ -1703,7 +1740,7 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types,
|
||||
throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
auto elem = sample_block_with_columns_to_add.getByName(column_name);
|
||||
if (or_null)
|
||||
if (or_null && JoinCommon::canBecomeNullable(elem.type))
|
||||
elem.type = makeNullable(elem.type);
|
||||
return elem.type;
|
||||
}
|
||||
|
@ -135,7 +135,7 @@ struct SelectQueryOptions
|
||||
return *this;
|
||||
}
|
||||
|
||||
SelectQueryOptions & ignoreASTOptimizationsAlias(bool value = true)
|
||||
SelectQueryOptions & ignoreASTOptimizations(bool value = true)
|
||||
{
|
||||
ignore_ast_optimizations = value;
|
||||
return *this;
|
||||
|
@ -479,6 +479,13 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context)
|
||||
src_names.push_back(original);
|
||||
dst_columns.push_back({col.name, col.type});
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Can't extract column from dictionary table
|
||||
/// TODO: Sometimes it should be possible to recunstruct required column,
|
||||
/// e.g. if it's an expression depending on dictionary attributes
|
||||
return false;
|
||||
}
|
||||
}
|
||||
dictionary_reader = std::make_shared<DictionaryReader>(dict_name, src_names, dst_columns, context);
|
||||
|
||||
|
@ -46,8 +46,15 @@ std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
checkStackSize();
|
||||
|
||||
auto query_plan = std::make_unique<QueryPlan>();
|
||||
/// Do not apply AST optimizations, because query
|
||||
/// is already optimized and some optimizations
|
||||
/// can be applied only for non-distributed tables
|
||||
/// and we can produce query, inconsistent with remote plans.
|
||||
auto interpreter = InterpreterSelectQuery(
|
||||
query_ast, context, SelectQueryOptions(processed_stage).setShardInfo(shard_num, shard_count));
|
||||
query_ast, context,
|
||||
SelectQueryOptions(processed_stage)
|
||||
.setShardInfo(shard_num, shard_count)
|
||||
.ignoreASTOptimizations());
|
||||
|
||||
interpreter.setProperClientInfo();
|
||||
if (coordinator)
|
||||
|
@ -109,7 +109,7 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const
|
||||
InterpreterSelectQuery select(
|
||||
result.query_ast, query_context, storage, {},
|
||||
/// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants.
|
||||
SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizationsAlias());
|
||||
SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations());
|
||||
|
||||
result.required_columns = select.getRequiredColumns();
|
||||
result.sample_block = select.getSampleBlock();
|
||||
@ -221,7 +221,7 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection(
|
||||
InterpreterSelectQuery select(
|
||||
result.query_ast, query_context, storage, {},
|
||||
/// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants.
|
||||
SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizationsAlias());
|
||||
SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations());
|
||||
result.required_columns = select.getRequiredColumns();
|
||||
result.sample_block = select.getSampleBlock();
|
||||
|
||||
|
@ -165,6 +165,11 @@ def create_test_html_report(
|
||||
num_fails = 0
|
||||
has_test_time = False
|
||||
has_test_logs = False
|
||||
|
||||
if with_raw_logs:
|
||||
# Display entires with logs at the top (they correspond to failed tests)
|
||||
test_result.sort(key=lambda result: len(result) <= 3)
|
||||
|
||||
for result in test_result:
|
||||
test_name = result[0]
|
||||
test_status = result[1]
|
||||
|
@ -7,7 +7,7 @@
|
||||
:main jepsen.clickhouse-keeper.main
|
||||
:plugins [[lein-cljfmt "0.7.0"]]
|
||||
:dependencies [[org.clojure/clojure "1.10.1"]
|
||||
[jepsen "0.2.3"]
|
||||
[jepsen "0.2.6"]
|
||||
[zookeeper-clj "0.9.4"]
|
||||
[org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]]
|
||||
:repl-options {:init-ns jepsen.clickhouse-keeper.main})
|
||||
|
@ -5,6 +5,7 @@
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-keeper/clickhouse-keeper.log</log>
|
||||
<errorlog>/var/log/clickhouse-keeper/clickhouse-keeper.err.log</errorlog>
|
||||
<size>never</size>
|
||||
</logger>
|
||||
|
||||
<keeper_server>
|
||||
|
@ -10,10 +10,27 @@
|
||||
[jepsen.control.util :as cu]
|
||||
[jepsen.os.ubuntu :as ubuntu]))
|
||||
|
||||
(defn get-clickhouse-sky
|
||||
[version]
|
||||
(c/exec :sky :get :-d common-prefix :-N :Backbone version)
|
||||
(str common-prefix "/clickhouse"))
|
||||
|
||||
(ns jepsen.control.scp)
|
||||
|
||||
;; We need to overwrite Jepsen's implementation of scp! because it
|
||||
;; doesn't use strict-host-key-checking
|
||||
|
||||
(defn scp!
|
||||
"Runs an SCP command by shelling out. Takes a conn-spec (used for port, key,
|
||||
etc), a seq of sources, and a single destination, all as strings."
|
||||
[conn-spec sources dest]
|
||||
(apply util/sh "scp" "-rpC"
|
||||
"-P" (str (:port conn-spec))
|
||||
(concat (when-let [k (:private-key-path conn-spec)]
|
||||
["-i" k])
|
||||
(if-not (:strict-host-key-checking conn-spec)
|
||||
["-o StrictHostKeyChecking=no"])
|
||||
sources
|
||||
[dest]))
|
||||
nil)
|
||||
|
||||
(ns jepsen.clickhouse-keeper.db)
|
||||
|
||||
(defn get-clickhouse-url
|
||||
[url]
|
||||
@ -27,7 +44,6 @@
|
||||
[source]
|
||||
(info "Downloading clickhouse from" source)
|
||||
(cond
|
||||
(clojure.string/starts-with? source "rbtorrent:") (get-clickhouse-sky source)
|
||||
(clojure.string/starts-with? source "http") (get-clickhouse-url source)
|
||||
(.exists (io/file source)) (get-clickhouse-scp source)
|
||||
:else (throw (Exception. (str "Don't know how to download clickhouse from" source)))))
|
||||
@ -141,8 +157,13 @@
|
||||
(do
|
||||
(info node "Coordination files exists, going to compress")
|
||||
(c/cd data-dir
|
||||
(c/exec :tar :czf "coordination.tar.gz" "coordination")))))
|
||||
(let [common-logs [stderr-file (str logs-dir "/clickhouse-keeper.log") (str data-dir "/coordination.tar.gz")]
|
||||
(c/exec :tar :czf "coordination.tar.gz" "coordination"))))
|
||||
(if (cu/exists? (str logs-dir))
|
||||
(do
|
||||
(info node "Logs exist, going to compress")
|
||||
(c/cd common-prefix
|
||||
(c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing")))
|
||||
(let [common-logs [(str common-prefix "/logs.tar.gz") (str data-dir "/coordination.tar.gz")]
|
||||
gdb-log (str logs-dir "/gdb.log")]
|
||||
(if (cu/exists? (str logs-dir "/gdb.log"))
|
||||
(conj common-logs gdb-log)
|
||||
|
@ -1,5 +1,6 @@
|
||||
\N
|
||||
\N
|
||||
[]
|
||||
1396-01-12
|
||||
1396-01-13
|
||||
\N
|
||||
|
@ -9,7 +9,7 @@ SELECT joinGetOrNull('join_test', 'num', 500);
|
||||
DROP TABLE join_test;
|
||||
|
||||
CREATE TABLE join_test (id UInt16, num Array(UInt16)) engine = Join(ANY, LEFT, id);
|
||||
SELECT joinGetOrNull('join_test', 'num', 500); -- { serverError 43 }
|
||||
SELECT joinGetOrNull('join_test', 'num', 500);
|
||||
DROP TABLE join_test;
|
||||
|
||||
drop table if exists test;
|
||||
|
@ -1 +1,2 @@
|
||||
0
|
||||
0
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel
|
||||
-- Tags: no-parallel, no-backward-compatibility-check
|
||||
|
||||
DROP DATABASE IF EXISTS db_01391;
|
||||
CREATE DATABASE db_01391;
|
||||
@ -20,8 +20,8 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' DB 'db_01391' t
|
||||
LIFETIME(MIN 1 MAX 1)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
select click_country_id from t cc
|
||||
left join d on toUInt32(d.id) = cc.click_city_id;
|
||||
SELECT click_country_id FROM t AS cc LEFT JOIN d ON toUInt32(d.id) = cc.click_city_id;
|
||||
SELECT click_country_id FROM t AS cc LEFT JOIN d ON d.country_id < 99 AND d.id = cc.click_city_id;
|
||||
|
||||
DROP DICTIONARY d;
|
||||
DROP TABLE t;
|
||||
|
@ -8,20 +8,16 @@ CREATE TABLE lc_table
|
||||
|
||||
INSERT INTO lc_table VALUES('x');
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON lc_table.col = lc_table2.col;
|
||||
SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2
|
||||
ON lc_table.col = lc_table2.col;
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON CAST(lc_table.col AS String) = CAST(lc_table2.col AS String);
|
||||
SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2
|
||||
ON CAST(lc_table.col AS String) = CAST(lc_table2.col AS String);
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON (lc_table.col = lc_table2.col) OR (lc_table.col = lc_table2.col);
|
||||
SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2
|
||||
ON (lc_table.col = lc_table2.col) OR (lc_table.col = lc_table2.col);
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)) OR (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String));
|
||||
SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2
|
||||
ON (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)) OR (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String));
|
||||
|
||||
DROP TABLE IF EXISTS lc_table;
|
||||
|
@ -0,0 +1 @@
|
||||
[]
|
6
tests/queries/0_stateless/02307_join_get_array_null.sql
Normal file
6
tests/queries/0_stateless/02307_join_get_array_null.sql
Normal file
@ -0,0 +1,6 @@
|
||||
drop table if exists id_val;
|
||||
|
||||
create table id_val(id Int32, val Array(Int32)) engine Join(ANY, LEFT, id) settings join_use_nulls = 1;
|
||||
select joinGet(id_val, 'val', toInt32(number)) from numbers(1);
|
||||
|
||||
drop table id_val;
|
@ -0,0 +1,29 @@
|
||||
-- { echoOn }
|
||||
SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b);
|
||||
1 0 0 3
|
||||
1 0 2 3
|
||||
1 0 4 3
|
||||
1 0 6 3
|
||||
1 0 8 3
|
||||
1 1 1 3
|
||||
1 1 3 3
|
||||
1 1 5 3
|
||||
1 1 7 3
|
||||
1 1 9 3
|
||||
5 0 0 2
|
||||
5 1 0 2
|
||||
10 0 0 0
|
||||
SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b);
|
||||
1 0 0 3
|
||||
1 0 2 3
|
||||
1 0 4 3
|
||||
1 0 6 3
|
||||
1 0 8 3
|
||||
1 1 1 3
|
||||
1 1 3 3
|
||||
1 1 5 3
|
||||
1 1 7 3
|
||||
1 1 9 3
|
||||
5 0 0 2
|
||||
5 1 0 2
|
||||
10 0 0 0
|
@ -0,0 +1,13 @@
|
||||
DROP TABLE IF EXISTS test02315;
|
||||
|
||||
CREATE TABLE test02315(a UInt64, b UInt64) ENGINE=MergeTree() ORDER BY (a, b);
|
||||
|
||||
INSERT INTO test02315 SELECT number % 2 as a, number as b FROM numbers(10);
|
||||
|
||||
-- { echoOn }
|
||||
SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b);
|
||||
|
||||
SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b);
|
||||
|
||||
-- { echoOff }
|
||||
DROP TABLE test02315;
|
@ -0,0 +1,2 @@
|
||||
0
|
||||
0
|
@ -0,0 +1,6 @@
|
||||
SET prefer_localhost_replica = 1;
|
||||
SET optimize_monotonous_functions_in_order_by = 1;
|
||||
|
||||
SELECT *
|
||||
FROM cluster(test_cluster_two_shards_localhost, system, one)
|
||||
ORDER BY toDateTime(dummy);
|
Loading…
Reference in New Issue
Block a user