Date: Thu, 1 Jul 2021 16:59:16 +0100
Subject: [PATCH 27/56] CLICKHOUSE-1194: add skipping index to the beginning of
the list
add the FIRST keyword to the ADD INDEX command to be able to add index in the beginning of the list.
Signed-off-by: Aleksei Semiglazov
---
.../statements/alter/index/index.md | 2 +-
docs/ja/sql-reference/statements/alter.md | 2 +-
.../statements/alter/index/index.md | 2 +-
docs/zh/sql-reference/statements/alter.md | 2 +-
src/Parsers/ASTAlterQuery.cpp | 5 ++--
src/Parsers/ParserAlterQuery.cpp | 4 ++-
src/Storages/AlterCommands.cpp | 5 ++++
src/Storages/AlterCommands.h | 2 +-
.../01932_alter_index_with_order.reference | 9 ++++++
.../01932_alter_index_with_order.sql | 28 +++++++++++++++++++
10 files changed, 53 insertions(+), 8 deletions(-)
create mode 100644 tests/queries/0_stateless/01932_alter_index_with_order.reference
create mode 100644 tests/queries/0_stateless/01932_alter_index_with_order.sql
diff --git a/docs/en/sql-reference/statements/alter/index/index.md b/docs/en/sql-reference/statements/alter/index/index.md
index 56d81aaf52f..fd5657c3666 100644
--- a/docs/en/sql-reference/statements/alter/index/index.md
+++ b/docs/en/sql-reference/statements/alter/index/index.md
@@ -8,7 +8,7 @@ toc_title: INDEX
The following operations are available:
-- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - Adds index description to tables metadata.
+- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - Adds index description to tables metadata.
- `ALTER TABLE [db].name DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk.
diff --git a/docs/ja/sql-reference/statements/alter.md b/docs/ja/sql-reference/statements/alter.md
index 226565dd226..0967f60e06a 100644
--- a/docs/ja/sql-reference/statements/alter.md
+++ b/docs/ja/sql-reference/statements/alter.md
@@ -175,7 +175,7 @@ MODIFY ORDER BY new_expression
[複製](../../engines/table-engines/mergetree-family/replication.md) テーブル)。 次の操作
利用できます:
-- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` -付加価指数の説明をテーブルメタデータを指すものとします。
+- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` -付加価指数の説明をテーブルメタデータを指すものとします。
- `ALTER TABLE [db].name DROP INDEX name` -除去す指標の説明からテーブルメタデータを削除を行指数のファイルからディスク。
diff --git a/docs/ru/sql-reference/statements/alter/index/index.md b/docs/ru/sql-reference/statements/alter/index/index.md
index 632f11ed906..1f6bbea5c4b 100644
--- a/docs/ru/sql-reference/statements/alter/index/index.md
+++ b/docs/ru/sql-reference/statements/alter/index/index.md
@@ -9,7 +9,7 @@ toc_title: "Манипуляции с индексами"
Добавить или удалить индекс можно с помощью операций
``` sql
-ALTER TABLE [db.]name ADD INDEX name expression TYPE type GRANULARITY value [AFTER name]
+ALTER TABLE [db.]name ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]
ALTER TABLE [db.]name DROP INDEX name
ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name
```
diff --git a/docs/zh/sql-reference/statements/alter.md b/docs/zh/sql-reference/statements/alter.md
index 446feac96ce..4d1cdca71e5 100644
--- a/docs/zh/sql-reference/statements/alter.md
+++ b/docs/zh/sql-reference/statements/alter.md
@@ -174,7 +174,7 @@ MODIFY ORDER BY new_expression
该操作仅支持 [`MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) 系列表 (含 [replicated](../../engines/table-engines/mergetree-family/replication.md) 表)。
下列操作是允许的:
-- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - 在表的元数据中增加索引说明
+- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - 在表的元数据中增加索引说明
- `ALTER TABLE [db].name DROP INDEX name` - 从表的元数据中删除索引描述,并从磁盘上删除索引文件
diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp
index 918abc39037..7e60d1175e2 100644
--- a/src/Parsers/ASTAlterQuery.cpp
+++ b/src/Parsers/ASTAlterQuery.cpp
@@ -137,8 +137,9 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
index_decl->formatImpl(settings, state, frame);
- /// AFTER
- if (index)
+ if (first)
+ settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : "");
+ else if (index) /// AFTER
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : "");
index->formatImpl(settings, state, frame);
diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp
index d659db64b83..2908b171ca6 100644
--- a/src/Parsers/ParserAlterQuery.cpp
+++ b/src/Parsers/ParserAlterQuery.cpp
@@ -231,7 +231,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (!parser_idx_decl.parse(pos, command->index_decl, expected))
return false;
- if (s_after.ignore(pos, expected))
+ if (s_first.ignore(pos, expected))
+ command->first = true;
+ else if (s_after.ignore(pos, expected))
{
if (!parser_name.parse(pos, command->index, expected))
return false;
diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp
index 1cb936cbb84..3ac457e52d6 100644
--- a/src/Storages/AlterCommands.cpp
+++ b/src/Storages/AlterCommands.cpp
@@ -211,6 +211,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_
command.after_index_name = command_ast->index->as().name();
command.if_not_exists = command_ast->if_not_exists;
+ command.first = command_ast->first;
return command;
}
@@ -454,6 +455,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
auto insert_it = metadata.secondary_indices.end();
+ /// insert the index in the beginning of the indices list
+ if (first)
+ insert_it = metadata.secondary_indices.begin();
+
if (!after_index_name.empty())
{
insert_it = std::find_if(
diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h
index 4e9c9764753..d523bb2783e 100644
--- a/src/Storages/AlterCommands.h
+++ b/src/Storages/AlterCommands.h
@@ -77,7 +77,7 @@ struct AlterCommand
/// For ADD or MODIFY - after which column to add a new one. If an empty string, add to the end.
String after_column;
- /// For ADD_COLUMN, MODIFY_COLUMN - Add to the begin if it is true.
+ /// For ADD_COLUMN, MODIFY_COLUMN, ADD_INDEX - Add to the begin if it is true.
bool first = false;
/// For DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
diff --git a/tests/queries/0_stateless/01932_alter_index_with_order.reference b/tests/queries/0_stateless/01932_alter_index_with_order.reference
new file mode 100644
index 00000000000..07e1aab3df9
--- /dev/null
+++ b/tests/queries/0_stateless/01932_alter_index_with_order.reference
@@ -0,0 +1,9 @@
+default alter_index_test index_a set a 1
+default alter_index_test index_b minmax b 1
+default alter_index_test index_c set c 2
+default alter_index_test index_a set a 1
+default alter_index_test index_d set d 1
+default alter_index_test index_b minmax b 1
+default alter_index_test index_c set c 2
+default alter_index_test index_a set a 1
+default alter_index_test index_d set d 1
diff --git a/tests/queries/0_stateless/01932_alter_index_with_order.sql b/tests/queries/0_stateless/01932_alter_index_with_order.sql
new file mode 100644
index 00000000000..0f2953b53f9
--- /dev/null
+++ b/tests/queries/0_stateless/01932_alter_index_with_order.sql
@@ -0,0 +1,28 @@
+DROP TABLE IF EXISTS alter_index_test;
+
+CREATE TABLE alter_index_test (
+ a UInt32,
+ b Date,
+ c UInt32,
+ d UInt32,
+ INDEX index_a a TYPE set(0) GRANULARITY 1
+)
+ENGINE = MergeTree()
+ORDER BY tuple();
+
+SELECT * FROM system.data_skipping_indices WHERE table = 'alter_index_test' AND database = currentDatabase();
+
+ALTER TABLE alter_index_test ADD INDEX index_b b type minmax granularity 1 FIRST;
+
+ALTER TABLE alter_index_test ADD INDEX index_c c type set(0) granularity 2 AFTER index_b;
+
+ALTER TABLE alter_index_test ADD INDEX index_d d type set(0) granularity 1;
+
+SELECT * FROM system.data_skipping_indices WHERE table = 'alter_index_test' AND database = currentDatabase();
+
+DETACH TABLE alter_index_test;
+ATTACH TABLE alter_index_test;
+
+SELECT * FROM system.data_skipping_indices WHERE table = 'alter_index_test' AND database = currentDatabase();
+
+DROP TABLE IF EXISTS alter_index_test;
From 012f67e6f865f81ec15c08fde846885e6eb63885 Mon Sep 17 00:00:00 2001
From: kssenii
Date: Thu, 1 Jul 2021 23:29:00 +0300
Subject: [PATCH 28/56] Update libpq
---
contrib/libpq | 2 +-
contrib/libpq-cmake/CMakeLists.txt | 8 ++++++--
2 files changed, 7 insertions(+), 3 deletions(-)
diff --git a/contrib/libpq b/contrib/libpq
index c7624588ddd..69e8a80e98f 160000
--- a/contrib/libpq
+++ b/contrib/libpq
@@ -1 +1 @@
-Subproject commit c7624588ddd84f153dd5990e81b886e4568bddde
+Subproject commit 69e8a80e98f27e3a5deec617334e31db2b9ed7d7
diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt
index 028fabe52b8..4f6a1554d10 100644
--- a/contrib/libpq-cmake/CMakeLists.txt
+++ b/contrib/libpq-cmake/CMakeLists.txt
@@ -8,7 +8,7 @@ set(SRCS
"${LIBPQ_SOURCE_DIR}/fe-lobj.c"
"${LIBPQ_SOURCE_DIR}/fe-misc.c"
"${LIBPQ_SOURCE_DIR}/fe-print.c"
- "${LIBPQ_SOURCE_DIR}/fe-protocol2.c"
+ "${LIBPQ_SOURCE_DIR}/fe-trace.c"
"${LIBPQ_SOURCE_DIR}/fe-protocol3.c"
"${LIBPQ_SOURCE_DIR}/fe-secure.c"
"${LIBPQ_SOURCE_DIR}/fe-secure-common.c"
@@ -18,8 +18,12 @@ set(SRCS
"${LIBPQ_SOURCE_DIR}/pqexpbuffer.c"
"${LIBPQ_SOURCE_DIR}/common/scram-common.c"
- "${LIBPQ_SOURCE_DIR}/common/sha2_openssl.c"
+ "${LIBPQ_SOURCE_DIR}/common/sha2.c"
+ "${LIBPQ_SOURCE_DIR}/common/sha1.c"
"${LIBPQ_SOURCE_DIR}/common/md5.c"
+ "${LIBPQ_SOURCE_DIR}/common/md5_common.c"
+ "${LIBPQ_SOURCE_DIR}/common/hmac_openssl.c"
+ "${LIBPQ_SOURCE_DIR}/common/cryptohash.c"
"${LIBPQ_SOURCE_DIR}/common/saslprep.c"
"${LIBPQ_SOURCE_DIR}/common/unicode_norm.c"
"${LIBPQ_SOURCE_DIR}/common/ip.c"
From 531e48afa34409fdb942b22aaa939816f17ef346 Mon Sep 17 00:00:00 2001
From: alexey-milovidov
Date: Fri, 2 Jul 2021 00:39:41 +0300
Subject: [PATCH 29/56] Update 01923_network_receive_time_metric_insert.sh
---
.../0_stateless/01923_network_receive_time_metric_insert.sh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh
index 2a9807af10d..bcb3775f86a 100755
--- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh
+++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh
@@ -11,7 +11,7 @@ seq 1 1000 | pv --quiet --rate-limit 1000 | ${CLICKHOUSE_CLIENT} --query "INSERT
# We check that the value of NetworkReceiveElapsedMicroseconds correctly includes the time spent waiting data from the client.
${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS;
- WITH ProfileEvents.Values[indexOf(ProfileEvents.Names, 'NetworkReceiveElapsedMicroseconds')] AS time
+ WITH ProfileEvents['NetworkReceiveElapsedMicroseconds'] AS time
SELECT time >= 1000000 ? 1 : time FROM system.query_log
WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;"
From 2866d45681e6dd3b9429d5fd4347b1fb2216da98 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 02:01:13 +0300
Subject: [PATCH 30/56] Add support for queries with `null` quoted identifier
and ON CLUSTER
---
src/Common/StringUtils/StringUtils.h | 6 +++++-
.../0_stateless/01932_null_valid_identifier.reference | 3 +++
tests/queries/0_stateless/01932_null_valid_identifier.sql | 3 +++
3 files changed, 11 insertions(+), 1 deletion(-)
create mode 100644 tests/queries/0_stateless/01932_null_valid_identifier.reference
create mode 100644 tests/queries/0_stateless/01932_null_valid_identifier.sql
diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h
index 20c0a5ca380..f6ad61f8fd9 100644
--- a/src/Common/StringUtils/StringUtils.h
+++ b/src/Common/StringUtils/StringUtils.h
@@ -149,7 +149,11 @@ inline bool isPunctuationASCII(char c)
inline bool isValidIdentifier(const std::string_view & str)
{
- return !str.empty() && isValidIdentifierBegin(str[0]) && std::all_of(str.begin() + 1, str.end(), isWordCharASCII);
+ return !str.empty()
+ && isValidIdentifierBegin(str[0])
+ && std::all_of(str.begin() + 1, str.end(), isWordCharASCII)
+ /// NULL is not a valid identifier in SQL, any case.
+ && !(str.size() == strlen("null") && 0 == strncasecmp(str.data(), "null", strlen("null")));
}
/// Works assuming isAlphaASCII.
diff --git a/tests/queries/0_stateless/01932_null_valid_identifier.reference b/tests/queries/0_stateless/01932_null_valid_identifier.reference
new file mode 100644
index 00000000000..8600160f48c
--- /dev/null
+++ b/tests/queries/0_stateless/01932_null_valid_identifier.reference
@@ -0,0 +1,3 @@
+1
+1
+1 \N
diff --git a/tests/queries/0_stateless/01932_null_valid_identifier.sql b/tests/queries/0_stateless/01932_null_valid_identifier.sql
new file mode 100644
index 00000000000..31f1a771675
--- /dev/null
+++ b/tests/queries/0_stateless/01932_null_valid_identifier.sql
@@ -0,0 +1,3 @@
+SELECT `null` FROM remote('127.0.0.2', view(SELECT 1 AS `null`));
+SELECT `NULL` FROM remote('127.0.0.2', view(SELECT 1 AS `NULL`));
+SELECT `nULl`, null FROM remote('127.0.0.2', view(SELECT 1 AS `nULl`));
From e6f0997924cfd3f80cb842550d160eec793f4d57 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 02:01:52 +0300
Subject: [PATCH 31/56] Remove Arcadia
---
tests/queries/0_stateless/arcadia_skip_list.txt | 1 +
1 file changed, 1 insertion(+)
diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt
index afd11cb5a7d..8453094cc65 100644
--- a/tests/queries/0_stateless/arcadia_skip_list.txt
+++ b/tests/queries/0_stateless/arcadia_skip_list.txt
@@ -251,3 +251,4 @@
01924_argmax_bitmap_state
01914_exchange_dictionaries
01923_different_expression_name_alias
+01932_null_valid_identifier
From 63c71a7b4b330dc18ca57ded95364be6e5c758de Mon Sep 17 00:00:00 2001
From: Ilya Golshtein
Date: Fri, 2 Jul 2021 02:12:31 +0300
Subject: [PATCH 32/56] kerberized HDFS test fix if run in parallel
---
tests/integration/helpers/cluster.py | 9 ++++++---
.../hdfs_configs/bootstrap.sh | 1 +
2 files changed, 7 insertions(+), 3 deletions(-)
diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py
index bd2f7d2bd8a..0799c8ed0e8 100644
--- a/tests/integration/helpers/cluster.py
+++ b/tests/integration/helpers/cluster.py
@@ -1188,15 +1188,18 @@ class ClickHouseCluster:
time.sleep(1)
- def wait_hdfs_to_start(self, timeout=300):
+ def wait_hdfs_to_start(self, timeout=300, check_marker=False):
start = time.time()
while time.time() - start < timeout:
try:
self.hdfs_api.write_data("/somefilewithrandomname222", "1")
logging.debug("Connected to HDFS and SafeMode disabled! ")
+ if check_marker:
+ self.hdfs_api.read_data("/preparations_done_marker")
+
return
except Exception as ex:
- logging.exception("Can't connect to HDFS " + str(ex))
+ logging.exception("Can't connect to HDFS or preparations are not done yet " + str(ex))
time.sleep(1)
raise Exception("Can't wait HDFS to start")
@@ -1443,7 +1446,7 @@ class ClickHouseCluster:
os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO)
run_and_check(self.base_kerberized_hdfs_cmd + common_opts)
self.make_hdfs_api(kerberized=True)
- self.wait_hdfs_to_start()
+ self.wait_hdfs_to_start(check_marker=True)
if self.with_mongo and self.base_mongo_cmd:
logging.debug('Setup Mongo')
diff --git a/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh b/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh
index 971491d4053..769056d70b3 100755
--- a/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh
+++ b/tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh
@@ -242,6 +242,7 @@ sleep 3
/usr/local/hadoop/bin/hdfs dfs -mkdir /user/specuser
/usr/local/hadoop/bin/hdfs dfs -chown specuser /user/specuser
+echo "chown_completed" | /usr/local/hadoop/bin/hdfs dfs -appendToFile - /preparations_done_marker
kdestroy
From 7a993404b4d222884bedf4a933f999213be48b5c Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 02:30:18 +0300
Subject: [PATCH 33/56] Whitespace
---
programs/benchmark/Benchmark.cpp | 1 +
1 file changed, 1 insertion(+)
diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp
index c8f1a4eef47..859222c236e 100644
--- a/programs/benchmark/Benchmark.cpp
+++ b/programs/benchmark/Benchmark.cpp
@@ -37,6 +37,7 @@
#include
#include
+
namespace fs = std::filesystem;
/** A tool for evaluating ClickHouse performance.
From b33d91412db32a9352c481f2e7d6b10f79bdb6c9 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 03:07:23 +0300
Subject: [PATCH 34/56] Correctly throw exception on invalid dates
---
src/IO/ReadHelpers.cpp | 2 +-
src/IO/ReadHelpers.h | 40 +++++++++++++------
.../0_stateless/01933_invalid_date.reference | 1 +
.../0_stateless/01933_invalid_date.sql | 10 +++++
4 files changed, 40 insertions(+), 13 deletions(-)
create mode 100644 tests/queries/0_stateless/01933_invalid_date.reference
create mode 100644 tests/queries/0_stateless/01933_invalid_date.sql
diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp
index 8e9a14a20fb..2a5594a6866 100644
--- a/src/IO/ReadHelpers.cpp
+++ b/src/IO/ReadHelpers.cpp
@@ -765,7 +765,7 @@ ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf)
auto ignore_delimiter = [&]
{
- if (!buf.eof())
+ if (!buf.eof() && !isNumericASCII(*buf.position()))
{
++buf.position();
return true;
diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h
index d4e2db0b553..4e101aaaf63 100644
--- a/src/IO/ReadHelpers.h
+++ b/src/IO/ReadHelpers.h
@@ -572,27 +572,43 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf)
/// Optimistic path, when whole value is in buffer.
if (!buf.eof() && buf.position() + 10 <= buf.buffer().end())
{
- UInt16 year = (buf.position()[0] - '0') * 1000 + (buf.position()[1] - '0') * 100 + (buf.position()[2] - '0') * 10 + (buf.position()[3] - '0');
- buf.position() += 5;
+ char * pos = buf.position();
- UInt8 month = buf.position()[0] - '0';
- if (isNumericASCII(buf.position()[1]))
+ /// YYYY-MM-DD
+ /// YYYY-MM-D
+ /// YYYY-M-DD
+ /// YYYY-M-D
+
+ /// The delimiters can be arbitrary characters, like YYYY/MM!DD, but obviously not digits.
+
+ UInt16 year = (pos[0] - '0') * 1000 + (pos[1] - '0') * 100 + (pos[2] - '0') * 10 + (pos[3] - '0');
+ pos += 5;
+
+ if (isNumericASCII(pos[-1]))
+ return ReturnType(false);
+
+ UInt8 month = pos[0] - '0';
+ if (isNumericASCII(pos[1]))
{
- month = month * 10 + buf.position()[1] - '0';
- buf.position() += 3;
+ month = month * 10 + pos[1] - '0';
+ pos += 3;
}
else
- buf.position() += 2;
+ pos += 2;
- UInt8 day = buf.position()[0] - '0';
- if (isNumericASCII(buf.position()[1]))
+ if (isNumericASCII(pos[-1]))
+ return ReturnType(false);
+
+ UInt8 day = pos[0] - '0';
+ if (isNumericASCII(pos[1]))
{
- day = day * 10 + buf.position()[1] - '0';
- buf.position() += 2;
+ day = day * 10 + pos[1] - '0';
+ pos += 2;
}
else
- buf.position() += 1;
+ pos += 1;
+ buf.position() = pos;
date = LocalDate(year, month, day);
return ReturnType(true);
}
diff --git a/tests/queries/0_stateless/01933_invalid_date.reference b/tests/queries/0_stateless/01933_invalid_date.reference
new file mode 100644
index 00000000000..829e7e8c420
--- /dev/null
+++ b/tests/queries/0_stateless/01933_invalid_date.reference
@@ -0,0 +1 @@
+2019-07-08
diff --git a/tests/queries/0_stateless/01933_invalid_date.sql b/tests/queries/0_stateless/01933_invalid_date.sql
new file mode 100644
index 00000000000..aac09c99e60
--- /dev/null
+++ b/tests/queries/0_stateless/01933_invalid_date.sql
@@ -0,0 +1,10 @@
+SELECT toDate('07-08-2019'); -- { serverError 6 }
+SELECT toDate('2019-0708'); -- { serverError 38 }
+SELECT toDate('201907-08'); -- { serverError 38 }
+SELECT toDate('2019^7^8');
+
+CREATE TEMPORARY TABLE test (d Date);
+INSERT INTO test VALUES ('2018-01-01');
+
+SELECT * FROM test WHERE d >= '07-08-2019'; -- { serverError 53 }
+SELECT * FROM test WHERE d >= '2019-07-08';
From 0e621788c7f8821b6a2b3fffb1885f15ba3e5bcb Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 03:53:08 +0300
Subject: [PATCH 35/56] Allow constexpr parameters for aggregate functions
---
.../parseAggregateFunctionParameters.cpp | 41 ++++++++++---------
.../parseAggregateFunctionParameters.h | 12 ++++--
src/Functions/array/arrayReduce.cpp | 7 ++--
src/Functions/array/arrayReduceInRanges.cpp | 7 ++--
src/Functions/initializeAggregation.cpp | 7 ++--
src/Interpreters/ExpressionAnalyzer.cpp | 4 +-
.../evaluateConstantExpression.cpp | 9 ++--
.../MergeTree/registerStorageMergeTree.cpp | 15 ++++---
...pr_aggregate_function_parameters.reference | 2 +
...onstexpr_aggregate_function_parameters.sql | 11 +++++
10 files changed, 72 insertions(+), 43 deletions(-)
create mode 100644 tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.reference
create mode 100644 tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.sql
diff --git a/src/AggregateFunctions/parseAggregateFunctionParameters.cpp b/src/AggregateFunctions/parseAggregateFunctionParameters.cpp
index 3826d993c4a..64eb0932de9 100644
--- a/src/AggregateFunctions/parseAggregateFunctionParameters.cpp
+++ b/src/AggregateFunctions/parseAggregateFunctionParameters.cpp
@@ -4,6 +4,8 @@
#include
#include
+#include
+
namespace DB
{
@@ -15,7 +17,7 @@ namespace ErrorCodes
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS;
}
-Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const std::string & error_context)
+Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const std::string & error_context, ContextPtr context)
{
const ASTs & parameters = expression_list->children;
if (parameters.empty())
@@ -25,25 +27,25 @@ Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const
for (size_t i = 0; i < parameters.size(); ++i)
{
- const auto * literal = parameters[i]->as();
-
- ASTPtr func_literal;
- if (!literal)
- if (const auto * func = parameters[i]->as())
- if ((func_literal = func->toLiteral()))
- literal = func_literal->as();
-
- if (!literal)
+ ASTPtr literal;
+ try
{
- throw Exception(
- ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS,
- "Parameters to aggregate functions must be literals. "
- "Got parameter '{}'{}",
- parameters[i]->formatForErrorMessage(),
- (error_context.empty() ? "" : " (in " + error_context +")"));
+ literal = evaluateConstantExpressionAsLiteral(parameters[i], context);
+ }
+ catch (Exception & e)
+ {
+ if (e.code() == ErrorCodes::BAD_ARGUMENTS)
+ throw Exception(
+ ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS,
+ "Parameters to aggregate functions must be literals. "
+ "Got parameter '{}'{}",
+ parameters[i]->formatForErrorMessage(),
+ (error_context.empty() ? "" : " (in " + error_context +")"));
+
+ throw;
}
- params_row[i] = literal->value;
+ params_row[i] = literal->as()->value;
}
return params_row;
@@ -54,7 +56,8 @@ void getAggregateFunctionNameAndParametersArray(
const std::string & aggregate_function_name_with_params,
std::string & aggregate_function_name,
Array & aggregate_function_parameters,
- const std::string & error_context)
+ const std::string & error_context,
+ ContextPtr context)
{
if (aggregate_function_name_with_params.back() != ')')
{
@@ -84,7 +87,7 @@ void getAggregateFunctionNameAndParametersArray(
throw Exception("Incorrect list of parameters to aggregate function "
+ aggregate_function_name, ErrorCodes::BAD_ARGUMENTS);
- aggregate_function_parameters = getAggregateFunctionParametersArray(args_ast);
+ aggregate_function_parameters = getAggregateFunctionParametersArray(args_ast, error_context, context);
}
}
diff --git a/src/AggregateFunctions/parseAggregateFunctionParameters.h b/src/AggregateFunctions/parseAggregateFunctionParameters.h
index 37f1f1d5097..033e92714dd 100644
--- a/src/AggregateFunctions/parseAggregateFunctionParameters.h
+++ b/src/AggregateFunctions/parseAggregateFunctionParameters.h
@@ -1,19 +1,23 @@
#pragma once
+
#include
#include
+#include
namespace DB
{
-struct Settings;
-
-Array getAggregateFunctionParametersArray(const ASTPtr & expression_list, const std::string & error_context = "");
+Array getAggregateFunctionParametersArray(
+ const ASTPtr & expression_list,
+ const std::string & error_context,
+ ContextPtr context);
void getAggregateFunctionNameAndParametersArray(
const std::string & aggregate_function_name_with_params,
std::string & aggregate_function_name,
Array & aggregate_function_parameters,
- const std::string & error_context);
+ const std::string & error_context,
+ ContextPtr context);
}
diff --git a/src/Functions/array/arrayReduce.cpp b/src/Functions/array/arrayReduce.cpp
index 9a4b5aafdb9..3387d50a1f6 100644
--- a/src/Functions/array/arrayReduce.cpp
+++ b/src/Functions/array/arrayReduce.cpp
@@ -33,11 +33,12 @@ namespace ErrorCodes
* arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...`
* If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function.
*/
-class FunctionArrayReduce : public IFunction
+class FunctionArrayReduce : public IFunction, private WithContext
{
public:
static constexpr auto name = "arrayReduce";
- static FunctionPtr create(ContextPtr) { return std::make_shared(); }
+ static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); }
+ FunctionArrayReduce(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; }
@@ -95,7 +96,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params,
- aggregate_function_name, params_row, "function " + getName());
+ aggregate_function_name, params_row, "function " + getName(), getContext());
AggregateFunctionProperties properties;
aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row, properties);
diff --git a/src/Functions/array/arrayReduceInRanges.cpp b/src/Functions/array/arrayReduceInRanges.cpp
index 9a2e8e1ca95..ffb047f2231 100644
--- a/src/Functions/array/arrayReduceInRanges.cpp
+++ b/src/Functions/array/arrayReduceInRanges.cpp
@@ -35,12 +35,13 @@ namespace ErrorCodes
*
* arrayReduceInRanges('agg', indices, lengths, arr1, ...)
*/
-class FunctionArrayReduceInRanges : public IFunction
+class FunctionArrayReduceInRanges : public IFunction, private WithContext
{
public:
static const size_t minimum_step = 64;
static constexpr auto name = "arrayReduceInRanges";
- static FunctionPtr create(ContextPtr) { return std::make_shared(); }
+ static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); }
+ FunctionArrayReduceInRanges(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; }
@@ -113,7 +114,7 @@ DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithType
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params,
- aggregate_function_name, params_row, "function " + getName());
+ aggregate_function_name, params_row, "function " + getName(), getContext());
AggregateFunctionProperties properties;
aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row, properties);
diff --git a/src/Functions/initializeAggregation.cpp b/src/Functions/initializeAggregation.cpp
index b9b3d219551..060788773b6 100644
--- a/src/Functions/initializeAggregation.cpp
+++ b/src/Functions/initializeAggregation.cpp
@@ -25,11 +25,12 @@ namespace ErrorCodes
namespace
{
-class FunctionInitializeAggregation : public IFunction
+class FunctionInitializeAggregation : public IFunction, private WithContext
{
public:
static constexpr auto name = "initializeAggregation";
- static FunctionPtr create(ContextPtr) { return std::make_shared(); }
+ static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); }
+ FunctionInitializeAggregation(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; }
@@ -78,7 +79,7 @@ DataTypePtr FunctionInitializeAggregation::getReturnTypeImpl(const ColumnsWithTy
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params,
- aggregate_function_name, params_row, "function " + getName());
+ aggregate_function_name, params_row, "function " + getName(), getContext());
AggregateFunctionProperties properties;
aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row, properties);
diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp
index 5b2339975c1..e693d4ba988 100644
--- a/src/Interpreters/ExpressionAnalyzer.cpp
+++ b/src/Interpreters/ExpressionAnalyzer.cpp
@@ -468,7 +468,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
}
AggregateFunctionProperties properties;
- aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters) : Array();
+ aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters, "", getContext()) : Array();
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters, properties);
aggregate_descriptions.push_back(aggregate);
@@ -651,7 +651,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
window_function.function_parameters
= window_function.function_node->parameters
? getAggregateFunctionParametersArray(
- window_function.function_node->parameters)
+ window_function.function_node->parameters, "", getContext())
: Array();
// Requiring a constant reference to a shared pointer to non-const AST
diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp
index 90f6ac84afc..d91ea9208e4 100644
--- a/src/Interpreters/evaluateConstantExpression.cpp
+++ b/src/Interpreters/evaluateConstantExpression.cpp
@@ -49,17 +49,20 @@ std::pair> evaluateConstantExpression(co
expr_for_constant_folding->execute(block_with_constants);
if (!block_with_constants || block_with_constants.rows() == 0)
- throw Exception("Logical error: empty block after evaluation of constant expression for IN, VALUES or LIMIT", ErrorCodes::LOGICAL_ERROR);
+ throw Exception("Logical error: empty block after evaluation of constant expression for IN, VALUES or LIMIT or aggregate function parameter",
+ ErrorCodes::LOGICAL_ERROR);
if (!block_with_constants.has(name))
- throw Exception("Element of set in IN, VALUES or LIMIT is not a constant expression (result column not found): " + name, ErrorCodes::BAD_ARGUMENTS);
+ throw Exception(ErrorCodes::BAD_ARGUMENTS,
+ "Element of set in IN, VALUES or LIMIT or aggregate function parameter is not a constant expression (result column not found): {}", name);
const ColumnWithTypeAndName & result = block_with_constants.getByName(name);
const IColumn & result_column = *result.column;
/// Expressions like rand() or now() are not constant
if (!isColumnConst(result_column))
- throw Exception("Element of set in IN, VALUES or LIMIT is not a constant expression (result column is not const): " + name, ErrorCodes::BAD_ARGUMENTS);
+ throw Exception(ErrorCodes::BAD_ARGUMENTS,
+ "Element of set in IN, VALUES or LIMIT or aggregate function parameter is not a constant expression (result column is not const): {}", name);
return std::make_pair(result_column[0], result.type);
}
diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp
index b3b9ce31ff5..539f7713320 100644
--- a/src/Storages/MergeTree/registerStorageMergeTree.cpp
+++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp
@@ -116,8 +116,11 @@ static bool compareRetentions(const Graphite::Retention & a, const Graphite::Ret
*
*
*/
-static void
-appendGraphitePattern(const Poco::Util::AbstractConfiguration & config, const String & config_element, Graphite::Patterns & patterns)
+static void appendGraphitePattern(
+ const Poco::Util::AbstractConfiguration & config,
+ const String & config_element,
+ Graphite::Patterns & out_patterns,
+ ContextPtr context)
{
Graphite::Pattern pattern;
@@ -137,7 +140,7 @@ appendGraphitePattern(const Poco::Util::AbstractConfiguration & config, const St
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(
- aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization");
+ aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context);
/// TODO Not only Float64
AggregateFunctionProperties properties;
@@ -181,7 +184,7 @@ appendGraphitePattern(const Poco::Util::AbstractConfiguration & config, const St
if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll
std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions);
- patterns.emplace_back(pattern);
+ out_patterns.emplace_back(pattern);
}
static void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params)
@@ -204,7 +207,7 @@ static void setGraphitePatternsFromConfig(ContextPtr context, const String & con
{
if (startsWith(key, "pattern"))
{
- appendGraphitePattern(config, config_element + "." + key, params.patterns);
+ appendGraphitePattern(config, config_element + "." + key, params.patterns, context);
}
else if (key == "default")
{
@@ -219,7 +222,7 @@ static void setGraphitePatternsFromConfig(ContextPtr context, const String & con
}
if (config.has(config_element + ".default"))
- appendGraphitePattern(config, config_element + "." + ".default", params.patterns);
+ appendGraphitePattern(config, config_element + "." + ".default", params.patterns, context);
}
diff --git a/tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.reference b/tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.reference
new file mode 100644
index 00000000000..61be3e78ae7
--- /dev/null
+++ b/tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.reference
@@ -0,0 +1,2 @@
+[0,1,2,3,4]
+[0,1,2,3,4]
diff --git a/tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.sql b/tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.sql
new file mode 100644
index 00000000000..3ab969ca256
--- /dev/null
+++ b/tests/queries/0_stateless/01934_constexpr_aggregate_function_parameters.sql
@@ -0,0 +1,11 @@
+SELECT groupArray(2 + 3)(number) FROM numbers(10);
+SELECT groupArray('5'::UInt8)(number) FROM numbers(10);
+
+SELECT groupArray()(number) FROM numbers(10); -- { serverError 36 }
+SELECT groupArray(NULL)(number) FROM numbers(10); -- { serverError 36 }
+SELECT groupArray(NULL + NULL)(number) FROM numbers(10); -- { serverError 36 }
+SELECT groupArray([])(number) FROM numbers(10); -- { serverError 36 }
+SELECT groupArray(throwIf(1))(number) FROM numbers(10); -- { serverError 395 }
+
+-- Not the best error message, can be improved.
+SELECT groupArray(number)(number) FROM numbers(10); -- { serverError 47 }
From b8a0b4caf48654aa92d47ce2b2e6e9abafcedd4e Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 03:57:46 +0300
Subject: [PATCH 36/56] One more test
---
...ametrized_query_parametric_aggregate_function.reference | 1 +
...935_parametrized_query_parametric_aggregate_function.sh | 7 +++++++
2 files changed, 8 insertions(+)
create mode 100644 tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.reference
create mode 100755 tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.sh
diff --git a/tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.reference b/tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.reference
new file mode 100644
index 00000000000..0cfbf08886f
--- /dev/null
+++ b/tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.reference
@@ -0,0 +1 @@
+2
diff --git a/tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.sh b/tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.sh
new file mode 100755
index 00000000000..bbc24af1214
--- /dev/null
+++ b/tests/queries/0_stateless/01935_parametrized_query_parametric_aggregate_function.sh
@@ -0,0 +1,7 @@
+#!/usr/bin/env bash
+
+CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CUR_DIR"/../shell_config.sh
+
+${CLICKHOUSE_CURL} -sS -XPOST "${CLICKHOUSE_URL}¶m_lim=2" --data-binary 'select length(topKArray({lim:UInt32})([1,1,2,3,4,5,6,7,7,7]))'
From 23912c606609e257fa1ecb282b294674999ee8a2 Mon Sep 17 00:00:00 2001
From: alexey-milovidov
Date: Fri, 2 Jul 2021 04:05:28 +0300
Subject: [PATCH 37/56] Update adopters.md
---
docs/en/introduction/adopters.md | 1 +
1 file changed, 1 insertion(+)
diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md
index fdced7f354c..47927cd306a 100644
--- a/docs/en/introduction/adopters.md
+++ b/docs/en/introduction/adopters.md
@@ -155,5 +155,6 @@ toc_title: Adopters
| Argedor | ClickHouse support | — | — | — | [Official website](https://www.argedor.com/en/clickhouse/) |
| SigNoz | Observability Platform | Main Product | — | — | [Source code](https://github.com/SigNoz/signoz) |
| ChelPipe Group | Analytics | — | — | — | [Blog post, June 2021](https://vc.ru/trade/253172-tyazhelomu-proizvodstvu-user-friendly-sayt-internet-magazin-trub-dlya-chtpz) |
+| Zagrava Trading | — | — | — | — | [Job offer, May 2021](https://twitter.com/datastackjobs/status/1394707267082063874) |
[Original article](https://clickhouse.tech/docs/en/introduction/adopters/)
From a41a1b7c74d7eb1a15bc731f902174d628754b07 Mon Sep 17 00:00:00 2001
From: alexey-milovidov
Date: Fri, 2 Jul 2021 04:25:07 +0300
Subject: [PATCH 38/56] Update ReplxxLineReader.cpp
---
base/common/ReplxxLineReader.cpp | 5 ++++-
1 file changed, 4 insertions(+), 1 deletion(-)
diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp
index 3d665744223..9c65b1dfe4c 100644
--- a/base/common/ReplxxLineReader.cpp
+++ b/base/common/ReplxxLineReader.cpp
@@ -25,7 +25,10 @@ void trim(String & s)
s.erase(std::find_if(s.rbegin(), s.rend(), [](int ch) { return !std::isspace(ch); }).base(), s.end());
}
-/// Copied from replxx::src/util.cxx::now_ms_str()
+/// Copied from replxx::src/util.cxx::now_ms_str() under the terms of 3-clause BSD license of Replxx.
+/// Copyright (c) 2017-2018, Marcin Konarski (amok at codestation.org)
+/// Copyright (c) 2010, Salvatore Sanfilippo (antirez at gmail dot com)
+/// Copyright (c) 2010, Pieter Noordhuis (pcnoordhuis at gmail dot com)
std::string replxx_now_ms_str()
{
std::chrono::milliseconds ms(std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()));
From 24759a9b67f5e002349f0161546dce3aa98fcd54 Mon Sep 17 00:00:00 2001
From: alexey-milovidov
Date: Fri, 2 Jul 2021 04:26:49 +0300
Subject: [PATCH 39/56] Update ReplxxLineReader.cpp
---
base/common/ReplxxLineReader.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp
index 9c65b1dfe4c..de169b1581b 100644
--- a/base/common/ReplxxLineReader.cpp
+++ b/base/common/ReplxxLineReader.cpp
@@ -29,7 +29,7 @@ void trim(String & s)
/// Copyright (c) 2017-2018, Marcin Konarski (amok at codestation.org)
/// Copyright (c) 2010, Salvatore Sanfilippo (antirez at gmail dot com)
/// Copyright (c) 2010, Pieter Noordhuis (pcnoordhuis at gmail dot com)
-std::string replxx_now_ms_str()
+static std::string replxx_now_ms_str()
{
std::chrono::milliseconds ms(std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()));
time_t t = ms.count() / 1000;
From 1d332da0ed8f5e9f042da8584ab8ce21f38cf7b9 Mon Sep 17 00:00:00 2001
From: feng lv
Date: Fri, 2 Jul 2021 05:51:53 +0000
Subject: [PATCH 40/56] fix special build on clang 11
---
base/common/ReplxxLineReader.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp
index de169b1581b..9c65b1dfe4c 100644
--- a/base/common/ReplxxLineReader.cpp
+++ b/base/common/ReplxxLineReader.cpp
@@ -29,7 +29,7 @@ void trim(String & s)
/// Copyright (c) 2017-2018, Marcin Konarski (amok at codestation.org)
/// Copyright (c) 2010, Salvatore Sanfilippo (antirez at gmail dot com)
/// Copyright (c) 2010, Pieter Noordhuis (pcnoordhuis at gmail dot com)
-static std::string replxx_now_ms_str()
+std::string replxx_now_ms_str()
{
std::chrono::milliseconds ms(std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()));
time_t t = ms.count() / 1000;
From 3ae127839188a1da088c5c1681831f02183ee098 Mon Sep 17 00:00:00 2001
From: Maksim Kita
Date: Fri, 2 Jul 2021 11:22:30 +0300
Subject: [PATCH 41/56] Fixed tests
---
src/Functions/array/arrayReduce.cpp | 2 +-
src/Functions/array/arrayReduceInRanges.cpp | 2 +-
tests/queries/skip_list.json | 3 ++-
3 files changed, 4 insertions(+), 3 deletions(-)
diff --git a/src/Functions/array/arrayReduce.cpp b/src/Functions/array/arrayReduce.cpp
index 3387d50a1f6..e070596e5ee 100644
--- a/src/Functions/array/arrayReduce.cpp
+++ b/src/Functions/array/arrayReduce.cpp
@@ -38,7 +38,7 @@ class FunctionArrayReduce : public IFunction, private WithContext
public:
static constexpr auto name = "arrayReduce";
static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); }
- FunctionArrayReduce(ContextPtr context_) : WithContext(context_) {}
+ explicit FunctionArrayReduce(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; }
diff --git a/src/Functions/array/arrayReduceInRanges.cpp b/src/Functions/array/arrayReduceInRanges.cpp
index ffb047f2231..18140fe504d 100644
--- a/src/Functions/array/arrayReduceInRanges.cpp
+++ b/src/Functions/array/arrayReduceInRanges.cpp
@@ -41,7 +41,7 @@ public:
static const size_t minimum_step = 64;
static constexpr auto name = "arrayReduceInRanges";
static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); }
- FunctionArrayReduceInRanges(ContextPtr context_) : WithContext(context_) {}
+ explicit FunctionArrayReduceInRanges(ContextPtr context_) : WithContext(context_) {}
String getName() const override { return name; }
diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json
index 7c1f998e91d..64aef86ec9c 100644
--- a/tests/queries/skip_list.json
+++ b/tests/queries/skip_list.json
@@ -520,7 +520,8 @@
"01914_exchange_dictionaries",
"01915_create_or_replace_dictionary",
"01913_names_of_tuple_literal",
- "01925_merge_prewhere_table"
+ "01925_merge_prewhere_table",
+ "01934_constexpr_aggregate_function_parameters"
],
"parallel":
[
From 55889eacf507f3173c851fc83543d961b59967d8 Mon Sep 17 00:00:00 2001
From: Maksim Kita
Date: Fri, 2 Jul 2021 11:24:45 +0300
Subject: [PATCH 42/56] Add test to ANTLR skip list
---
tests/queries/skip_list.json | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json
index 7c1f998e91d..803199b9121 100644
--- a/tests/queries/skip_list.json
+++ b/tests/queries/skip_list.json
@@ -520,7 +520,8 @@
"01914_exchange_dictionaries",
"01915_create_or_replace_dictionary",
"01913_names_of_tuple_literal",
- "01925_merge_prewhere_table"
+ "01925_merge_prewhere_table",
+ "01932_null_valid_identifier"
],
"parallel":
[
From e992ed780a23af724113c9f2f619de2e61a8a06f Mon Sep 17 00:00:00 2001
From: Maksim Kita
Date: Fri, 2 Jul 2021 11:30:57 +0300
Subject: [PATCH 43/56] Update rabbitmq.md
---
docs/zh/engines/table-engines/integrations/rabbitmq.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/zh/engines/table-engines/integrations/rabbitmq.md b/docs/zh/engines/table-engines/integrations/rabbitmq.md
index c43218da14f..a4a5be5f685 100644
--- a/docs/zh/engines/table-engines/integrations/rabbitmq.md
+++ b/docs/zh/engines/table-engines/integrations/rabbitmq.md
@@ -96,7 +96,7 @@ RabbitMQ 服务器配置应使用 ClickHouse 配置文件添加。
## 描述 {#description}
-`SELECT`对于读取消息不是特别有用(除了调试),因为每个消息只能读取一次。使用[物化视图](../../../sql-reference/statements/create/view.md)创建实时线程更为实用。要做到这一点:
+`SELECT`对于读取消息不是特别有用(除了调试),因为每个消息只能读取一次。使用[物化视图](../../../sql-reference/statements/create.md#create-view)创建实时线程更为实用。要做到这一点:
1. 使用引擎创建一个 RabbitMQ 消费者,并将其视为一个数据流。
2. 创建一个具有所需结构的表。
From e4a0e831f0e3a51ddc787475fb7ae7a7b0e7f415 Mon Sep 17 00:00:00 2001
From: Maksim Kita
Date: Fri, 2 Jul 2021 14:08:11 +0300
Subject: [PATCH 44/56] Update skip_list.json
---
tests/queries/skip_list.json | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json
index 0cd57ed39fb..be52bee71b1 100644
--- a/tests/queries/skip_list.json
+++ b/tests/queries/skip_list.json
@@ -520,7 +520,7 @@
"01914_exchange_dictionaries",
"01915_create_or_replace_dictionary",
"01913_names_of_tuple_literal",
- "01925_merge_prewhere_table"
+ "01925_merge_prewhere_table",
"01932_null_valid_identifier",
"01934_constexpr_aggregate_function_parameters"
],
From 8b4fabe60ce6aa3c5e62c2bb799ff76a36a71181 Mon Sep 17 00:00:00 2001
From: Vitaly Baranov
Date: Fri, 2 Jul 2021 14:20:41 +0300
Subject: [PATCH 45/56] Fix crash on call dictGet() with bad arguments.
---
.../MarkTableIdentifiersVisitor.cpp | 43 +++++++++++--------
.../MarkTableIdentifiersVisitor.h | 2 +-
...arts_identifiers_in_wrong_places.reference | 1 +
...hree_parts_identifiers_in_wrong_places.sql | 7 +++
4 files changed, 33 insertions(+), 20 deletions(-)
create mode 100644 tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.reference
create mode 100644 tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql
diff --git a/src/Interpreters/MarkTableIdentifiersVisitor.cpp b/src/Interpreters/MarkTableIdentifiersVisitor.cpp
index 52f180aa199..1f418e759e7 100644
--- a/src/Interpreters/MarkTableIdentifiersVisitor.cpp
+++ b/src/Interpreters/MarkTableIdentifiersVisitor.cpp
@@ -11,6 +11,26 @@
namespace DB
{
+namespace
+{
+ void replaceArgumentWithTableIdentifierIfNotAlias(ASTFunction & func, size_t argument_pos, const Aliases & aliases)
+ {
+ if (!func.arguments || (func.arguments->children.size() <= argument_pos))
+ return;
+ auto arg = func.arguments->children[argument_pos];
+ auto identifier = arg->as();
+ if (!identifier)
+ return;
+ if (aliases.contains(identifier->name()))
+ return;
+ auto table_identifier = identifier->createTable();
+ if (!table_identifier)
+ return;
+ func.arguments->children[argument_pos] = table_identifier;
+ }
+}
+
+
bool MarkTableIdentifiersMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
{
if (child->as())
@@ -23,37 +43,22 @@ bool MarkTableIdentifiersMatcher::needChildVisit(ASTPtr & node, const ASTPtr & c
void MarkTableIdentifiersMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * node_func = ast->as())
- visit(*node_func, ast, data);
+ visit(*node_func, data);
}
-void MarkTableIdentifiersMatcher::visit(const ASTFunction & func, ASTPtr & ptr, Data & data)
+void MarkTableIdentifiersMatcher::visit(ASTFunction & func, const Data & data)
{
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (checkFunctionIsInOrGlobalInOperator(func))
{
- auto ast = func.arguments->children.at(1);
- auto opt_name = tryGetIdentifierName(ast);
- if (opt_name && !data.aliases.count(*opt_name) && ast->as())
- {
- ptr->as()->arguments->children[1] = ast->as()->createTable();
- assert(ptr->as()->arguments->children[1]);
- }
+ replaceArgumentWithTableIdentifierIfNotAlias(func, 1, data.aliases);
}
// First argument of joinGet can be a table name, perhaps with a database.
// First argument of dictGet can be a dictionary name, perhaps with a database.
else if (functionIsJoinGet(func.name) || functionIsDictGet(func.name))
{
- if (!func.arguments || func.arguments->children.empty())
- return;
-
- auto ast = func.arguments->children.at(0);
- auto opt_name = tryGetIdentifierName(ast);
- if (opt_name && !data.aliases.count(*opt_name) && ast->as())
- {
- ptr->as()->arguments->children[0] = ast->as()->createTable();
- assert(ptr->as()->arguments->children[0]);
- }
+ replaceArgumentWithTableIdentifierIfNotAlias(func, 0, data.aliases);
}
}
diff --git a/src/Interpreters/MarkTableIdentifiersVisitor.h b/src/Interpreters/MarkTableIdentifiersVisitor.h
index 0d80b865e53..d05c067397b 100644
--- a/src/Interpreters/MarkTableIdentifiersVisitor.h
+++ b/src/Interpreters/MarkTableIdentifiersVisitor.h
@@ -24,7 +24,7 @@ public:
static void visit(ASTPtr & ast, Data & data);
private:
- static void visit(const ASTFunction & func, ASTPtr &, Data &);
+ static void visit(ASTFunction & func, const Data & data);
};
using MarkTableIdentifiersVisitor = MarkTableIdentifiersMatcher::Visitor;
diff --git a/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.reference b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.reference
new file mode 100644
index 00000000000..bbf76e61257
--- /dev/null
+++ b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.reference
@@ -0,0 +1 @@
+still alive
diff --git a/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql
new file mode 100644
index 00000000000..d2ca771edc5
--- /dev/null
+++ b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql
@@ -0,0 +1,7 @@
+SELECT dictGet(t.nest.a, concat(currentDatabase(), '.dict.dict'), 's', number) FROM numbers(5); -- { serverError 47 }
+
+SELECT dictGetFloat64(t.b.s, 'database_for_dict.dict1', dictGetFloat64('Ta\0', toUInt64('databas\0_for_dict.dict1databas\0_for_dict.dict1', dictGetFloat64('', '', toUInt64(1048577), toDate(NULL)), NULL), toDate(dictGetFloat64(257, 'database_for_dict.dict1database_for_dict.dict1', '', toUInt64(NULL), 2, toDate(NULL)), '2019-05-2\0')), NULL, toUInt64(dictGetFloat64('', '', toUInt64(-9223372036854775808), toDate(NULL)), NULL)); -- { serverError 47 }
+
+SELECT NULL AND (2147483648 AND NULL) AND -2147483647, toUUID(((1048576 AND NULL) AND (2147483647 AND 257 AND NULL AND -2147483649) AND NULL) IN (test_01103.t1_distr.id), '00000000-e1fe-11e\0-bb8f\0853d60c00749'), stringToH3('89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff'); -- { serverError 47 }
+
+SELECT 'still alive';
From 23dd7544922fdc62369a8271169702203419b6e0 Mon Sep 17 00:00:00 2001
From: kssenii
Date: Fri, 2 Jul 2021 22:26:33 +0300
Subject: [PATCH 46/56] Update libpq
---
contrib/libpq | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/contrib/libpq b/contrib/libpq
index 69e8a80e98f..e071ea570f8 160000
--- a/contrib/libpq
+++ b/contrib/libpq
@@ -1 +1 @@
-Subproject commit 69e8a80e98f27e3a5deec617334e31db2b9ed7d7
+Subproject commit e071ea570f8985aa00e34f5b9d50a3cfe666327e
From 132edc9e2217ae99e7936560779ff6b2daefa327 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Fri, 2 Jul 2021 22:47:43 +0300
Subject: [PATCH 47/56] Allow quantiles* functions to work with
`aggregate_functions_null_for_empty`
---
.../AggregateFunctionQuantile.cpp | 29 ++++++++++---------
...936_quantiles_cannot_return_null.reference | 4 +++
.../01936_quantiles_cannot_return_null.sql | 9 ++++++
3 files changed, 29 insertions(+), 13 deletions(-)
create mode 100644 tests/queries/0_stateless/01936_quantiles_cannot_return_null.reference
create mode 100644 tests/queries/0_stateless/01936_quantiles_cannot_return_null.sql
diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/src/AggregateFunctions/AggregateFunctionQuantile.cpp
index cae0021082f..11b14585653 100644
--- a/src/AggregateFunctions/AggregateFunctionQuantile.cpp
+++ b/src/AggregateFunctions/AggregateFunctionQuantile.cpp
@@ -125,44 +125,47 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
{
+ /// For aggregate functions returning array we cannot return NULL on empty set.
+ AggregateFunctionProperties properties = { .returns_default_when_only_null = true };
+
factory.registerFunction(NameQuantile::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantiles::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantiles::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileDeterministic::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesDeterministic::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesDeterministic::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileExact::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesExact::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesExact::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileExactLow::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesExactLow::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesExactLow::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileExactHigh::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesExactHigh::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesExactHigh::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileExactExclusive::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesExactExclusive::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesExactExclusive::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileExactInclusive::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesExactInclusive::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesExactInclusive::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileExactWeighted::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesExactWeighted::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesExactWeighted::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileTiming::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesTiming::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesTiming::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileTimingWeighted::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesTimingWeighted::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesTimingWeighted::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileTDigest::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesTDigest::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesTDigest::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileTDigestWeighted::name, createAggregateFunctionQuantile);
- factory.registerFunction(NameQuantilesTDigestWeighted::name, createAggregateFunctionQuantile);
+ factory.registerFunction(NameQuantilesTDigestWeighted::name, { createAggregateFunctionQuantile, properties });
factory.registerFunction(NameQuantileBFloat16::name, createAggregateFunctionQuantile