Merge branch 'master' of github.com:ClickHouse/ClickHouse into BLAKE3

This commit is contained in:
BoloniniD 2022-06-01 14:36:46 +03:00
commit def1dc0ae6
21 changed files with 165 additions and 29 deletions

View File

@ -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");

View File

@ -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;
}

View File

@ -135,7 +135,7 @@ struct SelectQueryOptions
return *this;
}
SelectQueryOptions & ignoreASTOptimizationsAlias(bool value = true)
SelectQueryOptions & ignoreASTOptimizations(bool value = true)
{
ignore_ast_optimizations = value;
return *this;

View File

@ -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);

View File

@ -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)

View File

@ -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();

View File

@ -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]

View File

@ -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})

View File

@ -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>

View File

@ -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)

View File

@ -1,5 +1,6 @@
\N
\N
[]
1396-01-12
1396-01-13
\N

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -0,0 +1 @@
[]

View 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;

View File

@ -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

View File

@ -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;

View File

@ -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);