Skip to content

Commit 6a8b390

Browse files
authored
Merge branch 'antalya-25.8' into export_replicated_mt_partition_v2
2 parents 4024b52 + 615b144 commit 6a8b390

File tree

6 files changed

+132
-2
lines changed

6 files changed

+132
-2
lines changed

src/Parsers/ASTSystemQuery.cpp

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,13 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti
163163

164164
print_keyword("SYSTEM") << " ";
165165
print_keyword(typeToString(type));
166-
if (!cluster.empty())
166+
167+
std::unordered_set<Type> queries_with_on_cluster_at_end = {
168+
Type::DROP_FILESYSTEM_CACHE,
169+
Type::SYNC_FILESYSTEM_CACHE,
170+
};
171+
172+
if (!queries_with_on_cluster_at_end.contains(type) && !cluster.empty())
167173
formatOnCluster(ostr, settings);
168174

169175
switch (type)
@@ -519,6 +525,9 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti
519525
case Type::END:
520526
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown SYSTEM command");
521527
}
528+
529+
if (queries_with_on_cluster_at_end.contains(type) && !cluster.empty())
530+
formatOnCluster(ostr, settings);
522531
}
523532

524533

src/Storages/IStorage.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -412,6 +412,7 @@ class IStorage : public std::enable_shared_from_this<IStorage>, public TypePromo
412412
size_t /*max_block_size*/,
413413
size_t /*num_streams*/);
414414

415+
public:
415416
/// Should we process blocks of data returned by the storage in parallel
416417
/// even when the storage returned only one stream of data for reading?
417418
/// It is beneficial, for example, when you read from a file quickly,
@@ -422,7 +423,6 @@ class IStorage : public std::enable_shared_from_this<IStorage>, public TypePromo
422423
/// useless).
423424
virtual bool parallelizeOutputAfterReading(ContextPtr) const { return !isSystemStorage(); }
424425

425-
public:
426426
/// Other version of read which adds reading step to query plan.
427427
/// Default implementation creates ReadFromStorageStep and uses usual read.
428428
/// Can be called after `shutdown`, but not after `drop`.

src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp

Lines changed: 70 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -935,4 +935,74 @@ void StorageObjectStorageCluster::commitExportPartitionTransaction(
935935
return IStorageCluster::commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, local_context);
936936
}
937937

938+
bool StorageObjectStorageCluster::supportsPartitionBy() const
939+
{
940+
if (pure_storage)
941+
return pure_storage->supportsPartitionBy();
942+
return IStorageCluster::supportsPartitionBy();
943+
}
944+
945+
bool StorageObjectStorageCluster::supportsSubcolumns() const
946+
{
947+
if (pure_storage)
948+
return pure_storage->supportsSubcolumns();
949+
return IStorageCluster::supportsSubcolumns();
950+
}
951+
952+
bool StorageObjectStorageCluster::supportsDynamicSubcolumns() const
953+
{
954+
if (pure_storage)
955+
return pure_storage->supportsDynamicSubcolumns();
956+
return IStorageCluster::supportsDynamicSubcolumns();
957+
}
958+
959+
bool StorageObjectStorageCluster::supportsTrivialCountOptimization(const StorageSnapshotPtr & snapshot, ContextPtr context) const
960+
{
961+
if (pure_storage)
962+
return pure_storage->supportsTrivialCountOptimization(snapshot, context);
963+
return IStorageCluster::supportsTrivialCountOptimization(snapshot, context);
964+
}
965+
966+
bool StorageObjectStorageCluster::supportsPrewhere() const
967+
{
968+
if (pure_storage)
969+
return pure_storage->supportsPrewhere();
970+
return IStorageCluster::supportsPrewhere();
971+
}
972+
973+
bool StorageObjectStorageCluster::canMoveConditionsToPrewhere() const
974+
{
975+
if (pure_storage)
976+
return pure_storage->canMoveConditionsToPrewhere();
977+
return IStorageCluster::canMoveConditionsToPrewhere();
978+
}
979+
980+
std::optional<NameSet> StorageObjectStorageCluster::supportedPrewhereColumns() const
981+
{
982+
if (pure_storage)
983+
return pure_storage->supportedPrewhereColumns();
984+
return IStorageCluster::supportedPrewhereColumns();
985+
}
986+
987+
IStorageCluster::ColumnSizeByName StorageObjectStorageCluster::getColumnSizes() const
988+
{
989+
if (pure_storage)
990+
return pure_storage->getColumnSizes();
991+
return IStorageCluster::getColumnSizes();
992+
}
993+
994+
bool StorageObjectStorageCluster::parallelizeOutputAfterReading(ContextPtr context) const
995+
{
996+
if (pure_storage)
997+
return pure_storage->parallelizeOutputAfterReading(context);
998+
return IStorageCluster::parallelizeOutputAfterReading(context);
999+
}
1000+
1001+
bool StorageObjectStorageCluster::supportsDelete() const
1002+
{
1003+
if (pure_storage)
1004+
return pure_storage->supportsDelete();
1005+
return IStorageCluster::supportsDelete();
1006+
}
1007+
9381008
}

src/Storages/ObjectStorage/StorageObjectStorageCluster.h

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -139,6 +139,24 @@ class StorageObjectStorageCluster : public IStorageCluster
139139
const Strings & exported_paths,
140140
ContextPtr local_context) override;
141141

142+
bool supportsPartitionBy() const override;
143+
144+
bool supportsSubcolumns() const override;
145+
146+
bool supportsDynamicSubcolumns() const override;
147+
148+
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override;
149+
150+
/// Things required for PREWHERE.
151+
bool supportsPrewhere() const override;
152+
bool canMoveConditionsToPrewhere() const override;
153+
std::optional<NameSet> supportedPrewhereColumns() const override;
154+
ColumnSizeByName getColumnSizes() const override;
155+
156+
bool parallelizeOutputAfterReading(ContextPtr context) const override;
157+
158+
bool supportsDelete() const override;
159+
142160
private:
143161
void updateQueryToSendIfNeeded(
144162
ASTPtr & query,
Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,2 @@
1+
localhost 9000 0 0 0
2+
localhost 9000 0 0 0
Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
1+
#!/usr/bin/env bash
2+
# Tags: no-fasttest, no-parallel, no-object-storage, no-random-settings
3+
4+
# set -x
5+
6+
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
7+
# shellcheck source=../shell_config.sh
8+
. "$CUR_DIR"/../shell_config.sh
9+
10+
11+
disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}"
12+
$CLICKHOUSE_CLIENT -m --query """
13+
DROP TABLE IF EXISTS test;
14+
CREATE TABLE test (a Int32, b String)
15+
ENGINE = MergeTree() ORDER BY tuple()
16+
SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk);
17+
18+
INSERT INTO test SELECT 1, 'test';
19+
"""
20+
21+
$CLICKHOUSE_CLIENT --query """
22+
SYSTEM SYNC FILESYSTEM CACHE '$disk_name' ON CLUSTER 'test_shard_localhost';
23+
"""
24+
25+
$CLICKHOUSE_CLIENT --query """
26+
SYSTEM DROP FILESYSTEM CACHE '$disk_name' ON CLUSTER 'test_shard_localhost';
27+
"""
28+
29+
$CLICKHOUSE_CLIENT --query """
30+
DROP TABLE IF EXISTS test;
31+
"""

0 commit comments

Comments
 (0)