diff --git a/docker/test/coverage/s3downloader b/docker/test/coverage/s3downloader
index 363ece8dac6..eb3b3cd9faf 100755
--- a/docker/test/coverage/s3downloader
+++ b/docker/test/coverage/s3downloader
@@ -92,7 +92,7 @@ if __name__ == "__main__":
logging.info("Some exception occured %s", str(ex))
raise
finally:
- logging.info("Will remove dowloaded file %s from filesystem if it exists", temp_archive_path)
+ logging.info("Will remove downloaded file %s from filesystem if it exists", temp_archive_path)
if os.path.exists(temp_archive_path):
os.remove(temp_archive_path)
logging.info("Processing of %s finished", dataset)
diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader
index 363ece8dac6..eb3b3cd9faf 100755
--- a/docker/test/stateful/s3downloader
+++ b/docker/test/stateful/s3downloader
@@ -92,7 +92,7 @@ if __name__ == "__main__":
logging.info("Some exception occured %s", str(ex))
raise
finally:
- logging.info("Will remove dowloaded file %s from filesystem if it exists", temp_archive_path)
+ logging.info("Will remove downloaded file %s from filesystem if it exists", temp_archive_path)
if os.path.exists(temp_archive_path):
os.remove(temp_archive_path)
logging.info("Processing of %s finished", dataset)
diff --git a/docs/en/getting-started/example-datasets/metrica.md b/docs/en/getting-started/example-datasets/metrica.md
index 75b198ef323..483220d12ee 100644
--- a/docs/en/getting-started/example-datasets/metrica.md
+++ b/docs/en/getting-started/example-datasets/metrica.md
@@ -42,7 +42,11 @@ md5sum hits_v1.tsv
# Checksum should be equal to: f3631b6295bf06989c1437491f7592cb
# now create table
clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
+# for hits_v1
clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
+# for hits_100m_obfuscated
+clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
+
# import data
cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000
# optionally you can optimize table
diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md
index 943ee9e2a9f..0f018ae799f 100644
--- a/docs/en/introduction/adopters.md
+++ b/docs/en/introduction/adopters.md
@@ -56,6 +56,7 @@ toc_title: Adopters
| Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) |
| Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) |
| Glaber | Monitoring | Main product | — | — | [Website](https://glaber.io/) |
+| GraphCDN | CDN | Traffic Analytics | — | — | [Blog Post in English, August 2021](https://altinity.com/blog/delivering-insight-on-graphql-apis-with-clickhouse-at-graphcdn/) |
| HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) |
| ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) |
| Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.com/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) |
diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md
index faa86527c7d..3c3ed7b8932 100644
--- a/docs/en/sql-reference/functions/geo/h3.md
+++ b/docs/en/sql-reference/functions/geo/h3.md
@@ -594,14 +594,14 @@ Result:
└─────┘
```
-## h3ResIsClassIII {#h3resisclassIII}
+## h3IsResClassIII {#h3isresclassIII}
Returns whether [H3](#h3index) index has a resolution with Class III orientation.
**Syntax**
``` sql
-h3ResIsClassIII(index)
+h3IsResClassIII(index)
```
**Parameter**
@@ -620,7 +620,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT h3ResIsClassIII(617420388352917503) as res;
+SELECT h3IsResClassIII(617420388352917503) as res;
```
Result:
diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp
index c9a44dba6f2..4c2c64ef5e1 100644
--- a/src/AggregateFunctions/AggregateFunctionFactory.cpp
+++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp
@@ -155,7 +155,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
}
/// Combinators of aggregate functions.
- /// For every aggregate function 'agg' and combiner '-Comb' there is combined aggregate function with name 'aggComb',
+ /// For every aggregate function 'agg' and combiner '-Comb' there is a combined aggregate function with the name 'aggComb',
/// that can have different number and/or types of arguments, different result type and different behaviour.
if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name))
@@ -172,13 +172,12 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
String nested_name = name.substr(0, name.size() - combinator_name.size());
/// Nested identical combinators (i.e. uniqCombinedIfIf) is not
- /// supported (since they even don't work -- silently).
+ /// supported (since they don't work -- silently).
///
- /// But non-identical does supported and works, for example
- /// uniqCombinedIfMergeIf, it is useful in case when the underlying
+ /// But non-identical is supported and works. For example,
+ /// uniqCombinedIfMergeIf is useful in cases when the underlying
/// storage stores AggregateFunction(uniqCombinedIf) and in SELECT you
- /// need to filter aggregation result based on another column for
- /// example.
+ /// need to filter aggregation result based on another column.
if (!combinator->supportsNesting() && nested_name.ends_with(combinator_name))
{
throw Exception(ErrorCodes::ILLEGAL_AGGREGATION,
@@ -234,7 +233,7 @@ std::optional AggregateFunctionFactory::tryGetPrope
return found.properties;
/// Combinators of aggregate functions.
- /// For every aggregate function 'agg' and combiner '-Comb' there is combined aggregate function with name 'aggComb',
+ /// For every aggregate function 'agg' and combiner '-Comb' there is a combined aggregate function with the name 'aggComb',
/// that can have different number and/or types of arguments, different result type and different behaviour.
if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name))
diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp
index a64c4409cc9..3c78fddfd39 100644
--- a/src/DataStreams/RemoteQueryExecutor.cpp
+++ b/src/DataStreams/RemoteQueryExecutor.cpp
@@ -208,6 +208,12 @@ void RemoteQueryExecutor::sendQuery()
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
ClientInfo modified_client_info = context->getClientInfo();
modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
+ /// Set initial_query_id to query_id for the clickhouse-benchmark.
+ ///
+ /// (since first query of clickhouse-benchmark will be issued as SECONDARY_QUERY,
+ /// due to it executes queries via RemoteBlockInputStream)
+ if (modified_client_info.initial_query_id.empty())
+ modified_client_info.initial_query_id = query_id;
if (CurrentThread::isInitialized())
{
modified_client_info.client_trace_context = CurrentThread::get().thread_trace_context;
diff --git a/src/Functions/h3ResIsClassIII.cpp b/src/Functions/h3IsResClassIII.cpp
similarity index 88%
rename from src/Functions/h3ResIsClassIII.cpp
rename to src/Functions/h3IsResClassIII.cpp
index 044e6d565ac..25a2fc1952f 100644
--- a/src/Functions/h3ResIsClassIII.cpp
+++ b/src/Functions/h3IsResClassIII.cpp
@@ -24,12 +24,12 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT;
namespace
{
-class FunctionH3ResIsClassIII : public IFunction
+class FunctionH3IsResClassIII : public IFunction
{
public:
- static constexpr auto name = "h3ResIsClassIII";
+ static constexpr auto name = "h3IsResClassIII";
- static FunctionPtr create(ContextPtr) { return std::make_shared(); }
+ static FunctionPtr create(ContextPtr) { return std::make_shared(); }
std::string getName() const override { return name; }
@@ -69,9 +69,9 @@ public:
}
-void registerFunctionH3ResIsClassIII(FunctionFactory & factory)
+void registerFunctionH3IsResClassIII(FunctionFactory & factory)
{
- factory.registerFunction();
+ factory.registerFunction();
}
}
diff --git a/src/Functions/registerFunctionsGeo.cpp b/src/Functions/registerFunctionsGeo.cpp
index a86ff18ec65..a0ae38f6b85 100644
--- a/src/Functions/registerFunctionsGeo.cpp
+++ b/src/Functions/registerFunctionsGeo.cpp
@@ -42,7 +42,7 @@ void registerFunctionH3IndexesAreNeighbors(FunctionFactory &);
void registerFunctionStringToH3(FunctionFactory &);
void registerFunctionH3ToString(FunctionFactory &);
void registerFunctionH3HexAreaM2(FunctionFactory &);
-void registerFunctionH3ResIsClassIII(FunctionFactory &);
+void registerFunctionH3IsResClassIII(FunctionFactory &);
void registerFunctionH3IsPentagon(FunctionFactory &);
void registerFunctionH3GetFaces(FunctionFactory &);
#endif
@@ -98,7 +98,7 @@ void registerFunctionsGeo(FunctionFactory & factory)
registerFunctionStringToH3(factory);
registerFunctionH3ToString(factory);
registerFunctionH3HexAreaM2(factory);
- registerFunctionH3ResIsClassIII(factory);
+ registerFunctionH3IsResClassIII(factory);
registerFunctionH3IsPentagon(factory);
registerFunctionH3GetFaces(factory);
#endif
diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp
index 459c0879cda..7c5d36b37ce 100644
--- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp
+++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp
@@ -36,6 +36,7 @@
#include
#include
+#include
namespace fs = std::filesystem;
@@ -213,9 +214,9 @@ public:
std::vector slices_keys(num_keys);
const auto & sample_block = metadata_snapshot->getSampleBlock();
- const auto & key_column = sample_block.getByName(storage.primary_key);
+ const auto & key_column = sample_block.getByName(storage.getPrimaryKey());
auto columns = sample_block.cloneEmptyColumns();
- size_t primary_key_pos = sample_block.getPositionByName(storage.primary_key);
+ size_t primary_key_pos = sample_block.getPositionByName(storage.getPrimaryKey());
size_t rows_processed = 0;
while (it < end && rows_processed < max_block_size)
@@ -230,8 +231,7 @@ public:
}
std::vector values;
- auto statuses = storage.rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values);
-
+ auto statuses = storage.multiGet(slices_keys, values);
for (size_t i = 0; i < statuses.size(); ++i)
{
if (statuses[i].ok())
@@ -285,7 +285,10 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &)
{
+ std::unique_lock lock(rocksdb_ptr_mx);
rocksdb_ptr->Close();
+ rocksdb_ptr = nullptr;
+
fs::remove_all(rocksdb_dir);
fs::create_directories(rocksdb_dir);
initDb();
@@ -460,9 +463,20 @@ static StoragePtr create(const StorageFactory::Arguments & args)
std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const
{
+ std::shared_lock lock(rocksdb_ptr_mx);
+ if (!rocksdb_ptr)
+ return nullptr;
return rocksdb_ptr->GetOptions().statistics;
}
+std::vector StorageEmbeddedRocksDB::multiGet(const std::vector & slices_keys, std::vector & values) const
+{
+ std::shared_lock lock(rocksdb_ptr_mx);
+ if (!rocksdb_ptr)
+ return {};
+ return rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values);
+}
+
void registerStorageEmbeddedRocksDB(StorageFactory & factory)
{
StorageFactory::StorageFeatures features{
diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h
index 3f1b3b49492..71460a1667e 100644
--- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h
+++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h
@@ -1,8 +1,10 @@
#pragma once
#include
+#include
#include
#include
+#include
namespace rocksdb
@@ -20,7 +22,6 @@ class Context;
class StorageEmbeddedRocksDB final : public shared_ptr_helper, public IStorage, WithContext
{
friend struct shared_ptr_helper;
- friend class EmbeddedRocksDBSource;
friend class EmbeddedRocksDBSink;
friend class EmbeddedRocksDBBlockInputStream;
public:
@@ -50,6 +51,8 @@ public:
Strings getDataPaths() const override { return {rocksdb_dir}; }
std::shared_ptr getRocksDBStatistics() const;
+ std::vector multiGet(const std::vector & slices_keys, std::vector & values) const;
+ const String & getPrimaryKey() const { return primary_key; }
protected:
StorageEmbeddedRocksDB(const StorageID & table_id_,
@@ -63,6 +66,7 @@ private:
const String primary_key;
using RocksDBPtr = std::unique_ptr;
RocksDBPtr rocksdb_ptr;
+ mutable std::shared_mutex rocksdb_ptr_mx;
String rocksdb_dir;
void initDb();
diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp
index 7d31d5ddc21..cbb96ed4001 100644
--- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp
+++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp
@@ -43,7 +43,8 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con
const auto access = context->getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
- std::map> tables;
+ using RocksDBStoragePtr = std::shared_ptr;
+ std::map> tables;
for (const auto & db : DatabaseCatalog::instance().getDatabases())
{
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
@@ -51,18 +52,17 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
StoragePtr table = iterator->table();
- if (!table)
+ RocksDBStoragePtr rocksdb_table = table ? std::dynamic_pointer_cast(table) : nullptr;
+ if (!rocksdb_table)
continue;
- if (!dynamic_cast(table.get()))
- continue;
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
continue;
- tables[db.first][iterator->name()] = table;
+
+ tables[db.first][iterator->name()] = rocksdb_table;
}
}
-
MutableColumnPtr col_database_mut = ColumnString::create();
MutableColumnPtr col_table_mut = ColumnString::create();
@@ -101,10 +101,9 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con
String database = (*col_database_to_filter)[i].safeGet();
String table = (*col_table_to_filter)[i].safeGet();
- auto & rocksdb_table = dynamic_cast(*tables[database][table]);
- auto statistics = rocksdb_table.getRocksDBStatistics();
+ auto statistics = tables[database][table]->getRocksDBStatistics();
if (!statistics)
- throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb statistics is not enabled");
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB statistics are not available");
for (auto [tick, name] : rocksdb::TickersNameMap)
{
diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh
index f9a58818404..f01b3010d9c 100755
--- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh
+++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh
@@ -55,7 +55,7 @@ export -f insert1
export -f insert2
export -f select1
export -f select2
-export -f truncate1
+# export -f truncate1
for _ in {1..5}; do
timeout $TIMEOUT bash -c insert1 &
@@ -64,7 +64,7 @@ done
timeout $TIMEOUT bash -c select1 &
timeout $TIMEOUT bash -c select2 &
-timeout $TIMEOUT bash -c truncate1 &
+# timeout $TIMEOUT bash -c truncate1 &
wait
echo "OK"
diff --git a/tests/queries/0_stateless/02030_rocksdb_race_long.reference b/tests/queries/0_stateless/02030_rocksdb_race_long.reference
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/tests/queries/0_stateless/02030_rocksdb_race_long.sh b/tests/queries/0_stateless/02030_rocksdb_race_long.sh
new file mode 100755
index 00000000000..88c30852c86
--- /dev/null
+++ b/tests/queries/0_stateless/02030_rocksdb_race_long.sh
@@ -0,0 +1,49 @@
+#!/usr/bin/env bash
+# Tags: race
+
+unset CLICKHOUSE_LOG_COMMENT
+
+CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CURDIR"/../shell_config.sh
+
+set -o errexit
+set -o pipefail
+
+echo "
+ DROP TABLE IF EXISTS rocksdb_race;
+ CREATE TABLE rocksdb_race (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key);
+ INSERT INTO rocksdb_race SELECT '1_' || toString(number), number FROM numbers(100000);
+" | $CLICKHOUSE_CLIENT -n
+
+function read_stat_thread()
+{
+ while true; do
+ echo "
+ SELECT * FROM system.rocksdb FORMAT Null;
+ " | $CLICKHOUSE_CLIENT -n
+ done
+}
+
+function truncate_thread()
+{
+ while true; do
+ sleep 3s;
+ echo "
+ TRUNCATE TABLE rocksdb_race;
+ " | $CLICKHOUSE_CLIENT -n
+ done
+}
+
+# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
+export -f read_stat_thread;
+export -f truncate_thread;
+
+TIMEOUT=20
+
+timeout $TIMEOUT bash -c read_stat_thread 2> /dev/null &
+timeout $TIMEOUT bash -c truncate_thread 2> /dev/null &
+
+wait
+
+$CLICKHOUSE_CLIENT -q "DROP TABLE rocksdb_race"
diff --git a/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference
new file mode 100644
index 00000000000..00750edc07d
--- /dev/null
+++ b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference
@@ -0,0 +1 @@
+3
diff --git a/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh
new file mode 100755
index 00000000000..9cab6db6940
--- /dev/null
+++ b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh
@@ -0,0 +1,25 @@
+#!/usr/bin/env bash
+
+CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CURDIR"/../shell_config.sh
+
+query_id="${CLICKHOUSE_DATABASE}_$$"
+benchmark_args=(
+ --iterations 1
+ --log_queries 1
+ --query_id "$query_id"
+ --log_queries_min_type QUERY_FINISH
+)
+$CLICKHOUSE_BENCHMARK "${benchmark_args[@]}" --query "SELECT * FROM remote('127.2', 'system.one')" >& /dev/null
+$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
+# Number of queries:
+# - DESC TABLE system.one
+# - query on initiator
+# - query on shard
+# Total: 3
+#
+# -- NOTE: this test cannot use 'current_database = $CLICKHOUSE_DATABASE',
+# -- because it does not propagated via remote queries,
+# -- but it uses query_id, and this is enough.
+$CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "SELECT count() FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id = {query_id:String}"
diff --git a/tests/queries/0_stateless/2021_h3_res_is_classIII.reference b/tests/queries/0_stateless/2021_h3_is_res_classIII.reference
similarity index 100%
rename from tests/queries/0_stateless/2021_h3_res_is_classIII.reference
rename to tests/queries/0_stateless/2021_h3_is_res_classIII.reference
diff --git a/tests/queries/0_stateless/2021_h3_res_is_classIII.sql b/tests/queries/0_stateless/2021_h3_is_res_classIII.sql
similarity index 88%
rename from tests/queries/0_stateless/2021_h3_res_is_classIII.sql
rename to tests/queries/0_stateless/2021_h3_is_res_classIII.sql
index 67dae6d1fa2..7b9cc7acb48 100644
--- a/tests/queries/0_stateless/2021_h3_res_is_classIII.sql
+++ b/tests/queries/0_stateless/2021_h3_is_res_classIII.sql
@@ -11,6 +11,6 @@ INSERT INTO h3_indexes VALUES (geoToH3(10.0, 0.0, 1));
INSERT INTO h3_indexes VALUES (geoToH3(0.0, 10.0, 2));
INSERT INTO h3_indexes VALUES (geoToH3(10.0, 10.0, 3));
-SELECT h3ResIsClassIII(h3_index) FROM h3_indexes ORDER BY h3_index;
+SELECT h3IsResClassIII(h3_index) FROM h3_indexes ORDER BY h3_index;
DROP TABLE h3_indexes;
diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp
index 11db6fc61bc..842e6b09775 100644
--- a/utils/keeper-data-dumper/main.cpp
+++ b/utils/keeper-data-dumper/main.cpp
@@ -68,7 +68,7 @@ int main(int argc, char *argv[])
LOG_INFO(logger, "Last committed index: {}", last_commited_index);
- DB::KeeperLogStore changelog(argv[2], 10000000, true);
+ DB::KeeperLogStore changelog(argv[2], 10000000, true, settings->compress_logs);
changelog.init(last_commited_index, 10000000000UL); /// collect all logs
if (changelog.size() == 0)
LOG_INFO(logger, "Changelog empty");