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

This commit is contained in:
avogar 2024-07-16 19:07:52 +00:00
commit 44b4a5d9c3
26 changed files with 226 additions and 22 deletions

View File

@ -12,7 +12,7 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME))
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=1
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0

View File

@ -96,3 +96,22 @@ Result:
│ 1 │ [2] │ [[4,1]] │
└───────────┴───────────┴───────────┘
```
## Reading nested subcolumns from Array
If nested type `T` inside `Array` has subcolumns (for example, if it's a [named tuple](./tuple.md)), you can read its subcolumns from an `Array(T)` type with the same subcolumn names. The type of a subcolumn will be `Array` of the type of original subcolumn.
**Example**
```sql
CREATE TABLE t_arr (arr Array(Tuple(field1 UInt32, field2 String))) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO t_arr VALUES ([(1, 'Hello'), (2, 'World')]), ([(3, 'This'), (4, 'is'), (5, 'subcolumn')]);
SELECT arr.field1, toTypeName(arr.field1), arr.field2, toTypeName(arr.field2) from t_arr;
```
```test
┌─arr.field1─┬─toTypeName(arr.field1)─┬─arr.field2────────────────┬─toTypeName(arr.field2)─┐
│ [1,2] │ Array(UInt32) │ ['Hello','World'] │ Array(String) │
│ [3,4,5] │ Array(UInt32) │ ['This','is','subcolumn'] │ Array(String) │
└────────────┴────────────────────────┴───────────────────────────┴────────────────────────┘
```

View File

@ -3832,6 +3832,10 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(
node->convertToNullable();
break;
}
/// Check parent scopes until find current query scope.
if (scope_ptr->scope_node->getNodeType() == QueryTreeNodeType::QUERY)
break;
}
}

View File

@ -55,9 +55,9 @@ namespace
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.auth_settings.region,
context->getRemoteHostFilter(),
static_cast<unsigned>(global_settings.s3_max_redirects),
static_cast<unsigned>(global_settings.s3_retry_attempts),
global_settings.enable_s3_requests_logging,
static_cast<unsigned>(local_settings.s3_max_redirects),
static_cast<unsigned>(local_settings.backup_restore_s3_retry_attempts),
local_settings.enable_s3_requests_logging,
/* for_disk_s3 = */ false,
request_settings.get_request_throttler,
request_settings.put_request_throttler,

View File

@ -80,6 +80,7 @@
#include <Common/config_version.h>
#include "config.h"
namespace fs = std::filesystem;
using namespace std::literals;
@ -2565,12 +2566,12 @@ void ClientBase::runInteractive()
word_break_characters,
highlight_callback);
#else
(void)word_break_characters;
LineReader lr(
history_file,
getClientConfiguration().has("multiline"),
query_extenders,
query_delimiters,
word_break_characters);
query_delimiters);
#endif
static const std::initializer_list<std::pair<String, String>> backslash_aliases =

View File

@ -46,8 +46,7 @@ public:
Patterns delimiters,
std::istream & input_stream_ = std::cin,
std::ostream & output_stream_ = std::cout,
int in_fd_ = STDIN_FILENO
);
int in_fd_ = STDIN_FILENO);
virtual ~LineReader() = default;

View File

@ -362,6 +362,9 @@ ReplxxLineReader::ReplxxLineReader(
rx.bind_key(Replxx::KEY::control('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_NEXT, code); });
rx.bind_key(Replxx::KEY::control('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_PREVIOUS, code); });
/// We don't want the default, "suspend" behavior, it confuses people.
rx.bind_key_internal(replxx::Replxx::KEY::control('Z'), "insert_character");
auto commit_action = [this](char32_t code)
{
/// If we allow multiline and there is already something in the input, start a newline.

View File

@ -502,6 +502,7 @@ class IColumn;
M(UInt64, backup_restore_keeper_value_max_size, 1048576, "Maximum size of data of a [Zoo]Keeper's node during backup", 0) \
M(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore", 0) \
M(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, "Maximum size of batch for multi request to [Zoo]Keeper during backup or restore", 0) \
M(UInt64, backup_restore_s3_retry_attempts, 1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore.", 0) \
M(UInt64, max_backup_bandwidth, 0, "The maximum read speed in bytes per second for particular backup on server. Zero means unlimited.", 0) \
\
M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \

View File

@ -78,6 +78,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"},
{"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"},
{"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."},
{"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."},
{"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."},
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}
}},

View File

@ -162,7 +162,7 @@ public:
class RetryStrategy : public Aws::Client::RetryStrategy
{
public:
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000);
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 5000);
/// NOLINTNEXTLINE(google-runtime-int)
bool ShouldRetry(const Aws::Client::AWSError<Aws::Client::CoreErrors>& error, long attemptedRetries) const override;

View File

@ -125,7 +125,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
if (isCrossOrComma(kind))
{
data->type = Type::CROSS;
sample_block_with_columns_to_add = right_sample_block;
sample_block_with_columns_to_add = materializeBlock(right_sample_block);
}
else if (table_join->getClauses().empty())
{

View File

@ -226,10 +226,11 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_)
if (chunks_to_write.empty())
return;
size_t num_chunks = chunks_to_write.size();
auto [serialized_key_column, serialized_value_column]
= storage.ttl > 0 ? serializeChunks<true>(std::move(chunks_to_write)) : serializeChunks<false>(std::move(chunks_to_write));
auto sst_file_path = getTemporarySSTFilePath();
LOG_DEBUG(getLogger("EmbeddedRocksDBBulkSink"), "Writing {} rows to SST file {}", serialized_key_column->size(), sst_file_path);
LOG_DEBUG(getLogger("EmbeddedRocksDBBulkSink"), "Writing {} rows from {} chunks to SST file {}", serialized_key_column->size(), num_chunks, sst_file_path);
if (auto status = buildSSTFile(sst_file_path, *serialized_key_column, *serialized_value_column); !status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString());

View File

@ -420,7 +420,7 @@ String transformQueryForExternalDatabase(
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "No column names for query '{}' to external table '{}.{}'",
query_info.query_tree->formatASTForErrorMessage(), database, table);
auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree);
auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree, query_info.table_expression);
return transformQueryForExternalDatabaseImpl(
clone_query,

View File

@ -3,6 +3,7 @@
#include <Storages/transformQueryForExternalDatabaseAnalyzer.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Columns/ColumnConst.h>
@ -10,7 +11,7 @@
#include <Analyzer/QueryNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/ConstantValue.h>
#include <Analyzer/JoinNode.h>
#include <DataTypes/DataTypesNumber.h>
@ -20,6 +21,7 @@ namespace DB
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR;
}
namespace
@ -55,7 +57,7 @@ public:
}
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree)
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree, const QueryTreeNodePtr & table_expression)
{
auto new_tree = query_tree->clone();
@ -63,6 +65,21 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre
visitor.visit(new_tree);
const auto * query_node = new_tree->as<QueryNode>();
const auto & join_tree = query_node->getJoinTree();
bool allow_where = true;
if (const auto * join_node = join_tree->as<JoinNode>())
{
if (join_node->getStrictness() != JoinStrictness::All)
allow_where = false;
if (join_node->getKind() == JoinKind::Left)
allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression);
else if (join_node->getKind() == JoinKind::Right)
allow_where = join_node->getRightTableExpression()->isEqual(*table_expression);
else
allow_where = (join_node->getKind() == JoinKind::Inner);
}
auto query_node_ast = query_node->toAST({ .add_cast_for_constants = false, .fully_qualified_identifiers = false });
const IAST * ast = query_node_ast.get();
@ -76,7 +93,13 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre
if (union_ast->list_of_selects->children.size() != 1)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryNode AST is not a single ASTSelectQuery, got {}", union_ast->list_of_selects->children.size());
return union_ast->list_of_selects->children.at(0);
ASTPtr select_query = union_ast->list_of_selects->children.at(0);
auto * select_query_typed = select_query->as<ASTSelectQuery>();
if (!select_query_typed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ASTSelectQuery, got {}", select_query ? select_query->formatForErrorMessage() : "nullptr");
if (!allow_where)
select_query_typed->setExpression(ASTSelectQuery::Expression::WHERE, nullptr);
return select_query;
}
}

View File

@ -6,6 +6,6 @@
namespace DB
{
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree);
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree, const QueryTreeNodePtr & table_expression);
}

View File

@ -2,6 +2,7 @@
<profiles>
<default>
<s3_retry_attempts>5</s3_retry_attempts>
<backup_restore_s3_retry_attempts>5</backup_restore_s3_retry_attempts>
</default>
</profiles>
<users>

View File

@ -834,6 +834,60 @@ def test_literal_escaping(started_cluster):
cursor.execute(f"DROP TABLE escaping")
def test_filter_pushdown(started_cluster):
cursor = started_cluster.postgres_conn.cursor()
cursor.execute("CREATE SCHEMA test_filter_pushdown")
cursor.execute(
"CREATE TABLE test_filter_pushdown.test_table (id integer, value integer)"
)
cursor.execute(
"INSERT INTO test_filter_pushdown.test_table VALUES (1, 10), (1, 110), (2, 0), (3, 33), (4, 0)"
)
node1.query(
"""
CREATE TABLE test_filter_pushdown_pg_table (id UInt32, value UInt32)
ENGINE PostgreSQL('postgres1:5432', 'postgres', 'test_table', 'postgres', 'mysecretpassword', 'test_filter_pushdown');
"""
)
node1.query(
"""
CREATE TABLE test_filter_pushdown_local_table (id UInt32, value UInt32) ENGINE Memory AS SELECT * FROM test_filter_pushdown_pg_table
"""
)
node1.query(
"CREATE TABLE ch_table (id UInt32, pg_id UInt32) ENGINE MergeTree ORDER BY id"
)
node1.query("INSERT INTO ch_table VALUES (1, 1), (2, 2), (3, 1), (4, 2), (5, 999)")
def compare_results(query, **kwargs):
result1 = node1.query(
query.format(pg_table="test_filter_pushdown_pg_table", **kwargs)
)
result2 = node1.query(
query.format(pg_table="test_filter_pushdown_local_table", **kwargs)
)
assert result1 == result2
for kind in ["INNER", "LEFT", "RIGHT", "FULL"]:
for value in [0, 10]:
compare_results(
"SELECT * FROM ch_table {kind} JOIN {pg_table} as p ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL",
kind=kind,
value=value,
)
compare_results(
"SELECT * FROM {pg_table} as p {kind} JOIN ch_table ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL",
kind=kind,
value=value,
)
cursor.execute("DROP SCHEMA test_filter_pushdown CASCADE")
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")

View File

@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 1 --min_inser
for _ in {1..60}; do
$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS"
[[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 100)) ]] && break;
[[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break;
sleep 1
done

View File

@ -73,7 +73,7 @@ kill -TERM $PID_1 && kill -TERM $PID_2 && kill -TERM $PID_3 && kill -TERM $PID_4
wait
$CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'threads finished'"
wait_for_queries_to_finish
wait_for_queries_to_finish 60
$CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table0"
$CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1"

View File

@ -264,3 +264,44 @@ SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10;
9 \N 9
\N 0 20
\N 1 25
CREATE TABLE test
ENGINE = ReplacingMergeTree
PRIMARY KEY id
AS SELECT number AS id FROM numbers(100);
SELECT id
FROM test
GROUP BY id
WITH CUBE
HAVING id IN (
SELECT id
FROM test
)
FORMAT `NUll`
SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true;
SELECT id
FROM test
FINAL
GROUP BY id
WITH CUBE
HAVING id IN (
SELECT DISTINCT id
FROM test
FINAL
)
FORMAT `NUll`
SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true;
SELECT id
FROM test
FINAL
GROUP BY
GROUPING SETS ((id))
ORDER BY
id IN (
SELECT DISTINCT id
FROM test
FINAL
LIMIT 4
) ASC
LIMIT 256 BY id
FORMAT `NUll`
SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls=true;

View File

@ -83,3 +83,48 @@ GROUP BY
)
ORDER BY 1, tuple(val)
SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10;
CREATE TABLE test
ENGINE = ReplacingMergeTree
PRIMARY KEY id
AS SELECT number AS id FROM numbers(100);
SELECT id
FROM test
GROUP BY id
WITH CUBE
HAVING id IN (
SELECT id
FROM test
)
FORMAT `NUll`
SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true;
SELECT id
FROM test
FINAL
GROUP BY id
WITH CUBE
HAVING id IN (
SELECT DISTINCT id
FROM test
FINAL
)
FORMAT `NUll`
SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true;
SELECT id
FROM test
FINAL
GROUP BY
GROUPING SETS ((id))
ORDER BY
id IN (
SELECT DISTINCT id
FROM test
FINAL
LIMIT 4
) ASC
LIMIT 256 BY id
FORMAT `NUll`
SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls=true;

View File

@ -3,7 +3,7 @@
1000
1
1000
2
1
1000000
1000
0 999001

View File

@ -29,7 +29,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
# Must set both max_threads and max_insert_threads to 2 to make sure there is only two sinks
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '', optimize_trivial_insert_select = 1;"
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 2 because default bulk sink size is ~1M rows / SST file
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) IN (1, 2) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be not more than 2 because default bulk sink size is ~1M rows / SST file.
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
# Testing insert with duplicated keys

View File

@ -7,12 +7,14 @@ SETTINGS max_bytes_to_merge_at_max_space_in_pool = 80000, exclude_deleted_rows_f
INSERT INTO lwd_merge SELECT number FROM numbers(10000);
INSERT INTO lwd_merge SELECT number FROM numbers(10000, 10000);
OPTIMIZE TABLE lwd_merge;
SET optimize_throw_if_noop = 1;
OPTIMIZE TABLE lwd_merge; -- { serverError CANNOT_ASSIGN_OPTIMIZE }
SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_merge' AND active = 1;
DELETE FROM lwd_merge WHERE id % 10 > 0;
OPTIMIZE TABLE lwd_merge;
OPTIMIZE TABLE lwd_merge; -- { serverError CANNOT_ASSIGN_OPTIMIZE }
SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_merge' AND active = 1;
ALTER TABLE lwd_merge MODIFY SETTING exclude_deleted_rows_for_part_size_in_merge = 1;

View File

@ -0,0 +1,2 @@
1 nan 1048575 2
1 1 1 1 1

View File

@ -0,0 +1,7 @@
SELECT * FROM (SELECT toUInt256(1)) AS t, (SELECT greatCircleAngle(toLowCardinality(toNullable(toUInt256(1048575))), 257, -9223372036854775808, 1048576), 1048575, materialize(2)) AS u;
SET join_algorithm='hash';
SET allow_experimental_join_condition=1;
SELECT * FROM ( SELECT 1 AS a, toLowCardinality(1), 1) AS t1 CROSS JOIN (SELECT toLowCardinality(1 AS a), 1 AS b) AS t2;