From 37ba8004ff1ece618aee91d446a4c7cc23ccfdd7 Mon Sep 17 00:00:00 2001 From: liyang Date: Wed, 8 Dec 2021 02:40:59 +0000 Subject: [PATCH 001/103] Speep up mergetree starting up process --- programs/server/Server.cpp | 9 + programs/server/config.xml | 3 + src/CMakeLists.txt | 3 + src/Common/CurrentMetrics.cpp | 1 + src/Common/ProfileEvents.cpp | 8 + src/Functions/checkPartMetaCache.cpp | 157 ++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + src/Interpreters/Context.cpp | 92 +++++ src/Interpreters/Context.h | 29 +- src/Processors/CMakeLists.txt | 4 +- src/Processors/examples/CMakeLists.txt | 2 + .../examples/merge_tree_meta_cache.cpp | 51 +++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 336 ++++++++++++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 58 ++- src/Storages/MergeTree/KeyCondition.cpp | 2 +- src/Storages/MergeTree/KeyCondition.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 8 + src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeDataPartCompact.cpp | 13 + .../MergeTree/MergeTreeDataPartCompact.h | 4 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 11 + .../MergeTree/MergeTreeDataPartInMemory.h | 4 + .../MergeTree/MergeTreeDataPartWide.cpp | 15 +- .../MergeTree/MergeTreeDataPartWide.h | 5 + src/Storages/MergeTree/MergeTreePartition.cpp | 22 +- src/Storages/MergeTree/MergeTreePartition.h | 5 +- src/Storages/MergeTree/PartMetaCache.cpp | 134 +++++++ src/Storages/MergeTree/PartMetaCache.h | 45 +++ .../StorageSystemMergeTreeMetaCache.cpp | 139 ++++++++ .../System/StorageSystemMergeTreeMetaCache.h | 29 ++ src/Storages/System/attachSystemTables.cpp | 2 + .../01233_check_part_meta_cache.reference | 28 ++ .../01233_check_part_meta_cache.sql | 123 +++++++ ..._check_part_meta_cache_in_atomic.reference | 28 ++ .../01233_check_part_meta_cache_in_atomic.sql | 124 +++++++ ...check_part_meta_cache_replicated.reference | 28 ++ ...01233_check_part_meta_cache_replicated.sql | 125 +++++++ ..._meta_cache_replicated_in_atomic.reference | 28 ++ ...k_part_meta_cache_replicated_in_atomic.sql | 125 +++++++ 39 files changed, 1767 insertions(+), 39 deletions(-) create mode 100644 src/Functions/checkPartMetaCache.cpp create mode 100644 src/Processors/examples/merge_tree_meta_cache.cpp create mode 100644 src/Storages/MergeTree/PartMetaCache.cpp create mode 100644 src/Storages/MergeTree/PartMetaCache.h create mode 100644 src/Storages/System/StorageSystemMergeTreeMetaCache.cpp create mode 100644 src/Storages/System/StorageSystemMergeTreeMetaCache.h create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.sql create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference create mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 14075f9fbf2..fd2dc851ae7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -747,6 +747,15 @@ if (ThreadFuzzer::instance().isEffective()) /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(path / "metadata_dropped/"); + + fs::create_directories(path / "rocksdb/"); + } + + + /// initialize meta file cache + { + size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); + global_context->initializeMergeTreeMetaCache(path_str + "/" + "rocksdb", size); } if (config().has("interserver_http_port") && config().has("interserver_https_port")) diff --git a/programs/server/config.xml b/programs/server/config.xml index d88773a3fc4..470c4dfa35f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1291,4 +1291,7 @@ --> + + + 268435456 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7124961821e..42cd4f65f60 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -500,6 +500,9 @@ endif () if (USE_ROCKSDB) dbms_target_link_libraries(PUBLIC ${ROCKSDB_LIBRARY}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) + + target_link_libraries (clickhouse_common_io PUBLIC ${ROCKSDB_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) endif() if (USE_LIBPQXX) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 5c9ba177b78..d214952deae 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -78,6 +78,7 @@ M(SyncDrainedConnections, "Number of connections drained synchronously.") \ M(ActiveSyncDrainedConnections, "Number of active connections drained synchronously.") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ + M(ServerStartupSeconds, "Server start seconds") \ namespace CurrentMetrics { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 982523a3ef2..8b0144be842 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -274,6 +274,14 @@ M(ThreadPoolReaderPageCacheMissElapsedMicroseconds, "Time spent reading data inside the asynchronous job in ThreadPoolReader - when read was not done from page cache.") \ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ + \ + M(RocksdbGet, "Number of rocksdb reads(used for file meta cache)") \ + M(RocksdbPut, "Number of rocksdb puts(used for file meta cache)") \ + M(RocksdbDelete, "Number of rocksdb deletes(used for file meta cache)") \ + M(RocksdbSeek, "Number of rocksdb seeks(used for file meta cache)") \ + M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \ + M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \ + \ namespace ProfileEvents diff --git a/src/Functions/checkPartMetaCache.cpp b/src/Functions/checkPartMetaCache.cpp new file mode 100644 index 00000000000..e0479a5fdcc --- /dev/null +++ b/src/Functions/checkPartMetaCache.cpp @@ -0,0 +1,157 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + +class FunctionCheckPartMetaCache: public IFunction, WithContext +{ +public: + using uint128 = IMergeTreeDataPart::uint128; + using DataPartPtr = MergeTreeData::DataPartPtr; + using DataPartState = MergeTreeData::DataPartState; + using DataPartStates = MergeTreeData::DataPartStates; + + + static constexpr auto name = "checkPartMetaCache"; + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + static constexpr DataPartStates part_states = { + DataPartState::Committed, + DataPartState::Temporary, + DataPartState::PreCommitted, + DataPartState::Outdated, + DataPartState::Deleting, + DataPartState::DeleteOnDestroy + }; + + explicit FunctionCheckPartMetaCache(ContextPtr context_): WithContext(context_) {} + + String getName() const override { return name; } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + bool isDeterministicInScopeOfQuery() const override { return false; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + for (const auto & argument : arguments) + { + if (!isString(argument)) + throw Exception("The argument of function " + getName() + " must have String type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + DataTypePtr key_type = std::make_unique(); + DataTypePtr state_type = std::make_unique(); + DataTypePtr cache_checksum_type = std::make_unique(32); + DataTypePtr disk_checksum_type = std::make_unique(32); + DataTypePtr match_type = std::make_unique(); + DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); + return std::make_shared(tuple_type); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + // get database name + const auto * arg_database = arguments[0].column.get(); + const ColumnString * column_database = checkAndGetColumnConstData(arg_database); + if (! column_database) + throw Exception("The first argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); + String database_name = column_database->getDataAt(0).toString(); + + // get table name + const auto * arg_table = arguments[1].column.get(); + const ColumnString * column_table = checkAndGetColumnConstData(arg_table); + if (! column_table) + throw Exception("The second argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); + String table_name = column_table->getDataAt(0).toString(); + + // get storage + StorageID storage_id(database_name, table_name); + auto storage = DatabaseCatalog::instance().getTable(storage_id, getContext()); + auto data = std::dynamic_pointer_cast(storage); + if (! data) + throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); + + // fill in result + auto col_result = result_type->createColumn(); + auto& col_arr = assert_cast(*col_result); + col_arr.reserve(1); + auto& col_tuple = assert_cast(col_arr.getData()); + col_tuple.reserve(data->fileNumberOfDataParts(part_states)); + auto& col_key = assert_cast(col_tuple.getColumn(0)); + auto& col_state = assert_cast(col_tuple.getColumn(1)); + auto& col_cache_checksum = assert_cast(col_tuple.getColumn(2)); + auto& col_disk_checksum = assert_cast(col_tuple.getColumn(3)); + auto& col_match = assert_cast(col_tuple.getColumn(4)); + auto parts = data->getDataParts(part_states); + for (const auto & part : parts) + executePart(part, col_key, col_state, col_cache_checksum, col_disk_checksum, col_match); + col_arr.getOffsets().push_back(col_tuple.size()); + return result_type->createColumnConst(input_rows_count, col_arr[0]); + } + + static void executePart(const DataPartPtr& part, ColumnString& col_key, ColumnString& col_state, + ColumnFixedString& col_cache_checksum, ColumnFixedString& col_disk_checksum, ColumnUInt8& col_match) + { + Strings keys; + auto state_view = part->stateString(); + String state(state_view.data(), state_view.size()); + std::vector cache_checksums; + std::vector disk_checksums; + uint8_t match = 0; + size_t file_number = part->fileNumberOfColumnsChecksumsIndexes(); + keys.reserve(file_number); + cache_checksums.reserve(file_number); + disk_checksums.reserve(file_number); + + part->checkMetaCache(keys, cache_checksums, disk_checksums); + for (size_t i = 0; i < keys.size(); ++i) + { + col_key.insert(keys[i]); + col_state.insert(state); + col_cache_checksum.insert(getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + col_disk_checksum.insert(getHexUIntUppercase(disk_checksums[i].first) + getHexUIntUppercase(disk_checksums[i].second)); + + match = cache_checksums[i] == disk_checksums[i] ? 1 : 0; + col_match.insertValue(match); + } + } +}; + +void registerFunctionCheckPartMetaCache(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 76d61ce509a..d613d7c85bd 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -80,6 +80,7 @@ void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); +void registerFunctionCheckPartMetaCache(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -166,6 +167,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionServerUUID(factory); registerFunctionZooKeeperSessionUptime(factory); registerFunctionGetOSKernelVersion(factory); + registerFunctionCheckPartMetaCache(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db1d6a37877..81db45c6461 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -94,6 +94,11 @@ namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ContextLock; + extern const Event CompiledCacheSizeBytes; + extern const Event RocksdbPut; + extern const Event RocksdbGet; + extern const Event RocksdbDelete; + extern const Event RocksdbSeek; } namespace CurrentMetrics @@ -126,6 +131,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; + extern const int SYSTEM_ERROR; } @@ -272,6 +278,9 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; + /// MergeTree metadata cache stored in rocksdb. + MergeTreeMetaCachePtr merge_tree_meta_cache; + ContextSharedPart() : access_control(std::make_unique()), macros(std::make_unique()) { @@ -382,6 +391,13 @@ struct ContextSharedPart trace_collector.reset(); /// Stop zookeeper connection zookeeper.reset(); + + /// Shutdown meta file cache + if (merge_tree_meta_cache) + { + merge_tree_meta_cache->shutdown(); + merge_tree_meta_cache.reset(); + } } /// Can be removed w/o context lock @@ -425,6 +441,57 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } +MergeTreeMetaCache::Status MergeTreeMetaCache::put(const String & key, const String & value) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Put(options, key, value); + ProfileEvents::increment(ProfileEvents::RocksdbPut); + return status; +} + +MergeTreeMetaCache::Status MergeTreeMetaCache::del(const String & key) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Delete(options, key); + ProfileEvents::increment(ProfileEvents::RocksdbDelete); + LOG_TRACE(log, "Delete key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + return status; +} + +MergeTreeMetaCache::Status MergeTreeMetaCache::get(const String & key, String & value) +{ + auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); + ProfileEvents::increment(ProfileEvents::RocksdbGet); + LOG_TRACE(log, "Get key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + return status; +} + +void MergeTreeMetaCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) +{ + auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); + rocksdb::Slice target(prefix); + for (it->Seek(target); it->Valid(); it->Next()) + { + const auto key = it->key(); + if (!key.starts_with(target)) + break; + + const auto value = it->value(); + keys.emplace_back(key.data(), key.size()); + values.emplace_back(value.data(), value.size()); + } + LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetaCache items:{}", prefix, keys.size()); + ProfileEvents::increment(ProfileEvents::RocksdbSeek); +} + +void MergeTreeMetaCache::shutdown() +{ + if (rocksdb) + { + rocksdb->Close(); + } +} + ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { auto res = std::shared_ptr(new Context); @@ -2005,6 +2072,11 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const return zookeeper->second; } +MergeTreeMetaCachePtr Context::getMergeTreeMetaCache() const +{ + return shared->merge_tree_meta_cache; +} + void Context::resetZooKeeper() const { std::lock_guard lock(shared->zookeeper_mutex); @@ -2237,6 +2309,26 @@ void Context::initializeTraceCollector() shared->initializeTraceCollector(getTraceLog()); } +void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) +{ + rocksdb::Options options; + rocksdb::BlockBasedTableOptions table_options; + rocksdb::DB * db; + + options.create_if_missing = true; + options.statistics = rocksdb::CreateDBStatistics(); + auto cache = rocksdb::NewLRUCache(size); + table_options.block_cache = cache; + options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); + if (status != rocksdb::Status::OK()) + { + String message = "Fail to open rocksdb path at: " + dir + " status:" + status.ToString(); + throw Exception(message, ErrorCodes::SYSTEM_ERROR); + } + shared->merge_tree_meta_cache = std::make_shared(db); +} + bool Context::hasTraceCollector() const { return shared->hasTraceCollector(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 823bc028c15..b39e06b0b0f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include "config_core.h" @@ -29,7 +31,6 @@ namespace Poco::Net { class IPAddress; } namespace zkutil { class ZooKeeper; } - namespace DB { @@ -178,6 +179,27 @@ private: std::unique_ptr shared; }; +class MergeTreeMetaCache +{ +public: + using Status = rocksdb::Status; + + explicit MergeTreeMetaCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + MergeTreeMetaCache(const MergeTreeMetaCache &) = delete; + MergeTreeMetaCache & operator=(const MergeTreeMetaCache &) = delete; + + Status put(const String & key, const String & value); + Status del(const String & key); + Status get(const String & key, String & value); + void getByPrefix(const String & prefix, Strings & keys, Strings & values); + + void shutdown(); +private: + std::unique_ptr rocksdb; + Poco::Logger * log = &Poco::Logger::get("MergeTreeMetaCache"); +}; +using MergeTreeMetaCachePtr = std::shared_ptr; + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -677,6 +699,9 @@ public: UInt32 getZooKeeperSessionUptime() const; + MergeTreeMetaCachePtr getMergeTreeMetaCache() const; + + #if USE_NURAFT std::shared_ptr & getKeeperDispatcher() const; #endif @@ -763,6 +788,8 @@ public: /// Call after initialization before using trace collector. void initializeTraceCollector(); + void initializeMergeTreeMetaCache(const String & dir, size_t size); + bool hasTraceCollector() const; /// Nullptr if the query log is not ready for this moment. diff --git a/src/Processors/CMakeLists.txt b/src/Processors/CMakeLists.txt index 7e965188b4c..7c9ad405432 100644 --- a/src/Processors/CMakeLists.txt +++ b/src/Processors/CMakeLists.txt @@ -1,4 +1,4 @@ -if (ENABLE_EXAMPLES) +#if (ENABLE_EXAMPLES) add_subdirectory(examples) -endif () +#endif () diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index e69de29bb2d..dcb640c383a 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) +target_link_libraries (merge_tree_meta_cache PRIVATE dbms) diff --git a/src/Processors/examples/merge_tree_meta_cache.cpp b/src/Processors/examples/merge_tree_meta_cache.cpp new file mode 100644 index 00000000000..a394323bcb7 --- /dev/null +++ b/src/Processors/examples/merge_tree_meta_cache.cpp @@ -0,0 +1,51 @@ +#include +#include + +int main() +{ + using namespace DB; + auto shared_context = Context::createShared(); + auto global_context = Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + global_context->initializeMergeTreeMetaCache("./db/", 256 << 20); + + auto cache = global_context->getMergeTreeMetaCache(); + + std::vector files + = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; + String prefix = "data/test_meta_cache/check_part_meta_cache/201806_1_1_0_4/"; + + for (const auto & file : files) + { + auto status = cache->put(prefix + file, prefix + file); + std::cout << "put " << file << " " << status.ToString() << std::endl; + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; + } + + + for (const auto & file : files) + { + auto status = cache->del(prefix + file); + std::cout << "del " << file << " " << status.ToString() << std::endl; + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; + } + + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + for (size_t i=0; i +#include #include #include +#include #include #include #include @@ -61,7 +63,8 @@ static std::unique_ptr openForReading(const DiskPtr & di return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } -void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path) +void IMergeTreeDataPart::MinMaxIndex::load( + const MergeTreeData & data, const PartMetaCachePtr & cache, const DiskPtr & disk, const String & /*part_path*/) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -73,14 +76,15 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); - auto file = openForReading(disk_, file_name); + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + String value; + auto buf = cache->readOrSetMeta(disk, file_name, value); auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; - serialization->deserializeBinary(min_val, *file); + serialization->deserializeBinary(min_val, *buf); Field max_val; - serialization->deserializeBinary(max_val, *file); + serialization->deserializeBinary(max_val, *buf); // NULL_LAST if (min_val.isNull()) @@ -181,6 +185,19 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) } } +void IMergeTreeDataPart::MinMaxIndex::appendFiles(const MergeTreeData & data, Strings & files) +{ + auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + size_t minmax_idx_size = minmax_column_names.size(); + for (size_t i = 0; i < minmax_idx_size; ++i) + { + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + files.push_back(file_name); + } +} + static void incrementStateMetric(IMergeTreeDataPart::State state) { @@ -284,6 +301,9 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) + , meta_cache(std::make_shared( + storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) + { if (parent_part) state = State::Committed; @@ -309,6 +329,9 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) + , meta_cache(std::make_shared( + storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) + { if (parent_part) state = State::Committed; @@ -631,6 +654,41 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadDefaultCompressionCodec(); } +void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection) const +{ + if (isStoredOnDisk()) + { + appendFilesOfUUID(files); + appendFilesOfColumns(files); + appendFilesOfChecksums(files); + appendFilesOfIndexGranularity(files); + appendFilesofIndex(files); + appendFilesOfRowsCount(files); + appendFilesOfPartitionAndMinMaxIndex(files); + appendFilesOfTTLInfos(files); + appendFilesOfDefaultCompressionCodec(files); + } + + if (!parent_part && include_projection) + { + for (const auto & [projection_name, projection_part] : projection_parts) + { + Strings projection_files; + projection_part->appendFilesOfColumnsChecksumsIndexes(projection_files, true); + for (const auto & projection_file : projection_files) + files.push_back(fs::path(projection_part->relative_path) / projection_file); + } + } +} + +size_t IMergeTreeDataPart::fileNumberOfColumnsChecksumsIndexes() const +{ + Strings files; + files.reserve(16); + appendFilesOfColumnsChecksumsIndexes(files, true); + return files.size(); +} + void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); @@ -651,6 +709,11 @@ void IMergeTreeDataPart::loadIndexGranularity() throw Exception("Method 'loadIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); } +void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) const +{ + throw Exception("Method 'appendFilesOfIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); +} + void IMergeTreeDataPart::loadIndex() { /// It can be empty in case of mutations @@ -675,7 +738,8 @@ void IMergeTreeDataPart::loadIndex() } String index_path = fs::path(getFullRelativePath()) / "primary.idx"; - auto index_file = openForReading(volume->getDisk(), index_path); + String value; + auto index_buf = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); size_t marks_count = index_granularity.getMarksCount(); @@ -685,7 +749,7 @@ void IMergeTreeDataPart::loadIndex() for (size_t i = 0; i < marks_count; ++i) //-V756 for (size_t j = 0; j < key_size; ++j) - key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file); + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_buf); for (size_t i = 0; i < key_size; ++i) { @@ -696,13 +760,27 @@ void IMergeTreeDataPart::loadIndex() ErrorCodes::CANNOT_READ_ALL_DATA); } - if (!index_file->eof()) + if (!index_buf->eof()) throw Exception("Index file " + fullPath(volume->getDisk(), index_path) + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); } } +void IMergeTreeDataPart::appendFilesofIndex(Strings & files) const +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + if (parent_part) + metadata_snapshot = metadata_snapshot->projections.has(name) ? metadata_snapshot->projections.get(name).metadata : nullptr; + + if (!metadata_snapshot) + return; + + size_t key_size = metadata_snapshot->getPrimaryKeyColumns().size(); + if (key_size) + files.push_back("primary.idx"); +} + NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const { if (!isStoredOnDisk()) @@ -726,23 +804,30 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() return; } + String v; String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; - if (!volume->getDisk()->exists(path)) + auto in_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + if (!in_buf) { default_codec = detectDefaultCompressionCodec(); } else { - auto file_buf = openForReading(volume->getDisk(), path); String codec_line; - readEscapedStringUntilEOL(codec_line, *file_buf); + readEscapedStringUntilEOL(codec_line, *in_buf); ReadBufferFromString buf(codec_line); if (!checkString("CODEC", buf)) { - LOG_WARNING(storage.log, "Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced automatically, from data on disk", name, path, codec_line); + LOG_WARNING( + storage.log, + "Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced " + "automatically, from data on disk", + name, + path, + codec_line); default_codec = detectDefaultCompressionCodec(); } @@ -760,6 +845,11 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() } } +void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) const +{ + files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME); +} + CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { /// In memory parts doesn't have any compression @@ -822,7 +912,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() { String path = getFullRelativePath(); if (!parent_part) - partition.load(storage, volume->getDisk(), path); + partition.load(storage, meta_cache, volume->getDisk(), path); if (!isEmpty()) { @@ -830,7 +920,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() // projection parts don't have minmax_idx, and it's always initialized minmax_idx->initialized = true; else - minmax_idx->load(storage, volume->getDisk(), path); + minmax_idx->load(storage, meta_cache, volume->getDisk(), path); } if (parent_part) return; @@ -845,13 +935,27 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() ErrorCodes::CORRUPTED_DATA); } +void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) const +{ + if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING && !parent_part) + return; + + if (!parent_part) + partition.appendFiles(storage, files); + + if (!isEmpty()) + if (!parent_part) + minmax_idx->appendFiles(storage, files); +} + void IMergeTreeDataPart::loadChecksums(bool require) { const String path = fs::path(getFullRelativePath()) / "checksums.txt"; - if (volume->getDisk()->exists(path)) + String value; + auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); + if (buf) { - auto buf = openForReading(volume->getDisk(), path); if (checksums.read(*buf)) { assertEOF(*buf); @@ -882,6 +986,11 @@ void IMergeTreeDataPart::loadChecksums(bool require) } } +void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files) const +{ + files.push_back("checksums.txt"); +} + void IMergeTreeDataPart::loadRowsCount() { String path = fs::path(getFullRelativePath()) / "count.txt"; @@ -899,10 +1008,13 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { - if (!volume->getDisk()->exists(path)) + String v; + auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); + if (!buf) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - read_rows_count(); + readIntText(rows_count, *buf); + assertEOF(*buf); #ifndef NDEBUG /// columns have to be loaded @@ -997,12 +1109,18 @@ void IMergeTreeDataPart::loadRowsCount() } } +void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const +{ + files.push_back("count.txt"); +} + void IMergeTreeDataPart::loadTTLInfos() { String path = fs::path(getFullRelativePath()) / "ttl.txt"; - if (volume->getDisk()->exists(path)) + String v; + auto in = meta_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); + if (in) { - auto in = openForReading(volume->getDisk(), path); assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1024,19 +1142,30 @@ void IMergeTreeDataPart::loadTTLInfos() } } + +void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const +{ + files.push_back("ttl.txt"); +} + void IMergeTreeDataPart::loadUUID() { + String v; String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; - - if (volume->getDisk()->exists(path)) + auto in = meta_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); + if (in) { - auto in = openForReading(volume->getDisk(), path); readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); } } +void IMergeTreeDataPart::appendFilesOfUUID(Strings & files) const +{ + files.push_back(UUID_FILE_NAME); +} + void IMergeTreeDataPart::loadColumns(bool require) { String path = fs::path(getFullRelativePath()) / "columns.txt"; @@ -1045,7 +1174,9 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; - if (!volume->getDisk()->exists(path)) + String v; + auto in = meta_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); + if (!in) { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -1061,14 +1192,14 @@ void IMergeTreeDataPart::loadColumns(bool require) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); { - auto buf = volume->getDisk()->writeFile(path + ".tmp", 4096); - loaded_columns.writeText(*buf); + auto out = volume->getDisk()->writeFile(path + ".tmp", 4096); + loaded_columns.writeText(*out); } volume->getDisk()->moveFile(path + ".tmp", path); } else { - loaded_columns.readText(*volume->getDisk()->readFile(path)); + loaded_columns.readText(*in); for (const auto & column : loaded_columns) { @@ -1092,6 +1223,11 @@ void IMergeTreeDataPart::loadColumns(bool require) setColumns(loaded_columns, infos); } +void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) const +{ + files.push_back("columns.txt"); +} + bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const { /// `IMergeTreeDataPart::volume` describes space where current part belongs, and holds @@ -1142,9 +1278,14 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } } + modifyAllMetaCaches(ModifyCacheType::DROP, true); +// #ifndef NDEBUG + assertMetaCacheDropped(true); +// #endif volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; + modifyAllMetaCaches(ModifyCacheType::PUT, true); SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) @@ -1153,6 +1294,71 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } + +void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_projection) const +{ + Strings files; + files.reserve(16); + appendFilesOfColumnsChecksumsIndexes(files, include_projection); + LOG_TRACE( + storage.log, + "part name:{} path:{} {} keys:{}", + name, + getFullRelativePath(), + modifyCacheTypeToString(type), + boost::algorithm::join(files, ", ")); + + switch (type) + { + case ModifyCacheType::PUT: + meta_cache->setMetas(volume->getDisk(), files); + break; + case ModifyCacheType::DROP: + meta_cache->dropMetas(files); + break; + } +} + + +void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const +{ + Strings files; + std::vector _; + meta_cache->getFilesAndCheckSums(files, _); + if (files.empty()) + return; + + for (const auto & file : files) + { + String file_name = fs::path(file).filename(); + /// file belongs to current part + if (fs::path(getFullRelativePath()) / file_name == file) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Data part {} with type {} with meta file {} still in cache", name, getType().toString(), file); + } + + /// file belongs to projection part of current part + if (!parent_part && include_projection) + { + for (const auto & [projection_name, projection_part] : projection_parts) + { + if (fs::path(projection_part->getFullRelativePath()) / file_name == file) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Data part {} with type {} with meta file {} with projection name still in cache", + name, + getType().toString(), + file, + projection_name); + } + } + } + // LOG_WARNING(storage.log, "cache of file {} does't belong to any part", file); + } +} + std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { /// NOTE: It's needed for zero-copy replication @@ -1190,6 +1396,11 @@ void IMergeTreeDataPart::remove() const return; } + modifyAllMetaCaches(ModifyCacheType::DROP); +// #ifndef NDEBUG + assertMetaCacheDropped(); +// #endif + /** Atomic directory removal: * - rename directory to temporary name; * - remove it recursive. @@ -1293,6 +1504,11 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { + modifyAllMetaCaches(ModifyCacheType::DROP); +// #ifndef NDEBUG + assertMetaCacheDropped(); +// #endif + String to = parent_to + "/" + relative_path; auto disk = volume->getDisk(); if (checksums.empty()) @@ -1380,6 +1596,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) return "detached/" + getRelativePathForPrefix(prefix, /* detached */ true); } + void IMergeTreeDataPart::renameToDetached(const String & prefix) const { renameTo(getRelativePathForDetachedPart(prefix), true); @@ -1632,6 +1849,71 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } +IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const +{ + String file_name = std::filesystem::path(file_path).filename(); + const auto filenames_without_checksums = getFileNamesWithoutChecksums(); + auto it = checksums.files.find(file_name); + if (filenames_without_checksums.count(file_name) == 0 && it != checksums.files.end()) + { + return it->second.file_hash; + } + + if (!volume->getDisk()->exists(file_path)) + { + return {}; + } + std::unique_ptr in_file = volume->getDisk()->readFile(file_path); + HashingReadBuffer in_hash(*in_file); + + String value; + readStringUntilEOF(value, in_hash); + return in_hash.getHash(); +} + +void IMergeTreeDataPart::checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const +{ + /// checkMetaCache only applies for normal part + if (isProjectionPart()) + return; + + /// the directory of projection part is under the directory of its parent part + const auto filenames_without_checksums = getFileNamesWithoutChecksums(); + meta_cache->getFilesAndCheckSums(files, cache_checksums); + for (const auto & file : files) + { + // std::cout << "check key:" << file << std::endl; + String file_name = fs::path(file).filename(); + + /// file belongs to normal part + if (fs::path(getFullRelativePath()) / file_name == file) + { + auto disk_checksum = getActualChecksumByFile(file); + disk_checksums.push_back(disk_checksum); + continue; + } + + /// file belongs to projection part + String proj_dir_name = fs::path(file).parent_path().filename(); + auto pos = proj_dir_name.find_last_of('.'); + if (pos == String::npos) + { + disk_checksums.push_back({}); + continue; + } + String proj_name = proj_dir_name.substr(0, pos); + auto it = projection_parts.find(proj_name); + if (it == projection_parts.end()) + { + disk_checksums.push_back({}); + continue; + } + + auto disk_checksum = it->second->getActualChecksumByFile(file); + disk_checksums.push_back(disk_checksum); + } +} + bool isCompactPart(const MergeTreeDataPartPtr & data_part) { return (data_part && data_part->getType() == MergeTreeDataPartType::COMPACT); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ab08ca1c33a..ad072af10a5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -14,6 +14,7 @@ #include #include #include +#include #include @@ -44,6 +45,23 @@ class UncompressedCache; class IMergeTreeDataPart : public std::enable_shared_from_this { public: + enum ModifyCacheType + { + PUT, // override set + DROP, // remove keys + }; + + static String modifyCacheTypeToString(ModifyCacheType type) + { + switch (type) + { + case PUT: + return "PUT"; + case DROP: + return "DROP"; + } + } + static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -59,6 +77,7 @@ public: using IndexSizeByName = std::unordered_map; using Type = MergeTreeDataPartType; + using uint128 = PartMetaCache::uint128; IMergeTreeDataPart( @@ -138,6 +157,8 @@ public: /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; + void assertMetaCacheDropped(bool include_projection = false) const; + void remove() const; void projectionRemove(const String & parent_to, bool keep_shared_data = false) const; @@ -145,6 +166,8 @@ public: /// Initialize columns (from columns.txt if exists, or create from column files if not). /// Load checksums from checksums.txt if exists. Load index if required. void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); + void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const; + size_t fileNumberOfColumnsChecksumsIndexes() const; String getMarksFileExtension() const { return index_granularity_info.marks_file_extension; } @@ -239,7 +262,7 @@ public: using TTLInfo = MergeTreeDataPartTTLInfo; using TTLInfos = MergeTreeDataPartTTLInfos; - TTLInfos ttl_infos; + mutable TTLInfos ttl_infos; /// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex void setState(State new_state) const; @@ -296,12 +319,13 @@ public: { } - void load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path); - void store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; + void load(const MergeTreeData & data, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); + void store(const MergeTreeData & data, const DiskPtr & disk, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; void update(const Block & block, const Names & column_names); void merge(const MinMaxIndex & other); + static void appendFiles(const MergeTreeData & data, Strings & files); }; using MinMaxIndexPtr = std::shared_ptr; @@ -351,6 +375,8 @@ public: /// storage and pass it to this method. virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; } + virtual Strings getIndexGranularityFiles() const = 0; + /// Returns true if this part shall participate in merges according to /// settings of given storage policy. bool shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const; @@ -363,6 +389,8 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; + virtual void checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; + bool isProjectionPart() const { return parent_part != nullptr; } const IMergeTreeDataPart * getParentPart() const { return parent_part; } @@ -434,6 +462,8 @@ protected: std::map> projection_parts; + mutable PartMetaCachePtr meta_cache; + void removeIfNeeded(); virtual void checkConsistency(bool require_part_metadata) const; @@ -457,40 +487,62 @@ private: /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); + void appendFilesOfUUID(Strings & files) const; + /// Reads columns names and types from columns.txt void loadColumns(bool require); + void appendFilesOfColumns(Strings & files) const; + /// If checksums.txt exists, reads file's checksums (and sizes) from it void loadChecksums(bool require); + void appendFilesOfChecksums(Strings & files) const; + /// Loads marks index granularity into memory virtual void loadIndexGranularity(); + virtual void appendFilesOfIndexGranularity(Strings & files) const; + /// Loads index file. void loadIndex(); + void appendFilesofIndex(Strings & files) const; + /// Load rows count for this part from disk (for the newer storage format version). /// For the older format version calculates rows count from the size of a column with a fixed size. void loadRowsCount(); + void appendFilesOfRowsCount(Strings & files) const; + /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); + void appendFilesOfTTLInfos(Strings & files) const; + void loadPartitionAndMinMaxIndex(); void calculateColumnsSizesOnDisk(); void calculateSecondaryIndicesSizesOnDisk(); + void appendFilesOfPartitionAndMinMaxIndex(Strings & files) const; + /// Load default compression codec from file default_compression_codec.txt /// if it not exists tries to deduce codec from compressed column without /// any specifial compression. void loadDefaultCompressionCodec(); + void appendFilesOfDefaultCompressionCodec(Strings & files) const; + + void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; + /// Found column without specific compression and return codec /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; + IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; + mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 3d4e909cf60..10530f25927 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -53,7 +53,7 @@ String Range::toString() const /// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -static String extractFixedPrefixFromLikePattern(const String & like_pattern) +String extractFixedPrefixFromLikePattern(const String & like_pattern) { String fixed_prefix; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index dee46ae52ce..c8d9fda77c4 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -442,4 +442,6 @@ private: bool strict; }; +String extractFixedPrefixFromLikePattern(const String & like_pattern); + } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1b7be8ca98d..8d861e404f0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1330,6 +1330,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); } +size_t MergeTreeData::fileNumberOfDataParts(const DataPartStates & states) const +{ + size_t result = 0; + auto parts = getDataParts(states); + for (const auto & part : parts) + result += part->fileNumberOfColumnsChecksumsIndexes(); + return result; +} /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 380c2f4f4c5..d349646ab88 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -427,6 +427,8 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); + size_t fileNumberOfDataParts(const DataPartStates & states) const; + String getLogName() const { return log_name; } Int64 getMaxBlockNumber() const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index f4da730b1f0..e51b64b3842 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -91,6 +91,7 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac total_size.marks += mrk_checksum->second.file_size; } +// load marks from meta cache void MergeTreeDataPartCompact::loadIndexGranularity() { String full_path = getFullRelativePath(); @@ -192,4 +193,16 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } +// Do not cache mark file, because cache other meta files is enough to speed up loading. +void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const +{ +} + +// find all connected file and do modification +Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const +{ + auto marks_file = index_granularity_info.getMarksFilePath("data"); + return {marks_file}; +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 38bfa11652a..87066ab2ff0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -72,6 +72,10 @@ private: /// Compact parts doesn't support per column size, only total size void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + + void appendFilesOfIndexGranularity(Strings& files) const override; + + Strings getIndexGranularityFiles() const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 4ec53d88339..5c21ead3208 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -166,6 +166,17 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() return checksum; } +// No mark files for part in memory +void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files */) const +{ +} + +// No mark files for part in memory +Strings MergeTreeDataPartInMemory::getIndexGranularityFiles() const +{ + return {}; +} + DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) { return std::dynamic_pointer_cast(part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index c5ee9ebd01f..4f83b54d402 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -62,6 +62,10 @@ private: /// Calculates uncompressed sizes in memory. void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + + void appendFilesOfIndexGranularity(Strings & files) const override; + + Strings getIndexGranularityFiles() const override; }; using DataPartInMemoryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index b279c1aba6a..5132177aa5c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -107,7 +107,6 @@ void MergeTreeDataPartWide::loadIndexGranularity() String full_path = getFullRelativePath(); index_granularity_info.changeGranularityIfRequired(volume->getDisk(), full_path); - if (columns.empty()) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -268,4 +267,18 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col } } +// Do not cache mark files of part, because cache other meta files is enough to speed up loading. +void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) const +{ +} + +Strings MergeTreeDataPartWide::getIndexGranularityFiles() const +{ + if (columns.empty()) + return {}; + + auto marks_file = getFileNameForColumn(columns.front()); + return {marks_file}; +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 4796143e11d..bf73d16d758 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -66,9 +66,14 @@ private: /// Loads marks index granularity into memory void loadIndexGranularity() override; + void appendFilesOfIndexGranularity(Strings & files) const override; + + Strings getIndexGranularityFiles() const override; + ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set * processed_substreams) const; void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + }; } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 06fcb24f846..1d4e14c628b 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,11 +160,13 @@ namespace }; } +/* static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) { size_t file_size = disk->getFileSize(path); return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } +*/ String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -355,7 +357,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } } -void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) +void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) @@ -363,10 +365,12 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; auto partition_file_path = part_path + "partition.dat"; - auto file = openForReading(disk, partition_file_path); + + String v; + auto buf = meta_cache->readOrSetMeta(disk, "partition.dat", v); value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) - partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file); + partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *buf); } void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const @@ -384,7 +388,9 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr auto out = disk->writeFile(part_path + "partition.dat"); HashingWriteBuffer out_hashing(*out); for (size_t i = 0; i < value.size(); ++i) + { partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing); + } out_hashing.next(); checksums.files["partition.dat"].file_size = out_hashing.count(); @@ -443,4 +449,14 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & return partition_key; } + +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) const +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + if (!metadata_snapshot->hasPartitionKey()) + return; + + files.push_back("partition.dat"); +} + } diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index d501d615621..b8b5b301219 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -37,7 +38,7 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; - void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); + void load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; @@ -45,6 +46,8 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + void appendFiles(const MergeTreeData & storage, Strings & files) const; + /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); diff --git a/src/Storages/MergeTree/PartMetaCache.cpp b/src/Storages/MergeTree/PartMetaCache.cpp new file mode 100644 index 00000000000..33c95d6963e --- /dev/null +++ b/src/Storages/MergeTree/PartMetaCache.cpp @@ -0,0 +1,134 @@ +#include "PartMetaCache.h" + +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event MergeTreeMetaCacheHit; + extern const Event MergeTreeMetaCacheMiss; +} + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ + +std::unique_ptr +PartMetaCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) +{ + String file_path = fs::path(getFullRelativePath()) / file_name; + auto status = cache->get(file_path, value); + if (!status.ok()) + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheMiss); + if (!disk->exists(file_path)) + { + return nullptr; + } + + auto in = disk->readFile(file_path); + if (in) + { + readStringUntilEOF(value, *in); + cache->put(file_path, value); + } + } + else + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheHit); + } + return std::make_unique(value); +} + +void PartMetaCache::setMetas(const DiskPtr & disk, const Strings & file_names) +{ + String text; + String read_value; + for (const auto & file_name : file_names) + { + const String file_path = fs::path(getFullRelativePath()) / file_name; + if (!disk->exists(file_path)) + continue; + + auto in = disk->readFile(file_path); + if (!in) + continue; + + readStringUntilEOF(text, *in); + auto status = cache->put(file_path, text); + if (!status.ok()) + { + status = cache->get(file_path, read_value); + if (status.IsNotFound() || read_value == text) + continue; + throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); + } + } +} + +void PartMetaCache::dropMetas(const Strings & file_names) +{ + for (const auto & file_name : file_names) + { + String file_path = fs::path(getFullRelativePath()) / file_name; + auto status = cache->del(file_path); + if (!status.ok()) + { + String read_value; + status = cache->get(file_path, read_value); + if (status.IsNotFound()) + continue; + throw Exception(ErrorCodes::LOGICAL_ERROR, "drop meta failed status:{}, file_path:{}", status.ToString(), file_path); + } + } +} + +void PartMetaCache::setMeta(const String & file_name, const String & value) +{ + String file_path = fs::path(getFullRelativePath()) / file_name; + String read_value; + auto status = cache->get(file_path, read_value); + if (status == rocksdb::Status::OK() && value == read_value) + return; + + status = cache->put(file_path, value); + if (!status.ok()) + { + status = cache->get(file_path, read_value); + if (status.IsNotFound() || read_value == value) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); + } +} + +void PartMetaCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const +{ + String prefix = fs::path(getFullRelativePath()) / ""; + Strings values; + values.reserve(files.capacity()); + cache->getByPrefix(prefix, files, values); + size_t size = files.size(); + for (size_t i = 0; i < size; ++i) + { + ReadBufferFromString rbuf(values[i]); + HashingReadBuffer hbuf(rbuf); + checksums.push_back(hbuf.getHash()); + } +} + +String PartMetaCache::getFullRelativePath() const +{ + return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; +} + +} diff --git a/src/Storages/MergeTree/PartMetaCache.h b/src/Storages/MergeTree/PartMetaCache.h new file mode 100644 index 00000000000..d6a86d86ba1 --- /dev/null +++ b/src/Storages/MergeTree/PartMetaCache.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class SeekableReadBuffer; +class IMergeTreeDataPart; +class PartMetaCache; +using PartMetaCachePtr = std::shared_ptr; + +class PartMetaCache +{ +public: + using uint128 = CityHash_v1_0_2::uint128; + + PartMetaCache(const MergeTreeMetaCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) + : cache(cache_) + , relative_data_path(relative_data_path_) + , relative_path(relative_path_) + , parent_part(parent_part_) + { + } + + std::unique_ptr + readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value); + void setMetas(const DiskPtr & disk, const Strings & file_names); + void dropMetas(const Strings & file_names); + void setMeta(const String & file_name, const String & value); + void getFilesAndCheckSums(Strings & file_names, std::vector & checksums) const; + +private: + std::string getFullRelativePath() const; + + MergeTreeMetaCachePtr cache; + const String & relative_data_path; // relative path of table to disk + const String & relative_path; // relative path of part to table + const IMergeTreeDataPart * parent_part; +}; + +} diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp new file mode 100644 index 00000000000..45cf45edb31 --- /dev/null +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp @@ -0,0 +1,139 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +NamesAndTypesList StorageSystemMergeTreeMetaCache::getNamesAndTypes() +{ + return { + {"key", std::make_shared()}, + {"value", std::make_shared()}, + }; +} + +static bool extractKeyImpl(const IAST & elem, String & res, bool & precise) +{ + const auto * function = elem.as(); + if (!function) + return false; + + if (function->name == "and") + { + for (const auto & child : function->arguments->children) + { + bool tmp_precise = false; + if (extractKeyImpl(*child, res, tmp_precise)) + { + precise = tmp_precise; + return true; + } + } + return false; + } + + if (function->name == "equals" || function->name == "like") + { + const auto & args = function->arguments->as(); + const IAST * value; + + if (args.children.size() != 2) + return false; + + const ASTIdentifier * ident; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1).get(); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0).get(); + else + return false; + + if (ident->name() != "key") + return false; + + const auto * literal = value->as(); + if (!literal) + return false; + + if (literal->value.getType() != Field::Types::String) + return false; + + res = literal->value.safeGet(); + precise = function->name == "equals"; + return true; + } + return false; +} + + +/** Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause. + */ +static String extractKey(const ASTPtr & query, bool& precise) +{ + const auto & select = query->as(); + if (!select.where()) + return ""; + + String res; + return extractKeyImpl(*select.where(), res, precise) ? res : ""; +} + + +void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +{ + bool precise = false; + String key = extractKey(query_info.query, precise); + if (key.empty()) + throw Exception( + "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + + auto cache = context->getMergeTreeMetaCache(); + if (precise) + { + String value; + if (cache->get(key, value) != MergeTreeMetaCache::Status::OK()) + return; + + size_t col_num = 0; + res_columns[col_num++]->insert(key); + res_columns[col_num++]->insert(value); + } + else + { + String target = extractFixedPrefixFromLikePattern(key); + if (target.empty()) + throw Exception( + "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + + Strings keys; + Strings values; + keys.reserve(4096); + values.reserve(4096); + cache->getByPrefix(target, keys, values); + if (keys.empty()) + return; + + assert(keys.size() == values.size()); + for (size_t i = 0; i < keys.size(); ++i) + { + size_t col_num = 0; + res_columns[col_num++]->insert(keys[i]); + res_columns[col_num++]->insert(values[i]); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.h b/src/Storages/System/StorageSystemMergeTreeMetaCache.h new file mode 100644 index 00000000000..a5f65862243 --- /dev/null +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class Context; + + +/** Implements `merge_tree_meta_cache` system table, which allows you to view the metacache data in rocksdb for debugging purposes. + */ +class StorageSystemMergeTreeMetaCache : public shared_ptr_helper, public IStorageSystemOneBlock +{ + friend struct shared_ptr_helper; + +public: + std::string getName() const override { return "SystemMergeTreeMetaCache"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 023ced35a6b..0e7d4b624c5 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -129,6 +130,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) #endif #if USE_ROCKSDB attach(context, system_database, "rocksdb"); + attach(context, system_database, "merge_tree_meta_cache"); #endif } diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.reference b/tests/queries/0_stateless/01233_check_part_meta_cache.reference new file mode 100644 index 00000000000..914add905ce --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +63 0 +77 0 +84 0 +98 0 +122 0 +154 0 +122 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql new file mode 100644 index 00000000000..f0ca3b608d1 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -0,0 +1,123 @@ +-- Create table under database with engine ordinary. +set mutations_sync = 1; +DROP DATABASE IF EXISTS test_meta_cache; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; +CREATE DATABASE test_meta_cache ENGINE = Ordinary; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert third batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert third batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference new file mode 100644 index 00000000000..95de1ef56a9 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +63 0 +77 0 +84 0 +98 0 +124 0 +150 0 +124 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql new file mode 100644 index 00000000000..b57caf55cb8 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -0,0 +1,124 @@ +-- Create table under database with engine atomic. +set mutations_sync = 1; +DROP DATABASE IF EXISTS test_meta_cache; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache SYNC; +CREATE DATABASE test_meta_cache ENGINE = Atomic; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + 30; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + 60; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache SYNC; +CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- nsert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference new file mode 100644 index 00000000000..2275537d212 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +7 0 +14 0 +21 0 +35 0 +51 0 +67 0 +0 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql new file mode 100644 index 00000000000..6d08bb146a5 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -0,0 +1,125 @@ +-- Create table under database with engine ordinary. +set mutations_sync = 1; +set replication_alter_partitions_sync = 2; +DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Ordinary; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference new file mode 100644 index 00000000000..2275537d212 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference @@ -0,0 +1,28 @@ +0 0 +7 0 +14 0 +28 0 +42 0 +56 0 +70 0 +77 0 +7 0 +14 0 +21 0 +35 0 +51 0 +67 0 +0 0 +12 0 +24 0 +48 0 +72 0 +96 0 +120 0 +132 0 +108 0 +132 0 +144 0 +183 0 +235 0 +183 0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql new file mode 100644 index 00000000000..c41d036cef1 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -0,0 +1,125 @@ +-- Create table under database with engine ordinary. +set mutations_sync = 1; +set replication_alter_partitions_sync = 2; +DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Atomic; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Recreate table with projection. +drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; + +-- Insert first batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Insert second batch of data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Update some data. +alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +--Delete some data. +alter table test_meta_cache.check_part_meta_cache delete where k = 1; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +alter table test_meta_cache.check_part_meta_cache delete where k = 8; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Delete some data. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Drop partitioin 201805 +alter table test_meta_cache.check_part_meta_cache drop partition 201805; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Optimize table. +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_meta_cache.check_part_meta_cache FINAL; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add column. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Add TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Modify TTL info. +alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + +-- Truncate table. +truncate table test_meta_cache.check_part_meta_cache; +with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); + From e7401d2a5ee3a5a765e50af64af1375c2af15344 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 27 Dec 2021 11:50:59 +0800 Subject: [PATCH 002/103] wrap rocksdb metacache related code with USE_ROCKSDB --- programs/server/Server.cpp | 2 ++ src/Common/CurrentMetrics.cpp | 1 - src/Common/ProfileEvents.cpp | 8 ++++---- src/Functions/checkPartMetaCache.cpp | 9 +++++---- src/Functions/registerFunctionsMiscellaneous.cpp | 6 ++++++ src/Interpreters/Context.cpp | 14 ++++++++++++-- src/Interpreters/Context.h | 10 ++++++++++ src/Processors/CMakeLists.txt | 4 ++-- src/Processors/examples/CMakeLists.txt | 6 ++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 1 - src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 1 + src/Storages/MergeTree/MergeTreePartition.cpp | 7 ------- src/Storages/MergeTree/PartMetaCache.cpp | 2 ++ src/Storages/MergeTree/PartMetaCache.h | 4 ++++ .../System/StorageSystemMergeTreeMetaCache.cpp | 6 +++++- .../System/StorageSystemMergeTreeMetaCache.h | 4 ++++ src/Storages/System/attachSystemTables.cpp | 2 +- .../01233_check_part_meta_cache_replicated.sql | 2 +- ..._check_part_meta_cache_replicated_in_atomic.sql | 2 +- 20 files changed, 65 insertions(+), 28 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fd2dc851ae7..7228608c9f1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -748,7 +748,9 @@ if (ThreadFuzzer::instance().isEffective()) /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(path / "metadata_dropped/"); +#if USE_ROCKSDB fs::create_directories(path / "rocksdb/"); +#endif } diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index d214952deae..5c9ba177b78 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -78,7 +78,6 @@ M(SyncDrainedConnections, "Number of connections drained synchronously.") \ M(ActiveSyncDrainedConnections, "Number of active connections drained synchronously.") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ - M(ServerStartupSeconds, "Server start seconds") \ namespace CurrentMetrics { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index af87ba10a31..cb9c6f594a6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -277,10 +277,10 @@ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ \ - M(RocksdbGet, "Number of rocksdb reads(used for file meta cache)") \ - M(RocksdbPut, "Number of rocksdb puts(used for file meta cache)") \ - M(RocksdbDelete, "Number of rocksdb deletes(used for file meta cache)") \ - M(RocksdbSeek, "Number of rocksdb seeks(used for file meta cache)") \ + M(RocksdbGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ + M(RocksdbPut, "Number of rocksdb puts(used for merge tree metadata cache)") \ + M(RocksdbDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ + M(RocksdbSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \ M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \ \ diff --git a/src/Functions/checkPartMetaCache.cpp b/src/Functions/checkPartMetaCache.cpp index e0479a5fdcc..e77b8ca0d50 100644 --- a/src/Functions/checkPartMetaCache.cpp +++ b/src/Functions/checkPartMetaCache.cpp @@ -1,6 +1,8 @@ +#include "config_core.h" + +#if USE_ROCKSDB + #include -#include -#include #include #include #include @@ -14,8 +16,6 @@ #include #include #include -#include -#include #include #include @@ -155,3 +155,4 @@ void registerFunctionCheckPartMetaCache(FunctionFactory & factory) } } +#endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index d613d7c85bd..77e3e109081 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -80,7 +80,10 @@ void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); + +#if USE_ROCKSDB void registerFunctionCheckPartMetaCache(FunctionFactory &); +#endif #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -167,7 +170,10 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionServerUUID(factory); registerFunctionZooKeeperSessionUptime(factory); registerFunctionGetOSKernelVersion(factory); + +#if USE_ROCKSDB registerFunctionCheckPartMetaCache(factory); +#endif #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 81db45c6461..c5a5b3d1d49 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -94,11 +94,13 @@ namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ContextLock; - extern const Event CompiledCacheSizeBytes; + +#if USE_ROCKSDB extern const Event RocksdbPut; extern const Event RocksdbGet; extern const Event RocksdbDelete; extern const Event RocksdbSeek; +#endif } namespace CurrentMetrics @@ -278,8 +280,10 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; +#if USE_ROCKSDB /// MergeTree metadata cache stored in rocksdb. MergeTreeMetaCachePtr merge_tree_meta_cache; +#endif ContextSharedPart() : access_control(std::make_unique()), macros(std::make_unique()) @@ -392,12 +396,14 @@ struct ContextSharedPart /// Stop zookeeper connection zookeeper.reset(); - /// Shutdown meta file cache +#if USE_ROCKSDB + /// Shutdown meta file cache if (merge_tree_meta_cache) { merge_tree_meta_cache->shutdown(); merge_tree_meta_cache.reset(); } +#endif } /// Can be removed w/o context lock @@ -441,6 +447,7 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } +#if USE_ROCKSDB MergeTreeMetaCache::Status MergeTreeMetaCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); @@ -491,6 +498,7 @@ void MergeTreeMetaCache::shutdown() rocksdb->Close(); } } +#endif ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { @@ -2309,6 +2317,7 @@ void Context::initializeTraceCollector() shared->initializeTraceCollector(getTraceLog()); } +#if USE_ROCKSDB void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) { rocksdb::Options options; @@ -2328,6 +2337,7 @@ void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) } shared->merge_tree_meta_cache = std::make_shared(db); } +#endif bool Context::hasTraceCollector() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b39e06b0b0f..bc191e80c9a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,6 @@ #pragma once +#include "config_core.h" #include #include #include @@ -15,8 +16,11 @@ #include #include #include + +#if USE_ROCKSDB #include #include +#endif #include "config_core.h" @@ -179,6 +183,7 @@ private: std::unique_ptr shared; }; +#if USE_ROCKSDB class MergeTreeMetaCache { public: @@ -199,6 +204,7 @@ private: Poco::Logger * log = &Poco::Logger::get("MergeTreeMetaCache"); }; using MergeTreeMetaCachePtr = std::shared_ptr; +#endif /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) @@ -699,7 +705,9 @@ public: UInt32 getZooKeeperSessionUptime() const; +#if USE_ROCKSDB MergeTreeMetaCachePtr getMergeTreeMetaCache() const; +#endif #if USE_NURAFT @@ -788,7 +796,9 @@ public: /// Call after initialization before using trace collector. void initializeTraceCollector(); +#if USE_ROCKSDB void initializeMergeTreeMetaCache(const String & dir, size_t size); +#endif bool hasTraceCollector() const; diff --git a/src/Processors/CMakeLists.txt b/src/Processors/CMakeLists.txt index 7c9ad405432..7e965188b4c 100644 --- a/src/Processors/CMakeLists.txt +++ b/src/Processors/CMakeLists.txt @@ -1,4 +1,4 @@ -#if (ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory(examples) -#endif () +endif () diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index dcb640c383a..ceb022432a1 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,2 +1,4 @@ -add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) -target_link_libraries (merge_tree_meta_cache PRIVATE dbms) +if (USE_ROCKSDB) + add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) + target_link_libraries (merge_tree_meta_cache PRIVATE dbms) +endif() diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ad072af10a5..0d2b2f57fd0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -50,7 +50,7 @@ public: PUT, // override set DROP, // remove keys }; - + static String modifyCacheTypeToString(ModifyCacheType type) { switch (type) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index e51b64b3842..e5fbdd5cd19 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -91,7 +91,6 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac total_size.marks += mrk_checksum->second.file_size; } -// load marks from meta cache void MergeTreeDataPartCompact::loadIndexGranularity() { String full_path = getFullRelativePath(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 5132177aa5c..f6efdc5f05c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -107,6 +107,7 @@ void MergeTreeDataPartWide::loadIndexGranularity() String full_path = getFullRelativePath(); index_granularity_info.changeGranularityIfRequired(volume->getDisk(), full_path); + if (columns.empty()) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 1d4e14c628b..6b5bebd81f6 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,13 +160,6 @@ namespace }; } -/* -static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) -{ - size_t file_size = disk->getFileSize(path); - return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); -} -*/ String MergeTreePartition::getID(const MergeTreeData & storage) const { diff --git a/src/Storages/MergeTree/PartMetaCache.cpp b/src/Storages/MergeTree/PartMetaCache.cpp index 33c95d6963e..fe8475d0dda 100644 --- a/src/Storages/MergeTree/PartMetaCache.cpp +++ b/src/Storages/MergeTree/PartMetaCache.cpp @@ -1,5 +1,6 @@ #include "PartMetaCache.h" +#if USE_ROCKSDB #include #include #include @@ -132,3 +133,4 @@ String PartMetaCache::getFullRelativePath() const } } +#endif diff --git a/src/Storages/MergeTree/PartMetaCache.h b/src/Storages/MergeTree/PartMetaCache.h index d6a86d86ba1..5ffd0413c4b 100644 --- a/src/Storages/MergeTree/PartMetaCache.h +++ b/src/Storages/MergeTree/PartMetaCache.h @@ -1,5 +1,8 @@ #pragma once +#include "config_core.h" + +#if USE_ROCKSDB #include #include #include @@ -43,3 +46,4 @@ private: }; } +#endif diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp index 45cf45edb31..f53c32e5a42 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp @@ -1,3 +1,6 @@ +#include + +#if USE_ROCKSDB #include #include #include @@ -7,9 +10,9 @@ #include #include #include -#include #include #include + namespace DB { namespace ErrorCodes @@ -137,3 +140,4 @@ void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, Con } } +#endif diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.h b/src/Storages/System/StorageSystemMergeTreeMetaCache.h index a5f65862243..c8e0f475cd8 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetaCache.h @@ -1,5 +1,8 @@ #pragma once +#include "config_core.h" + +#if USE_ROCKSDB #include #include @@ -27,3 +30,4 @@ protected: }; } +#endif diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 0e7d4b624c5..5f5a17069f3 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -68,7 +68,6 @@ #include #include #include -#include #ifdef OS_LINUX #include @@ -76,6 +75,7 @@ #if USE_ROCKSDB #include +#include #endif diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 6d08bb146a5..cb028e77d54 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -4,7 +4,7 @@ set replication_alter_partitions_sync = 2; DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index c41d036cef1..c56e2cb0a99 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -4,7 +4,7 @@ set replication_alter_partitions_sync = 2; DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/test_meta_cache/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. From 30ffa57bd511f18252bea31e69b5cb6a08c134e0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 10:17:16 +0800 Subject: [PATCH 003/103] remove unused log --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 80d9d869772..5bc125c6787 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1355,7 +1355,6 @@ void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const } } } - // LOG_WARNING(storage.log, "cache of file {} does't belong to any part", file); } } From 7dab7caa9d44e21a5747c997d95252059f75a665 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 11:57:43 +0800 Subject: [PATCH 004/103] wrap cache related code with USE_ROCKSDB --- programs/server/Server.cpp | 2 + src/Interpreters/Context.cpp | 2 + src/Processors/examples/CMakeLists.txt | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 126 ++++++++++++++---- src/Storages/MergeTree/IMergeTreeDataPart.h | 22 ++- .../MergeTree/MergeTreeDataPartCompact.cpp | 4 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- src/Storages/MergeTree/MergeTreePartition.cpp | 20 ++- src/Storages/MergeTree/MergeTreePartition.h | 10 +- 9 files changed, 156 insertions(+), 36 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7228608c9f1..45f7834c96c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -754,11 +754,13 @@ if (ThreadFuzzer::instance().isEffective()) } +#if USE_ROCKSDB /// initialize meta file cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); global_context->initializeMergeTreeMetaCache(path_str + "/" + "rocksdb", size); } +#endif if (config().has("interserver_http_port") && config().has("interserver_https_port")) throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c5a5b3d1d49..8b8c8f982fd 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2080,10 +2080,12 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const return zookeeper->second; } +#if USE_ROCKSDB MergeTreeMetaCachePtr Context::getMergeTreeMetaCache() const { return shared->merge_tree_meta_cache; } +#endif void Context::resetZooKeeper() const { diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index ceb022432a1..a07224d4462 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,4 +1,4 @@ if (USE_ROCKSDB) add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) target_link_libraries (merge_tree_meta_cache PRIVATE dbms) -endif() +endif() \ No newline at end of file diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5bc125c6787..0b5f9f19782 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -63,8 +63,13 @@ static std::unique_ptr openForReading(const DiskPtr & di return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } +#if USE_ROCKSDB void IMergeTreeDataPart::MinMaxIndex::load( const MergeTreeData & data, const PartMetaCachePtr & cache, const DiskPtr & disk, const String & /*part_path*/) +#else +void IMergeTreeDataPart::MinMaxIndex::load( + const MergeTreeData & data, const DiskPtr & disk, const String & part_path) +#endif { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -76,15 +81,20 @@ void IMergeTreeDataPart::MinMaxIndex::load( hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { +#if USE_ROCKSDB String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; String value; - auto buf = cache->readOrSetMeta(disk, file_name, value); + auto file = cache->readOrSetMeta(disk, file_name, value); +#else + String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); + auto file = openForReading(disk, file_name); +#endif auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; - serialization->deserializeBinary(min_val, *buf); + serialization->deserializeBinary(min_val, *file); Field max_val; - serialization->deserializeBinary(max_val, *buf); + serialization->deserializeBinary(max_val, *file); // NULL_LAST if (min_val.isNull()) @@ -301,9 +311,10 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) +#if USE_ROCKSDB , meta_cache(std::make_shared( storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) - +#endif { if (parent_part) state = State::Committed; @@ -329,9 +340,10 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) +#if USE_ROCKSDB , meta_cache(std::make_shared( storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) - +#endif { if (parent_part) state = State::Committed; @@ -738,8 +750,13 @@ void IMergeTreeDataPart::loadIndex() } String index_path = fs::path(getFullRelativePath()) / "primary.idx"; + +#if USE_ROCKSDB String value; - auto index_buf = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + auto index_file = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); +#else + auto index_file = openForReading(volume->getDisk(), index_path); +#endif size_t marks_count = index_granularity.getMarksCount(); @@ -749,7 +766,7 @@ void IMergeTreeDataPart::loadIndex() for (size_t i = 0; i < marks_count; ++i) //-V756 for (size_t j = 0; j < key_size; ++j) - key_serializations[j]->deserializeBinary(*loaded_index[j], *index_buf); + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file); for (size_t i = 0; i < key_size; ++i) { @@ -760,7 +777,7 @@ void IMergeTreeDataPart::loadIndex() ErrorCodes::CANNOT_READ_ALL_DATA); } - if (!index_buf->eof()) + if (!index_file->eof()) throw Exception("Index file " + fullPath(volume->getDisk(), index_path) + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); @@ -804,18 +821,25 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() return; } - String v; String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; - auto in_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); - if (!in_buf) + +#if USE_ROCKSDB + String v; + auto file_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + if (!file_buf) +#else + if (!volume->getDisk()->exists(path)) +#endif { default_codec = detectDefaultCompressionCodec(); } else { - +#if !USE_ROCKSDB + auto file_buf = openForReading(volume->getDisk(), path); +#endif String codec_line; - readEscapedStringUntilEOL(codec_line, *in_buf); + readEscapedStringUntilEOL(codec_line, *file_buf); ReadBufferFromString buf(codec_line); @@ -912,7 +936,11 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() { String path = getFullRelativePath(); if (!parent_part) +#if USE_ROCKSDB partition.load(storage, meta_cache, volume->getDisk(), path); +#else + partition.load(storage, volume->getDisk(), path); +#endif if (!isEmpty()) { @@ -920,7 +948,11 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() // projection parts don't have minmax_idx, and it's always initialized minmax_idx->initialized = true; else +#if USE_ROCKSDB minmax_idx->load(storage, meta_cache, volume->getDisk(), path); +#else + minmax_idx->load(storage, volume->getDisk(), path); +#endif } if (parent_part) return; @@ -950,12 +982,20 @@ void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) c void IMergeTreeDataPart::loadChecksums(bool require) { - const String path = fs::path(getFullRelativePath()) / "checksums.txt"; +#if USE_ROCKSDB String value; auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); if (buf) +#else + const String path = fs::path(getFullRelativePath()) / "checksums.txt"; + if (volume->getDisk()->exists(path)) +#endif { + +#if !USE_ROCKSDB + auto buf = openForReading(volume->getDisk(), path); +#endif if (checksums.read(*buf)) { assertEOF(*buf); @@ -1008,6 +1048,7 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { +#if USE_ROCKSDB String v; auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); if (!buf) @@ -1015,6 +1056,12 @@ void IMergeTreeDataPart::loadRowsCount() readIntText(rows_count, *buf); assertEOF(*buf); +#else + if (!volume->getDisk()->exists(path)) + throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + + read_rows_count(); +#endif #ifndef NDEBUG /// columns have to be loaded @@ -1117,10 +1164,18 @@ void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const void IMergeTreeDataPart::loadTTLInfos() { String path = fs::path(getFullRelativePath()) / "ttl.txt"; +#if USE_ROCKSDB String v; auto in = meta_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); if (in) +#else + if (volume->getDisk()->exists(path)) +#endif { + +#if !USE_ROCKSDB + auto in = openForReading(volume->getDisk(), path); +#endif assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1150,11 +1205,18 @@ void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const void IMergeTreeDataPart::loadUUID() { +#if USE_ROCKSDB String v; - String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; auto in = meta_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); if (in) +#else + String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; + if (volume->getDisk()->exists(path)) +#endif { +#if !USE_ROCKSDB + auto in = openForReading(volume->getDisk(), path); +#endif readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); @@ -1174,9 +1236,13 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; +#if USE_ROCKSDB String v; auto in = meta_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); if (!in) +#else + if (!volume->getDisk()->exists(path)) +#endif { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -1192,14 +1258,18 @@ void IMergeTreeDataPart::loadColumns(bool require) throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); { - auto out = volume->getDisk()->writeFile(path + ".tmp", 4096); - loaded_columns.writeText(*out); + auto buf = volume->getDisk()->writeFile(path + ".tmp", 4096); + loaded_columns.writeText(*buf); } volume->getDisk()->moveFile(path + ".tmp", path); } else { +#if USE_ROCKSDB loaded_columns.readText(*in); +#else + loaded_columns.readText(*volume->getDisk()->readFile(path)); +#endif for (const auto & column : loaded_columns) { @@ -1278,14 +1348,18 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } } +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::DROP, true); -// #ifndef NDEBUG assertMetaCacheDropped(true); -// #endif +#endif + volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; + +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::PUT, true); +#endif SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) @@ -1294,7 +1368,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } - +#if USE_ROCKSDB void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_projection) const { Strings files; @@ -1357,6 +1431,7 @@ void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const } } } +#endif std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { @@ -1395,10 +1470,10 @@ void IMergeTreeDataPart::remove() const return; } +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::DROP); -// #ifndef NDEBUG assertMetaCacheDropped(); -// #endif +#endif /** Atomic directory removal: * - rename directory to temporary name; @@ -1503,10 +1578,10 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { +#if USE_ROCKSDB modifyAllMetaCaches(ModifyCacheType::DROP); -// #ifndef NDEBUG assertMetaCacheDropped(); -// #endif +#endif String to = parent_to + "/" + relative_path; auto disk = volume->getDisk(); @@ -1595,7 +1670,6 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) return "detached/" + getRelativePathForPrefix(prefix, /* detached */ true); } - void IMergeTreeDataPart::renameToDetached(const String & prefix) const { renameTo(getRelativePathForDetachedPart(prefix), true); @@ -1848,6 +1922,7 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } +#if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { String file_name = std::filesystem::path(file_path).filename(); @@ -1912,6 +1987,7 @@ void IMergeTreeDataPart::checkMetaCache(Strings & files, std::vector & disk_checksums.push_back(disk_checksum); } } +#endif bool isCompactPart(const MergeTreeDataPartPtr & data_part) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0d2b2f57fd0..0364012bfec 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -45,6 +45,8 @@ class UncompressedCache; class IMergeTreeDataPart : public std::enable_shared_from_this { public: + +#if USE_ROCKSDB enum ModifyCacheType { PUT, // override set @@ -62,6 +64,9 @@ public: } } + using uint128 = PartMetaCache::uint128; +#endif + static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -77,7 +82,6 @@ public: using IndexSizeByName = std::unordered_map; using Type = MergeTreeDataPartType; - using uint128 = PartMetaCache::uint128; IMergeTreeDataPart( @@ -157,7 +161,9 @@ public: /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; +#if USE_ROCKSDB void assertMetaCacheDropped(bool include_projection = false) const; +#endif void remove() const; @@ -319,7 +325,12 @@ public: { } +#if USE_ROCKSDB void load(const MergeTreeData & data, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); +#else + void load(const MergeTreeData & data, const DiskPtr & disk, const String & part_path); +#endif + void store(const MergeTreeData & data, const DiskPtr & disk, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; @@ -389,7 +400,9 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; +#if USE_ROCKSDB virtual void checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; +#endif bool isProjectionPart() const { return parent_part != nullptr; } @@ -462,7 +475,9 @@ protected: std::map> projection_parts; +#if USE_ROCKSDB mutable PartMetaCachePtr meta_cache; +#endif void removeIfNeeded(); @@ -535,13 +550,14 @@ private: void appendFilesOfDefaultCompressionCodec(Strings & files) const; - void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; - /// Found column without specific compression and return codec /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; +#if USE_ROCKSDB + void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; +#endif mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index e5fbdd5cd19..fa9996b3382 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -192,12 +192,12 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } -// Do not cache mark file, because cache other meta files is enough to speed up loading. +/// Do not cache mark file, because cache other meta files is enough to speed up loading. void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const { } -// find all connected file and do modification +/// find all connected file and do modification Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const { auto marks_file = index_granularity_info.getMarksFilePath("data"); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 5c21ead3208..f3c4b613078 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -166,12 +166,12 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() return checksum; } -// No mark files for part in memory +/// No mark files for part in memory void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files */) const { } -// No mark files for part in memory +/// No mark files for part in memory Strings MergeTreeDataPartInMemory::getIndexGranularityFiles() const { return {}; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 6b5bebd81f6..88a418b1129 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,6 +160,13 @@ namespace }; } +#if !USE_ROCKSDB +static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) +{ + size_t file_size = disk->getFileSize(path); + return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); +} +#endif String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -350,7 +357,11 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } } +#if USE_ROCKSDB void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path) +#else +void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) +#endif { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) @@ -359,11 +370,16 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCache const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; auto partition_file_path = part_path + "partition.dat"; +#if USE_ROCKSDB String v; - auto buf = meta_cache->readOrSetMeta(disk, "partition.dat", v); + auto file = meta_cache->readOrSetMeta(disk, "partition.dat", v); +#else + auto file = openForReading(disk, partition_file_path); +#endif + value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) - partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *buf); + partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file); } void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index b8b5b301219..4920e658163 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -4,9 +4,12 @@ #include #include #include -#include #include +#if USE_ROCKSDB +#include +#endif + namespace DB { @@ -38,7 +41,12 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; +#if USE_ROCKSDB void load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); +#else + void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); +#endif + void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; From 22b9f9d3e5062380335043b6fa2197cc797e7773 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 15:45:48 +0800 Subject: [PATCH 005/103] fix ut --- src/Interpreters/Context.h | 5 ++-- ...01233_check_part_meta_cache_replicated.sql | 28 ++++++++++--------- ...k_part_meta_cache_replicated_in_atomic.sql | 28 ++++++++++--------- 3 files changed, 32 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bc191e80c9a..dcc24999944 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,6 +1,5 @@ #pragma once -#include "config_core.h" #include #include #include @@ -17,13 +16,13 @@ #include #include +#include "config_core.h" + #if USE_ROCKSDB #include #include #endif -#include "config_core.h" - #include #include #include diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index cb028e77d54..7722bba71dd 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,10 +1,12 @@ +-- Tags: zookeeper + -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +DROP DATABASE IF EXISTS test_meta_cache; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; +CREATE DATABASE test_meta_cache ENGINE = Ordinary; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. @@ -43,21 +45,21 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +alter table test_meta_cache.check_part_meta_cache drop column v1; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); @@ -66,8 +68,8 @@ truncate table test_meta_cache.check_part_meta_cache; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_meta_cache.check_part_meta_cache ; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -- Insert first batch of data. INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -105,17 +107,17 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index c56e2cb0a99..4dcc4478139 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,10 +1,12 @@ +-- Tags: zookeeper + -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache on cluster preonline_hk5; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE DATABASE test_meta_cache on cluster preonline_hk5 ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; +DROP DATABASE IF EXISTS test_meta_cache ; +DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache ; +CREATE DATABASE test_meta_cache ENGINE = Atomic; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. @@ -43,21 +45,21 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 drop column v1; +alter table test_meta_cache.check_part_meta_cache drop column v1; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); @@ -66,8 +68,8 @@ truncate table test_meta_cache.check_part_meta_cache; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache on cluster preonline_hk5; -CREATE TABLE test_meta_cache.check_part_meta_cache on cluster preonline_hk5 ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_meta_cache.check_part_meta_cache ; +CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -- Insert first batch of data. INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -105,17 +107,17 @@ optimize table test_meta_cache.check_part_meta_cache FINAL; with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 add column v3 UInt64; +alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 10 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache on cluster preonline_hk5 modify TTL p + INTERVAL 15 YEAR; +alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); From 2a1fe52b9f2136405c3a5b2497e36be3528b4e74 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 18:06:13 +0800 Subject: [PATCH 006/103] rename symbols && fix uts && add setting use_metadata_cache --- programs/server/Server.cpp | 2 +- src/Common/ProfileEvents.cpp | 14 +- ...taCache.cpp => checkPartMetadataCache.cpp} | 79 +++-- .../registerFunctionsMiscellaneous.cpp | 4 +- src/Interpreters/Context.cpp | 48 +-- src/Interpreters/Context.h | 16 +- src/Processors/examples/CMakeLists.txt | 4 +- ...ache.cpp => merge_tree_metadata_cache.cpp} | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 286 ++++++++++++------ src/Storages/MergeTree/IMergeTreeDataPart.h | 17 +- src/Storages/MergeTree/MergeTreeData.cpp | 11 + src/Storages/MergeTree/MergeTreePartition.cpp | 18 +- src/Storages/MergeTree/MergeTreePartition.h | 4 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + ...artMetaCache.cpp => PartMetadataCache.cpp} | 22 +- .../{PartMetaCache.h => PartMetadataCache.h} | 10 +- ...> StorageSystemMergeTreeMetadataCache.cpp} | 14 +- ... => StorageSystemMergeTreeMetadataCache.h} | 8 +- src/Storages/System/attachSystemTables.cpp | 4 +- .../01233_check_part_meta_cache.sql | 140 ++++----- .../01233_check_part_meta_cache_in_atomic.sql | 140 ++++----- ...01233_check_part_meta_cache_replicated.sql | 139 ++++----- ...k_part_meta_cache_replicated_in_atomic.sql | 139 ++++----- 23 files changed, 634 insertions(+), 492 deletions(-) rename src/Functions/{checkPartMetaCache.cpp => checkPartMetadataCache.cpp} (68%) rename src/Processors/examples/{merge_tree_meta_cache.cpp => merge_tree_metadata_cache.cpp} (86%) rename src/Storages/MergeTree/{PartMetaCache.cpp => PartMetadataCache.cpp} (80%) rename src/Storages/MergeTree/{PartMetaCache.h => PartMetadataCache.h} (76%) rename src/Storages/System/{StorageSystemMergeTreeMetaCache.cpp => StorageSystemMergeTreeMetadataCache.cpp} (84%) rename src/Storages/System/{StorageSystemMergeTreeMetaCache.h => StorageSystemMergeTreeMetadataCache.h} (55%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6cfdc2376d6..1e10af9dc07 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -823,7 +823,7 @@ if (ThreadFuzzer::instance().isEffective()) /// initialize meta file cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); - global_context->initializeMergeTreeMetaCache(path_str + "/" + "rocksdb", size); + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); } #endif diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index e0383da29f4..45089312ed1 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -277,15 +277,15 @@ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ \ - M(RocksdbGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ - M(RocksdbPut, "Number of rocksdb puts(used for merge tree metadata cache)") \ - M(RocksdbDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ - M(RocksdbSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ - M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \ - M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \ - \ M(MainConfigLoads, "Number of times the main configuration was reloaded.") \ \ + M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \ + M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \ + \ namespace ProfileEvents { diff --git a/src/Functions/checkPartMetaCache.cpp b/src/Functions/checkPartMetadataCache.cpp similarity index 68% rename from src/Functions/checkPartMetaCache.cpp rename to src/Functions/checkPartMetadataCache.cpp index e77b8ca0d50..2883424f996 100644 --- a/src/Functions/checkPartMetaCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -21,14 +21,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } -class FunctionCheckPartMetaCache: public IFunction, WithContext +class FunctionCheckPartMetadataCache : public IFunction, WithContext { public: using uint128 = IMergeTreeDataPart::uint128; @@ -37,22 +36,18 @@ public: using DataPartStates = MergeTreeData::DataPartStates; - static constexpr auto name = "checkPartMetaCache"; - static FunctionPtr create(ContextPtr context_) - { - return std::make_shared(context_); - } + static constexpr auto name = "checkPartMetadataCache"; + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - static constexpr DataPartStates part_states = { - DataPartState::Committed, - DataPartState::Temporary, - DataPartState::PreCommitted, - DataPartState::Outdated, - DataPartState::Deleting, - DataPartState::DeleteOnDestroy - }; + static constexpr DataPartStates part_states + = {DataPartState::Committed, + DataPartState::Temporary, + DataPartState::PreCommitted, + DataPartState::Outdated, + DataPartState::Deleting, + DataPartState::DeleteOnDestroy}; - explicit FunctionCheckPartMetaCache(ContextPtr context_): WithContext(context_) {} + explicit FunctionCheckPartMetadataCache(ContextPtr context_) : WithContext(context_) { } String getName() const override { return name; } @@ -76,44 +71,44 @@ public: DataTypePtr cache_checksum_type = std::make_unique(32); DataTypePtr disk_checksum_type = std::make_unique(32); DataTypePtr match_type = std::make_unique(); - DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); + DataTypePtr tuple_type + = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); return std::make_shared(tuple_type); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - // get database name + /// Get database name const auto * arg_database = arguments[0].column.get(); - const ColumnString * column_database = checkAndGetColumnConstData(arg_database); - if (! column_database) + const ColumnString * column_database = checkAndGetColumnConstData(arg_database); + if (!column_database) throw Exception("The first argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); String database_name = column_database->getDataAt(0).toString(); - // get table name + /// Get table name const auto * arg_table = arguments[1].column.get(); const ColumnString * column_table = checkAndGetColumnConstData(arg_table); - if (! column_table) + if (!column_table) throw Exception("The second argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); String table_name = column_table->getDataAt(0).toString(); - // get storage + /// Get storage StorageID storage_id(database_name, table_name); auto storage = DatabaseCatalog::instance().getTable(storage_id, getContext()); auto data = std::dynamic_pointer_cast(storage); - if (! data) + if (!data || !data->getSettings()->use_metadata_cache) throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); - // fill in result + /// Fill in result auto col_result = result_type->createColumn(); - auto& col_arr = assert_cast(*col_result); - col_arr.reserve(1); - auto& col_tuple = assert_cast(col_arr.getData()); + auto & col_arr = assert_cast(*col_result); + auto & col_tuple = assert_cast(col_arr.getData()); col_tuple.reserve(data->fileNumberOfDataParts(part_states)); - auto& col_key = assert_cast(col_tuple.getColumn(0)); - auto& col_state = assert_cast(col_tuple.getColumn(1)); - auto& col_cache_checksum = assert_cast(col_tuple.getColumn(2)); - auto& col_disk_checksum = assert_cast(col_tuple.getColumn(3)); - auto& col_match = assert_cast(col_tuple.getColumn(4)); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_state = assert_cast(col_tuple.getColumn(1)); + auto & col_cache_checksum = assert_cast(col_tuple.getColumn(2)); + auto & col_disk_checksum = assert_cast(col_tuple.getColumn(3)); + auto & col_match = assert_cast(col_tuple.getColumn(4)); auto parts = data->getDataParts(part_states); for (const auto & part : parts) executePart(part, col_key, col_state, col_cache_checksum, col_disk_checksum, col_match); @@ -121,21 +116,26 @@ public: return result_type->createColumnConst(input_rows_count, col_arr[0]); } - static void executePart(const DataPartPtr& part, ColumnString& col_key, ColumnString& col_state, - ColumnFixedString& col_cache_checksum, ColumnFixedString& col_disk_checksum, ColumnUInt8& col_match) + static void executePart( + const DataPartPtr & part, + ColumnString & col_key, + ColumnString & col_state, + ColumnFixedString & col_cache_checksum, + ColumnFixedString & col_disk_checksum, + ColumnUInt8 & col_match) { Strings keys; auto state_view = part->stateString(); String state(state_view.data(), state_view.size()); std::vector cache_checksums; std::vector disk_checksums; - uint8_t match = 0; + uint8_t match = 0; size_t file_number = part->fileNumberOfColumnsChecksumsIndexes(); keys.reserve(file_number); cache_checksums.reserve(file_number); disk_checksums.reserve(file_number); - part->checkMetaCache(keys, cache_checksums, disk_checksums); + part->checkMetadataCache(keys, cache_checksums, disk_checksums); for (size_t i = 0; i < keys.size(); ++i) { col_key.insert(keys[i]); @@ -149,10 +149,9 @@ public: } }; -void registerFunctionCheckPartMetaCache(FunctionFactory & factory) +void registerFunctionCheckPartMetadataCache(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } - } #endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 77e3e109081..297e6dfb452 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -82,7 +82,7 @@ void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); #if USE_ROCKSDB -void registerFunctionCheckPartMetaCache(FunctionFactory &); +void registerFunctionCheckPartMetadataCache(FunctionFactory &); #endif #if USE_ICU @@ -172,7 +172,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionGetOSKernelVersion(factory); #if USE_ROCKSDB - registerFunctionCheckPartMetaCache(factory); + registerFunctionCheckPartMetadataCache(factory); #endif #if USE_ICU diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4d9c571232e..4897cdccac9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -97,10 +97,10 @@ namespace ProfileEvents extern const Event ContextLock; #if USE_ROCKSDB - extern const Event RocksdbPut; - extern const Event RocksdbGet; - extern const Event RocksdbDelete; - extern const Event RocksdbSeek; + extern const Event MergeTreeMetadataCachePut; + extern const Event MergeTreeMetadataCacheGet; + extern const Event MergeTreeMetadataCacheDelete; + extern const Event MergeTreeMetadataCacheSeek; #endif } @@ -284,7 +284,7 @@ struct ContextSharedPart #if USE_ROCKSDB /// MergeTree metadata cache stored in rocksdb. - MergeTreeMetaCachePtr merge_tree_meta_cache; + MergeTreeMetadataCachePtr merge_tree_metadata_cache; #endif ContextSharedPart() @@ -400,10 +400,10 @@ struct ContextSharedPart #if USE_ROCKSDB /// Shutdown meta file cache - if (merge_tree_meta_cache) + if (merge_tree_metadata_cache) { - merge_tree_meta_cache->shutdown(); - merge_tree_meta_cache.reset(); + merge_tree_metadata_cache->shutdown(); + merge_tree_metadata_cache.reset(); } #endif } @@ -450,32 +450,32 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } #if USE_ROCKSDB -MergeTreeMetaCache::Status MergeTreeMetaCache::put(const String & key, const String & value) +MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); auto status = rocksdb->Put(options, key, value); - ProfileEvents::increment(ProfileEvents::RocksdbPut); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); return status; } -MergeTreeMetaCache::Status MergeTreeMetaCache::del(const String & key) +MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) { auto options = rocksdb::WriteOptions(); auto status = rocksdb->Delete(options, key); - ProfileEvents::increment(ProfileEvents::RocksdbDelete); - LOG_TRACE(log, "Delete key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); + LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); return status; } -MergeTreeMetaCache::Status MergeTreeMetaCache::get(const String & key, String & value) +MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value) { auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); - ProfileEvents::increment(ProfileEvents::RocksdbGet); - LOG_TRACE(log, "Get key:{} from MergeTreeMetaCache status:{}", key, status.ToString()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet); + LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); return status; } -void MergeTreeMetaCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) +void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) { auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); rocksdb::Slice target(prefix); @@ -489,11 +489,11 @@ void MergeTreeMetaCache::getByPrefix(const String & prefix, Strings & keys, Stri keys.emplace_back(key.data(), key.size()); values.emplace_back(value.data(), value.size()); } - LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetaCache items:{}", prefix, keys.size()); - ProfileEvents::increment(ProfileEvents::RocksdbSeek); + LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); } -void MergeTreeMetaCache::shutdown() +void MergeTreeMetadataCache::shutdown() { if (rocksdb) { @@ -2083,9 +2083,9 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const } #if USE_ROCKSDB -MergeTreeMetaCachePtr Context::getMergeTreeMetaCache() const +MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { - return shared->merge_tree_meta_cache; + return shared->merge_tree_metadata_cache; } #endif @@ -2333,7 +2333,7 @@ void Context::initializeTraceCollector() } #if USE_ROCKSDB -void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) +void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) { rocksdb::Options options; rocksdb::BlockBasedTableOptions table_options; @@ -2350,7 +2350,7 @@ void Context::initializeMergeTreeMetaCache(const String & dir, size_t size) String message = "Fail to open rocksdb path at: " + dir + " status:" + status.ToString(); throw Exception(message, ErrorCodes::SYSTEM_ERROR); } - shared->merge_tree_meta_cache = std::make_shared(db); + shared->merge_tree_metadata_cache = std::make_shared(db); } #endif diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e579fc9c4bf..ea606fdea4e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -183,14 +183,14 @@ private: }; #if USE_ROCKSDB -class MergeTreeMetaCache +class MergeTreeMetadataCache { public: using Status = rocksdb::Status; - explicit MergeTreeMetaCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } - MergeTreeMetaCache(const MergeTreeMetaCache &) = delete; - MergeTreeMetaCache & operator=(const MergeTreeMetaCache &) = delete; + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; + MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; Status put(const String & key, const String & value); Status del(const String & key); @@ -200,9 +200,9 @@ public: void shutdown(); private: std::unique_ptr rocksdb; - Poco::Logger * log = &Poco::Logger::get("MergeTreeMetaCache"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache"); }; -using MergeTreeMetaCachePtr = std::shared_ptr; +using MergeTreeMetadataCachePtr = std::shared_ptr; #endif /** A set of known objects that can be used in the query. @@ -705,7 +705,7 @@ public: UInt32 getZooKeeperSessionUptime() const; #if USE_ROCKSDB - MergeTreeMetaCachePtr getMergeTreeMetaCache() const; + MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; #endif @@ -799,7 +799,7 @@ public: void initializeTraceCollector(); #if USE_ROCKSDB - void initializeMergeTreeMetaCache(const String & dir, size_t size); + void initializeMergeTreeMetadataCache(const String & dir, size_t size); #endif bool hasTraceCollector() const; diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index a07224d4462..2b6b9128e4c 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,4 +1,4 @@ if (USE_ROCKSDB) - add_executable (merge_tree_meta_cache merge_tree_meta_cache.cpp) - target_link_libraries (merge_tree_meta_cache PRIVATE dbms) + add_executable (merge_tree_metadata_cache merge_tree_metadata_cache.cpp) + target_link_libraries (merge_tree_metadata_cache PRIVATE dbms) endif() \ No newline at end of file diff --git a/src/Processors/examples/merge_tree_meta_cache.cpp b/src/Processors/examples/merge_tree_metadata_cache.cpp similarity index 86% rename from src/Processors/examples/merge_tree_meta_cache.cpp rename to src/Processors/examples/merge_tree_metadata_cache.cpp index a394323bcb7..c726eb7ce5a 100644 --- a/src/Processors/examples/merge_tree_meta_cache.cpp +++ b/src/Processors/examples/merge_tree_metadata_cache.cpp @@ -7,13 +7,13 @@ int main() auto shared_context = Context::createShared(); auto global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); - global_context->initializeMergeTreeMetaCache("./db/", 256 << 20); + global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - auto cache = global_context->getMergeTreeMetaCache(); + auto cache = global_context->getMergeTreeMetadataCache(); std::vector files = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; - String prefix = "data/test_meta_cache/check_part_meta_cache/201806_1_1_0_4/"; + String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/"; for (const auto & file : files) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0b5f9f19782..b29886265f6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -65,7 +65,7 @@ static std::unique_ptr openForReading(const DiskPtr & di #if USE_ROCKSDB void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const PartMetaCachePtr & cache, const DiskPtr & disk, const String & /*part_path*/) + const MergeTreeData & data, const PartMetadataCachePtr & cache, const DiskPtr & disk, const String & part_path) #else void IMergeTreeDataPart::MinMaxIndex::load( const MergeTreeData & data, const DiskPtr & disk, const String & part_path) @@ -78,16 +78,30 @@ void IMergeTreeDataPart::MinMaxIndex::load( auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); + auto read_min_max_index = [&](size_t i) + { + String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); + auto file = openForReading(disk, file_name); + return file; + }; + hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { + std::unique_ptr file; #if USE_ROCKSDB - String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; - String value; - auto file = cache->readOrSetMeta(disk, file_name, value); + if (cache) + { + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + String value; + file = cache->readOrSetMeta(disk, file_name, value); + } + else + { + file = read_min_max_index(i); + } #else - String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); - auto file = openForReading(disk, file_name); + file = read_min_max_index(i); #endif auto serialization = minmax_column_types[i]->getDefaultSerialization(); @@ -311,10 +325,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) -#if USE_ROCKSDB - , meta_cache(std::make_shared( - storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) -#endif + , use_metadata_cache(storage.getSettings()->use_metadata_cache) { if (parent_part) state = State::Committed; @@ -322,6 +333,12 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); + +#if USE_ROCKSDB + if (use_metadata_cache) + metadata_cache = std::make_shared( + storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); +#endif } IMergeTreeDataPart::IMergeTreeDataPart( @@ -340,10 +357,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) -#if USE_ROCKSDB - , meta_cache(std::make_shared( - storage.getContext()->getMergeTreeMetaCache(), storage.relative_data_path, relative_path, parent_part)) -#endif + , use_metadata_cache(storage.getSettings()->use_metadata_cache) { if (parent_part) state = State::Committed; @@ -351,6 +365,12 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); + +#if USE_ROCKSDB + if (use_metadata_cache) + metadata_cache = std::make_shared( + storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); +#endif } IMergeTreeDataPart::~IMergeTreeDataPart() @@ -751,11 +771,19 @@ void IMergeTreeDataPart::loadIndex() String index_path = fs::path(getFullRelativePath()) / "primary.idx"; + std::unique_ptr index_file; #if USE_ROCKSDB - String value; - auto index_file = meta_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + if (use_metadata_cache) + { + String value; + index_file = metadata_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + } + else + { + index_file = openForReading(volume->getDisk(), index_path); + } #else - auto index_file = openForReading(volume->getDisk(), index_path); + index_file = openForReading(volume->getDisk(), index_path); #endif size_t marks_count = index_granularity.getMarksCount(); @@ -823,21 +851,33 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; + bool exists = false; + std::unique_ptr file_buf; #if USE_ROCKSDB - String v; - auto file_buf = meta_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); - if (!file_buf) + if (use_metadata_cache) + { + String v; + file_buf = metadata_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + exists = file_buf != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + file_buf = openForReading(volume->getDisk(), path); + } #else - if (!volume->getDisk()->exists(path)) + exists = volume->getDisk()->exists(path); + if (exists) + file_buf = openForReading(volume->getDisk(), path); #endif + + if (!exists) { default_codec = detectDefaultCompressionCodec(); } else { -#if !USE_ROCKSDB - auto file_buf = openForReading(volume->getDisk(), path); -#endif String codec_line; readEscapedStringUntilEOL(codec_line, *file_buf); @@ -937,7 +977,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() String path = getFullRelativePath(); if (!parent_part) #if USE_ROCKSDB - partition.load(storage, meta_cache, volume->getDisk(), path); + partition.load(storage, metadata_cache, volume->getDisk(), path); #else partition.load(storage, volume->getDisk(), path); #endif @@ -949,7 +989,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() minmax_idx->initialized = true; else #if USE_ROCKSDB - minmax_idx->load(storage, meta_cache, volume->getDisk(), path); + minmax_idx->load(storage, metadata_cache, volume->getDisk(), path); #else minmax_idx->load(storage, volume->getDisk(), path); #endif @@ -982,20 +1022,31 @@ void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) c void IMergeTreeDataPart::loadChecksums(bool require) { + const String path = fs::path(getFullRelativePath()) / "checksums.txt"; + bool exists = false; + std::unique_ptr buf; #if USE_ROCKSDB - String value; - auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); - if (buf) -#else - const String path = fs::path(getFullRelativePath()) / "checksums.txt"; - if (volume->getDisk()->exists(path)) -#endif + if (use_metadata_cache) { - -#if !USE_ROCKSDB - auto buf = openForReading(volume->getDisk(), path); + String value; + buf = metadata_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); + exists = buf != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + buf = openForReading(volume->getDisk(), path); + } +#else + exists = volume->getDisk()->exists(path); + if (exists) + buf = openForReading(volume->getDisk(), path); #endif + + if (exists) + { if (checksums.read(*buf)) { assertEOF(*buf); @@ -1049,13 +1100,23 @@ void IMergeTreeDataPart::loadRowsCount() else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { #if USE_ROCKSDB - String v; - auto buf = meta_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); - if (!buf) - throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + if (use_metadata_cache) + { + String v; + auto buf = metadata_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); + if (!buf) + throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - readIntText(rows_count, *buf); - assertEOF(*buf); + readIntText(rows_count, *buf); + assertEOF(*buf); + } + else + { + if (!volume->getDisk()->exists(path)) + throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + + read_rows_count(); + } #else if (!volume->getDisk()->exists(path)) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -1164,18 +1225,30 @@ void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const void IMergeTreeDataPart::loadTTLInfos() { String path = fs::path(getFullRelativePath()) / "ttl.txt"; -#if USE_ROCKSDB - String v; - auto in = meta_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); - if (in) -#else - if (volume->getDisk()->exists(path)) -#endif - { + bool exists = false; + std::unique_ptr in; -#if !USE_ROCKSDB - auto in = openForReading(volume->getDisk(), path); +#if USE_ROCKSDB + if (use_metadata_cache) + { + String v; + in = metadata_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); + exists = in != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); + } +#else + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); #endif + + if (exists) + { assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1205,18 +1278,31 @@ void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const void IMergeTreeDataPart::loadUUID() { -#if USE_ROCKSDB - String v; - auto in = meta_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); - if (in) -#else String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; - if (volume->getDisk()->exists(path)) -#endif + bool exists = false; + std::unique_ptr in; + +#if USE_ROCKSDB + if (use_metadata_cache) { -#if !USE_ROCKSDB - auto in = openForReading(volume->getDisk(), path); + String v; + in = metadata_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); + exists = in != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); + } +#else + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); #endif + + if (exists) + { readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); @@ -1236,13 +1322,28 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; + bool exists = false; + std::unique_ptr in; #if USE_ROCKSDB - String v; - auto in = meta_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); - if (!in) + if (use_metadata_cache) + { + String v; + in = metadata_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); + exists = in != nullptr; + } + else + { + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); + } #else - if (!volume->getDisk()->exists(path)) + exists = volume->getDisk()->exists(path); + if (exists) + in = openForReading(volume->getDisk(), path); #endif + + if (!exists) { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::COMPACT) @@ -1265,11 +1366,7 @@ void IMergeTreeDataPart::loadColumns(bool require) } else { -#if USE_ROCKSDB loaded_columns.readText(*in); -#else - loaded_columns.readText(*volume->getDisk()->readFile(path)); -#endif for (const auto & column : loaded_columns) { @@ -1349,8 +1446,11 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::DROP, true); - assertMetaCacheDropped(true); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::DROP, true); + assertMetadataCacheDropped(true); + } #endif volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); @@ -1358,7 +1458,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ relative_path = new_relative_path; #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::PUT, true); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::PUT, true); + } #endif SyncGuardPtr sync_guard; @@ -1369,8 +1472,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } #if USE_ROCKSDB -void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_projection) const +void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool include_projection) const { + assert(use_metadata_cache); + Strings files; files.reserve(16); appendFilesOfColumnsChecksumsIndexes(files, include_projection); @@ -1385,20 +1490,21 @@ void IMergeTreeDataPart::modifyAllMetaCaches(ModifyCacheType type, bool include_ switch (type) { case ModifyCacheType::PUT: - meta_cache->setMetas(volume->getDisk(), files); + metadata_cache->setMetas(volume->getDisk(), files); break; case ModifyCacheType::DROP: - meta_cache->dropMetas(files); + metadata_cache->dropMetas(files); break; } } - -void IMergeTreeDataPart::assertMetaCacheDropped(bool include_projection) const +void IMergeTreeDataPart::assertMetadataCacheDropped(bool include_projection) const { + assert(use_metadata_cache); + Strings files; std::vector _; - meta_cache->getFilesAndCheckSums(files, _); + metadata_cache->getFilesAndCheckSums(files, _); if (files.empty()) return; @@ -1471,8 +1577,11 @@ void IMergeTreeDataPart::remove() const } #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::DROP); - assertMetaCacheDropped(); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::DROP); + assertMetadataCacheDropped(); + } #endif /** Atomic directory removal: @@ -1579,8 +1688,11 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { #if USE_ROCKSDB - modifyAllMetaCaches(ModifyCacheType::DROP); - assertMetaCacheDropped(); + if (use_metadata_cache) + { + modifyAllMetadataCaches(ModifyCacheType::DROP); + assertMetadataCacheDropped(); + } #endif String to = parent_to + "/" + relative_path; @@ -1925,6 +2037,8 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const #if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { + assert(use_metadata_cache); + String file_name = std::filesystem::path(file_path).filename(); const auto filenames_without_checksums = getFileNamesWithoutChecksums(); auto it = checksums.files.find(file_name); @@ -1945,15 +2059,17 @@ IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const St return in_hash.getHash(); } -void IMergeTreeDataPart::checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const +void IMergeTreeDataPart::checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const { - /// checkMetaCache only applies for normal part + assert(use_metadata_cache); + + /// Only applies for normal part if (isProjectionPart()) return; /// the directory of projection part is under the directory of its parent part const auto filenames_without_checksums = getFileNamesWithoutChecksums(); - meta_cache->getFilesAndCheckSums(files, cache_checksums); + metadata_cache->getFilesAndCheckSums(files, cache_checksums); for (const auto & file : files) { // std::cout << "check key:" << file << std::endl; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0364012bfec..2798f42b6c4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include @@ -64,7 +64,7 @@ public: } } - using uint128 = PartMetaCache::uint128; + using uint128 = PartMetadataCache::uint128; #endif static constexpr auto DATA_FILE_EXTENSION = ".bin"; @@ -162,7 +162,7 @@ public: void assertOnDisk() const; #if USE_ROCKSDB - void assertMetaCacheDropped(bool include_projection = false) const; + void assertMetadataCacheDropped(bool include_projection = false) const; #endif void remove() const; @@ -326,7 +326,7 @@ public: } #if USE_ROCKSDB - void load(const MergeTreeData & data, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); + void load(const MergeTreeData & data, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); #else void load(const MergeTreeData & data, const DiskPtr & disk, const String & part_path); #endif @@ -401,7 +401,7 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; #if USE_ROCKSDB - virtual void checkMetaCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; + virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; #endif bool isProjectionPart() const { return parent_part != nullptr; } @@ -475,8 +475,11 @@ protected: std::map> projection_parts; + /// Disabled when USE_ROCKSDB is OFF, or use_metadata_cache is set true in merge tree settings + bool use_metadata_cache = false; + #if USE_ROCKSDB - mutable PartMetaCachePtr meta_cache; + mutable PartMetadataCachePtr metadata_cache; #endif void removeIfNeeded(); @@ -555,7 +558,7 @@ private: CompressionCodecPtr detectDefaultCompressionCodec() const; #if USE_ROCKSDB - void modifyAllMetaCaches(ModifyCacheType type, bool include_projection = false) const; + void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; #endif diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8d861e404f0..6add439d1f9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -317,6 +317,17 @@ MergeTreeData::MergeTreeData( LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); +#if !USE_ROCKSDB + if (settings->use_metadata_cache) + { + LOG_WARNING( + log, + "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." + "set use_metadata_cache to false forcely"); + settings->use_metadata_cache = false; + } +#endif + common_assignee_trigger = [this] (bool delay) noexcept { if (delay) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 88a418b1129..933248c9a1e 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,13 +160,11 @@ namespace }; } -#if !USE_ROCKSDB static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) { size_t file_size = disk->getFileSize(path); return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } -#endif String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -358,7 +356,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } #if USE_ROCKSDB -void MergeTreePartition::load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path) +void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path) #else void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) #endif @@ -370,11 +368,19 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; auto partition_file_path = part_path + "partition.dat"; + std::unique_ptr file; #if USE_ROCKSDB - String v; - auto file = meta_cache->readOrSetMeta(disk, "partition.dat", v); + if (metadata_cache) + { + String v; + file = metadata_cache->readOrSetMeta(disk, "partition.dat", v); + } + else + { + file = openForReading(disk, partition_file_path); + } #else - auto file = openForReading(disk, partition_file_path); + file = openForReading(disk, partition_file_path); #endif value.resize(partition_key_sample.columns()); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 4920e658163..644ddd5ba88 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -7,7 +7,7 @@ #include #if USE_ROCKSDB -#include +#include #endif namespace DB @@ -42,7 +42,7 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; #if USE_ROCKSDB - void load(const MergeTreeData & storage, const PartMetaCachePtr & meta_cache, const DiskPtr & disk, const String & part_path); + void load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); #else void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); #endif diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b991166b3b6..51831eecf84 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -137,6 +137,7 @@ struct Settings; /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ + M(Bool, use_metadata_cache, false, "Experimental feature to speed up parts loading process by using MergeTree metadata cache", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ diff --git a/src/Storages/MergeTree/PartMetaCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp similarity index 80% rename from src/Storages/MergeTree/PartMetaCache.cpp rename to src/Storages/MergeTree/PartMetadataCache.cpp index fe8475d0dda..a8ab2c0bf2d 100644 --- a/src/Storages/MergeTree/PartMetaCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -1,4 +1,4 @@ -#include "PartMetaCache.h" +#include "PartMetadataCache.h" #if USE_ROCKSDB #include @@ -11,8 +11,8 @@ namespace ProfileEvents { - extern const Event MergeTreeMetaCacheHit; - extern const Event MergeTreeMetaCacheMiss; + extern const Event MergeTreeMetadataCacheHit; + extern const Event MergeTreeMetadataCacheMiss; } namespace ErrorCodes @@ -24,13 +24,13 @@ namespace DB { std::unique_ptr -PartMetaCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) +PartMetadataCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; auto status = cache->get(file_path, value); if (!status.ok()) { - ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheMiss); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); if (!disk->exists(file_path)) { return nullptr; @@ -45,12 +45,12 @@ PartMetaCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, Str } else { - ProfileEvents::increment(ProfileEvents::MergeTreeMetaCacheHit); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); } return std::make_unique(value); } -void PartMetaCache::setMetas(const DiskPtr & disk, const Strings & file_names) +void PartMetadataCache::setMetas(const DiskPtr & disk, const Strings & file_names) { String text; String read_value; @@ -76,7 +76,7 @@ void PartMetaCache::setMetas(const DiskPtr & disk, const Strings & file_names) } } -void PartMetaCache::dropMetas(const Strings & file_names) +void PartMetadataCache::dropMetas(const Strings & file_names) { for (const auto & file_name : file_names) { @@ -93,7 +93,7 @@ void PartMetaCache::dropMetas(const Strings & file_names) } } -void PartMetaCache::setMeta(const String & file_name, const String & value) +void PartMetadataCache::setMeta(const String & file_name, const String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; String read_value; @@ -112,7 +112,7 @@ void PartMetaCache::setMeta(const String & file_name, const String & value) } } -void PartMetaCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const +void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const { String prefix = fs::path(getFullRelativePath()) / ""; Strings values; @@ -127,7 +127,7 @@ void PartMetaCache::getFilesAndCheckSums(Strings & files, std::vector & } } -String PartMetaCache::getFullRelativePath() const +String PartMetadataCache::getFullRelativePath() const { return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; } diff --git a/src/Storages/MergeTree/PartMetaCache.h b/src/Storages/MergeTree/PartMetadataCache.h similarity index 76% rename from src/Storages/MergeTree/PartMetaCache.h rename to src/Storages/MergeTree/PartMetadataCache.h index 5ffd0413c4b..97880f41b6c 100644 --- a/src/Storages/MergeTree/PartMetaCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -13,15 +13,15 @@ namespace DB class SeekableReadBuffer; class IMergeTreeDataPart; -class PartMetaCache; -using PartMetaCachePtr = std::shared_ptr; +class PartMetadataCache; +using PartMetadataCachePtr = std::shared_ptr; -class PartMetaCache +class PartMetadataCache { public: using uint128 = CityHash_v1_0_2::uint128; - PartMetaCache(const MergeTreeMetaCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) + PartMetadataCache(const MergeTreeMetadataCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) : cache(cache_) , relative_data_path(relative_data_path_) , relative_path(relative_path_) @@ -39,7 +39,7 @@ public: private: std::string getFullRelativePath() const; - MergeTreeMetaCachePtr cache; + MergeTreeMetadataCachePtr cache; const String & relative_data_path; // relative path of table to disk const String & relative_path; // relative path of part to table const IMergeTreeDataPart * parent_part; diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp similarity index 84% rename from src/Storages/System/StorageSystemMergeTreeMetaCache.cpp rename to src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index f53c32e5a42..b62a7985c0c 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_ROCKSDB #include @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -NamesAndTypesList StorageSystemMergeTreeMetaCache::getNamesAndTypes() +NamesAndTypesList StorageSystemMergeTreeMetadataCache::getNamesAndTypes() { return { {"key", std::make_shared()}, @@ -95,19 +95,19 @@ static String extractKey(const ASTPtr & query, bool& precise) } -void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { bool precise = false; String key = extractKey(query_info.query, precise); if (key.empty()) throw Exception( - "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); - auto cache = context->getMergeTreeMetaCache(); + auto cache = context->getMergeTreeMetadataCache(); if (precise) { String value; - if (cache->get(key, value) != MergeTreeMetaCache::Status::OK()) + if (cache->get(key, value) != MergeTreeMetadataCache::Status::OK()) return; size_t col_num = 0; @@ -119,7 +119,7 @@ void StorageSystemMergeTreeMetaCache::fillData(MutableColumns & res_columns, Con String target = extractFixedPrefixFromLikePattern(key); if (target.empty()) throw Exception( - "SELECT from system.merge_tree_meta_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); + "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); Strings keys; Strings values; diff --git a/src/Storages/System/StorageSystemMergeTreeMetaCache.h b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h similarity index 55% rename from src/Storages/System/StorageSystemMergeTreeMetaCache.h rename to src/Storages/System/StorageSystemMergeTreeMetadataCache.h index c8e0f475cd8..a61f996f4df 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetaCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h @@ -12,14 +12,14 @@ namespace DB class Context; -/** Implements `merge_tree_meta_cache` system table, which allows you to view the metacache data in rocksdb for debugging purposes. +/** Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for debugging purposes. */ -class StorageSystemMergeTreeMetaCache : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMergeTreeMetadataCache : public shared_ptr_helper, public IStorageSystemOneBlock { - friend struct shared_ptr_helper; + friend struct shared_ptr_helper; public: - std::string getName() const override { return "SystemMergeTreeMetaCache"; } + std::string getName() const override { return "SystemMergeTreeMetadataCache"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 5f5a17069f3..96ee000c0bf 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -75,7 +75,7 @@ #if USE_ROCKSDB #include -#include +#include #endif @@ -130,7 +130,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) #endif #if USE_ROCKSDB attach(context, system_database, "rocksdb"); - attach(context, system_database, "merge_tree_meta_cache"); + attach(context, system_database, "merge_tree_metadata_cache"); #endif } diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index f0ca3b608d1..3c5d55a0069 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -1,123 +1,125 @@ +--Tags: no-fasttest + -- Create table under database with engine ordinary. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_meta_cache; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; -CREATE DATABASE test_meta_cache ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +CREATE DATABASE test_metadata_cache ENGINE = Ordinary; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert third batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert third batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index b57caf55cb8..7c78721f692 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -1,124 +1,126 @@ +-- Tags: no-fasttest + -- Create table under database with engine atomic. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_meta_cache; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache SYNC; -CREATE DATABASE test_meta_cache ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; +CREATE DATABASE test_metadata_cache ENGINE = Atomic; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + 30; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 30; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + 60; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 60; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache SYNC; -CREATE TABLE test_meta_cache.check_part_meta_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; +CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- nsert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 7722bba71dd..6111f20b599 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,127 +1,128 @@ +-- Tags: no-fasttest -- Tags: zookeeper -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache; -CREATE DATABASE test_meta_cache ENGINE = Ordinary; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +CREATE DATABASE test_metadata_cache ENGINE = Ordinary; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache ; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache ; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index 4dcc4478139..ee8ad61c97c 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,127 +1,128 @@ +-- Tags: no-fasttest -- Tags: zookeeper -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_meta_cache ; -DROP TABLE IF EXISTS test_meta_cache.check_part_meta_cache ; -CREATE DATABASE test_meta_cache ENGINE = Atomic; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_meta_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +DROP DATABASE IF EXISTS test_metadata_cache ; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache ; +CREATE DATABASE test_metadata_cache ENGINE = Atomic; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete column. -alter table test_meta_cache.check_part_meta_cache drop column v1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop column v1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_meta_cache.check_part_meta_cache ; -CREATE TABLE test_meta_cache.check_part_meta_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k; +drop table if exists test_metadata_cache.check_part_metadata_cache ; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Insert second batch of data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Update some data. -alter table test_meta_cache.check_part_meta_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); --Delete some data. -alter table test_meta_cache.check_part_meta_cache delete where k = 1; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -alter table test_meta_cache.check_part_meta_cache delete where k = 8; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Delete some data. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Drop partitioin 201805 -alter table test_meta_cache.check_part_meta_cache drop partition 201805; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Optimize table. -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_meta_cache.check_part_meta_cache FINAL; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +optimize table test_metadata_cache.check_part_metadata_cache FINAL; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add column. -alter table test_meta_cache.check_part_meta_cache add column v3 UInt64; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Add TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Modify TTL info. -alter table test_meta_cache.check_part_meta_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_meta_cache.check_part_meta_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); -- Truncate table. -truncate table test_meta_cache.check_part_meta_cache; -with arrayJoin(checkPartMetaCache('test_meta_cache', 'check_part_meta_cache')) as info select count(1), countIf(info.5 = 0); +truncate table test_metadata_cache.check_part_metadata_cache; +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); From 98f37afc80df33de1d3eb9c1676804c5285ab422 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 19:29:01 +0800 Subject: [PATCH 007/103] fix building --- programs/server/Server.cpp | 2 +- src/Functions/checkPartMetadataCache.cpp | 2 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 41 +++++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++- src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- src/Storages/MergeTree/PartMetadataCache.cpp | 9 ++-- src/Storages/MergeTree/PartMetadataCache.h | 10 ++--- .../StorageSystemMergeTreeMetadataCache.cpp | 3 +- .../StorageSystemMergeTreeMetadataCache.h | 3 +- 14 files changed, 42 insertions(+), 46 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1e10af9dc07..dee8a3ffd4e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -820,7 +820,7 @@ if (ThreadFuzzer::instance().isEffective()) #if USE_ROCKSDB - /// initialize meta file cache + /// Initialize merge tree metadata cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index 2883424f996..87d84297226 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -99,7 +99,7 @@ public: if (!data || !data->getSettings()->use_metadata_cache) throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); - /// Fill in result + /// Fill in checking results. auto col_result = result_type->createColumn(); auto & col_arr = assert_cast(*col_result); auto & col_tuple = assert_cast(col_arr.getData()); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4897cdccac9..76348a16710 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -283,7 +283,7 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; #if USE_ROCKSDB - /// MergeTree metadata cache stored in rocksdb. + /// Global merge tree metadata cache, stored in rocksdb. MergeTreeMetadataCachePtr merge_tree_metadata_cache; #endif @@ -399,7 +399,7 @@ struct ContextSharedPart zookeeper.reset(); #if USE_ROCKSDB - /// Shutdown meta file cache + /// Shutdown merge tree metadata cache if (merge_tree_metadata_cache) { merge_tree_metadata_cache->shutdown(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ea606fdea4e..fe8df22b8fb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -708,7 +708,6 @@ public: MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; #endif - #if USE_NURAFT std::shared_ptr & getKeeperDispatcher() const; #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b29886265f6..35125a7095f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -90,11 +90,11 @@ void IMergeTreeDataPart::MinMaxIndex::load( { std::unique_ptr file; #if USE_ROCKSDB + String _; if (cache) { String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; - String value; - file = cache->readOrSetMeta(disk, file_name, value); + file = cache->readOrSet(disk, file_name, _); } else { @@ -325,7 +325,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) - , use_metadata_cache(storage.getSettings()->use_metadata_cache) + , use_metadata_cache(storage.use_metadata_cache) { if (parent_part) state = State::Committed; @@ -357,7 +357,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) , parent_part(parent_part_) - , use_metadata_cache(storage.getSettings()->use_metadata_cache) + , use_metadata_cache(storage.use_metadata_cache) { if (parent_part) state = State::Committed; @@ -716,7 +716,6 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b size_t IMergeTreeDataPart::fileNumberOfColumnsChecksumsIndexes() const { Strings files; - files.reserve(16); appendFilesOfColumnsChecksumsIndexes(files, true); return files.size(); } @@ -773,10 +772,10 @@ void IMergeTreeDataPart::loadIndex() std::unique_ptr index_file; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String value; - index_file = metadata_cache->readOrSetMeta(volume->getDisk(), "primary.idx", value); + index_file = metadata_cache->readOrSet(volume->getDisk(), "primary.idx", _); } else { @@ -854,10 +853,10 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() bool exists = false; std::unique_ptr file_buf; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - file_buf = metadata_cache->readOrSetMeta(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, v); + file_buf = metadata_cache->readOrSet(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, _); exists = file_buf != nullptr; } else @@ -1027,10 +1026,10 @@ void IMergeTreeDataPart::loadChecksums(bool require) std::unique_ptr buf; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String value; - buf = metadata_cache->readOrSetMeta(volume->getDisk(), "checksums.txt", value); + buf = metadata_cache->readOrSet(volume->getDisk(), "checksums.txt", _); exists = buf != nullptr; } else @@ -1100,10 +1099,10 @@ void IMergeTreeDataPart::loadRowsCount() else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - auto buf = metadata_cache->readOrSetMeta(volume->getDisk(), "count.txt", v); + auto buf = metadata_cache->readOrSet(volume->getDisk(), "count.txt", _); if (!buf) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -1229,10 +1228,10 @@ void IMergeTreeDataPart::loadTTLInfos() std::unique_ptr in; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - in = metadata_cache->readOrSetMeta(volume->getDisk(), "ttl.txt", v); + in = metadata_cache->readOrSet(volume->getDisk(), "ttl.txt", _); exists = in != nullptr; } else @@ -1283,10 +1282,10 @@ void IMergeTreeDataPart::loadUUID() std::unique_ptr in; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - in = metadata_cache->readOrSetMeta(volume->getDisk(), UUID_FILE_NAME, v); + in = metadata_cache->readOrSet(volume->getDisk(), UUID_FILE_NAME, _); exists = in != nullptr; } else @@ -1325,10 +1324,10 @@ void IMergeTreeDataPart::loadColumns(bool require) bool exists = false; std::unique_ptr in; #if USE_ROCKSDB + String _; if (use_metadata_cache) { - String v; - in = metadata_cache->readOrSetMeta(volume->getDisk(), "columns.txt", v); + in = metadata_cache->readOrSet(volume->getDisk(), "columns.txt", _); exists = in != nullptr; } else @@ -1490,10 +1489,10 @@ void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool incl switch (type) { case ModifyCacheType::PUT: - metadata_cache->setMetas(volume->getDisk(), files); + metadata_cache->batchSet(volume->getDisk(), files); break; case ModifyCacheType::DROP: - metadata_cache->dropMetas(files); + metadata_cache->batchDelete(files); break; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6add439d1f9..d03b1e287e5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -206,6 +206,7 @@ MergeTreeData::MergeTreeData( , parts_mover(this) , background_operations_assignee(*this, BackgroundJobsAssignee::Type::DataProcessing, getContext()) , background_moves_assignee(*this, BackgroundJobsAssignee::Type::Moving, getContext()) + , use_metadata_cache(getSettings()->use_metadata_cache) { context_->getGlobalContext()->initializeBackgroundExecutorsIfNeeded(); @@ -318,13 +319,13 @@ MergeTreeData::MergeTreeData( "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); #if !USE_ROCKSDB - if (settings->use_metadata_cache) + if (use_metadata_cache) { LOG_WARNING( log, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." "set use_metadata_cache to false forcely"); - settings->use_metadata_cache = false; + use_metadata_cache = false; } #endif diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d349646ab88..468a5ccf72b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -972,6 +972,7 @@ protected: /// And for ReplicatedMergeTree we don't have LogEntry type for this operation. BackgroundJobsAssignee background_operations_assignee; BackgroundJobsAssignee background_moves_assignee; + bool use_metadata_cache; /// Strongly connected with two fields above. /// Every task that is finished will ask to assign a new one into an executor. diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index fa9996b3382..d294b69f79f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -192,12 +192,11 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } -/// Do not cache mark file, because cache other meta files is enough to speed up loading. +/// We don't cache mark file, because cache other metadata files is enough to speed up loading. void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const { } -/// find all connected file and do modification Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const { auto marks_file = index_granularity_info.getMarksFilePath("data"); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f6efdc5f05c..f4aec7deb15 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -268,7 +268,7 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col } } -// Do not cache mark files of part, because cache other meta files is enough to speed up loading. +/// We don't cache mark files of part, because cache other meta files is enough to speed up loading. void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) const { } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 933248c9a1e..6a394fa5baa 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -373,7 +373,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis if (metadata_cache) { String v; - file = metadata_cache->readOrSetMeta(disk, "partition.dat", v); + file = metadata_cache->readOrSet(disk, "partition.dat", v); } else { diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp index a8ab2c0bf2d..dcb6ce54378 100644 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -24,7 +24,7 @@ namespace DB { std::unique_ptr -PartMetadataCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value) +PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; auto status = cache->get(file_path, value); @@ -50,7 +50,7 @@ PartMetadataCache::readOrSetMeta(const DiskPtr & disk, const String & file_name, return std::make_unique(value); } -void PartMetadataCache::setMetas(const DiskPtr & disk, const Strings & file_names) +void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_names) { String text; String read_value; @@ -76,7 +76,7 @@ void PartMetadataCache::setMetas(const DiskPtr & disk, const Strings & file_name } } -void PartMetadataCache::dropMetas(const Strings & file_names) +void PartMetadataCache::batchDelete(const Strings & file_names) { for (const auto & file_name : file_names) { @@ -93,7 +93,7 @@ void PartMetadataCache::dropMetas(const Strings & file_names) } } -void PartMetadataCache::setMeta(const String & file_name, const String & value) +void PartMetadataCache::set(const String & file_name, const String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; String read_value; @@ -116,7 +116,6 @@ void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vectorgetByPrefix(prefix, files, values); size_t size = files.size(); for (size_t i = 0; i < size; ++i) diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h index 97880f41b6c..91440da014c 100644 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -30,11 +30,11 @@ public: } std::unique_ptr - readOrSetMeta(const DiskPtr & disk, const String & file_name, String & value); - void setMetas(const DiskPtr & disk, const Strings & file_names); - void dropMetas(const Strings & file_names); - void setMeta(const String & file_name, const String & value); - void getFilesAndCheckSums(Strings & file_names, std::vector & checksums) const; + readOrSet(const DiskPtr & disk, const String & file_name, String & value); + void batchSet(const DiskPtr & disk, const Strings & file_names); + void batchDelete(const Strings & file_names); + void set(const String & file_name, const String & value); + void getFilesAndCheckSums(Strings & files, std::vector & checksums) const; private: std::string getFullRelativePath() const; diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index b62a7985c0c..bf74f15e822 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -82,8 +82,7 @@ static bool extractKeyImpl(const IAST & elem, String & res, bool & precise) } -/** Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause. - */ +/// Retrieve from the query a condition of the form `key= 'key'`, from conjunctions in the WHERE clause. static String extractKey(const ASTPtr & query, bool& precise) { const auto & select = query->as(); diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.h b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h index a61f996f4df..8169d1a83fb 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h @@ -12,8 +12,7 @@ namespace DB class Context; -/** Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for debugging purposes. - */ +/// Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for testing purposes. class StorageSystemMergeTreeMetadataCache : public shared_ptr_helper, public IStorageSystemOneBlock { friend struct shared_ptr_helper; From 63dc6821d2d16c7100687ab83e47a70faf0cc5ec Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 29 Dec 2021 12:31:54 +0800 Subject: [PATCH 008/103] fix ut and some bug --- src/Interpreters/Context.cpp | 64 ++--------------- src/Interpreters/Context.h | 34 ++------- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 - .../MergeTree/MergeTreeDataPartCompact.cpp | 6 -- .../MergeTree/MergeTreeDataPartCompact.h | 2 - .../MergeTree/MergeTreeDataPartInMemory.cpp | 6 -- .../MergeTree/MergeTreeDataPartInMemory.h | 2 - .../MergeTree/MergeTreeDataPartWide.cpp | 9 --- .../MergeTree/MergeTreeDataPartWide.h | 2 - .../MergeTree/MergeTreeMetadataCache.cpp | 70 +++++++++++++++++++ .../MergeTree/MergeTreeMetadataCache.h | 39 +++++++++++ src/Storages/MergeTree/MergeTreePartition.cpp | 4 +- src/Storages/MergeTree/PartMetadataCache.cpp | 1 + src/Storages/MergeTree/PartMetadataCache.h | 11 ++- .../StorageSystemMergeTreeMetadataCache.cpp | 1 + .../01233_check_part_meta_cache.reference | 56 +++++++-------- .../01233_check_part_meta_cache.sql | 56 +++++++-------- ..._check_part_meta_cache_in_atomic.reference | 56 +++++++-------- .../01233_check_part_meta_cache_in_atomic.sql | 56 +++++++-------- ...check_part_meta_cache_replicated.reference | 56 +++++++-------- ...01233_check_part_meta_cache_replicated.sql | 56 +++++++-------- ..._meta_cache_replicated_in_atomic.reference | 56 +++++++-------- ...k_part_meta_cache_replicated_in_atomic.sql | 62 ++++++++-------- 24 files changed, 360 insertions(+), 350 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeMetadataCache.cpp create mode 100644 src/Storages/MergeTree/MergeTreeMetadataCache.h diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 76348a16710..3986b5bf822 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -84,24 +84,21 @@ #include #include #include +#include #include #include #include #include +#if USE_ROCKSDB +#include +#endif namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ContextLock; - -#if USE_ROCKSDB - extern const Event MergeTreeMetadataCachePut; - extern const Event MergeTreeMetadataCacheGet; - extern const Event MergeTreeMetadataCacheDelete; - extern const Event MergeTreeMetadataCacheSeek; -#endif } namespace CurrentMetrics @@ -449,59 +446,6 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr shar void SharedContextHolder::reset() { shared.reset(); } -#if USE_ROCKSDB -MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) -{ - auto options = rocksdb::WriteOptions(); - auto status = rocksdb->Put(options, key, value); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); - return status; -} - -MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) -{ - auto options = rocksdb::WriteOptions(); - auto status = rocksdb->Delete(options, key); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); - LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); - return status; -} - -MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value) -{ - auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet); - LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); - return status; -} - -void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) -{ - auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); - rocksdb::Slice target(prefix); - for (it->Seek(target); it->Valid(); it->Next()) - { - const auto key = it->key(); - if (!key.starts_with(target)) - break; - - const auto value = it->value(); - keys.emplace_back(key.data(), key.size()); - values.emplace_back(value.data(), value.size()); - } - LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); -} - -void MergeTreeMetadataCache::shutdown() -{ - if (rocksdb) - { - rocksdb->Close(); - } -} -#endif - ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) { auto res = std::shared_ptr(new Context); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fe8df22b8fb..b8616e8b634 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -16,12 +16,8 @@ #include #include -#include "config_core.h" -#if USE_ROCKSDB -#include -#include -#endif +#include "config_core.h" #include #include @@ -155,6 +151,12 @@ using ReadTaskCallback = std::function; using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; + +#if USE_ROCKSDB +class MergeTreeMetadataCache; +using MergeTreeMetadataCachePtr = std::shared_ptr; +#endif + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -182,28 +184,6 @@ private: std::unique_ptr shared; }; -#if USE_ROCKSDB -class MergeTreeMetadataCache -{ -public: - using Status = rocksdb::Status; - - explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } - MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; - MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; - - Status put(const String & key, const String & value); - Status del(const String & key); - Status get(const String & key, String & value); - void getByPrefix(const String & prefix, Strings & keys, Strings & values); - - void shutdown(); -private: - std::unique_ptr rocksdb; - Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache"); -}; -using MergeTreeMetadataCachePtr = std::shared_ptr; -#endif /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 35125a7095f..abcc52dd295 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -820,8 +820,7 @@ void IMergeTreeDataPart::appendFilesofIndex(Strings & files) const if (!metadata_snapshot) return; - size_t key_size = metadata_snapshot->getPrimaryKeyColumns().size(); - if (key_size) + if (metadata_snapshot->hasPrimaryKey()) files.push_back("primary.idx"); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2798f42b6c4..a992d2dadd7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -386,8 +386,6 @@ public: /// storage and pass it to this method. virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; } - virtual Strings getIndexGranularityFiles() const = 0; - /// Returns true if this part shall participate in merges according to /// settings of given storage policy. bool shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index d294b69f79f..6a747960a40 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -197,10 +197,4 @@ void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files * { } -Strings MergeTreeDataPartCompact::getIndexGranularityFiles() const -{ - auto marks_file = index_granularity_info.getMarksFilePath("data"); - return {marks_file}; -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 87066ab2ff0..b96afc4b972 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -74,8 +74,6 @@ private: void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; void appendFilesOfIndexGranularity(Strings& files) const override; - - Strings getIndexGranularityFiles() const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index f3c4b613078..e482f867e7b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -171,12 +171,6 @@ void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files { } -/// No mark files for part in memory -Strings MergeTreeDataPartInMemory::getIndexGranularityFiles() const -{ - return {}; -} - DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) { return std::dynamic_pointer_cast(part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 4f83b54d402..068b15d2bdc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -64,8 +64,6 @@ private: void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; void appendFilesOfIndexGranularity(Strings & files) const override; - - Strings getIndexGranularityFiles() const override; }; using DataPartInMemoryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f4aec7deb15..1582a7f3274 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -273,13 +273,4 @@ void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) { } -Strings MergeTreeDataPartWide::getIndexGranularityFiles() const -{ - if (columns.empty()) - return {}; - - auto marks_file = getFileNameForColumn(columns.front()); - return {marks_file}; -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index bf73d16d758..2b11fc4eb02 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -68,8 +68,6 @@ private: void appendFilesOfIndexGranularity(Strings & files) const override; - Strings getIndexGranularityFiles() const override; - ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set * processed_substreams) const; void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp new file mode 100644 index 00000000000..d9dacadcead --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -0,0 +1,70 @@ +#include "MergeTreeMetadataCache.h" + +#if USE_ROCKSDB +#include +#include + +namespace ProfileEvents +{ + extern const Event MergeTreeMetadataCachePut; + extern const Event MergeTreeMetadataCacheGet; + extern const Event MergeTreeMetadataCacheDelete; + extern const Event MergeTreeMetadataCacheSeek; +} + +namespace DB +{ +MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Put(options, key, value); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); + return status; +} + +MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) +{ + auto options = rocksdb::WriteOptions(); + auto status = rocksdb->Delete(options, key); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); + LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); + return status; +} + +MergeTreeMetadataCache::Status MergeTreeMetadataCache::get(const String & key, String & value) +{ + auto status = rocksdb->Get(rocksdb::ReadOptions(), key, &value); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheGet); + LOG_TRACE(log, "Get key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); + return status; +} + +void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, Strings & values) +{ + auto * it = rocksdb->NewIterator(rocksdb::ReadOptions()); + rocksdb::Slice target(prefix); + for (it->Seek(target); it->Valid(); it->Next()) + { + const auto key = it->key(); + if (!key.starts_with(target)) + break; + + const auto value = it->value(); + keys.emplace_back(key.data(), key.size()); + values.emplace_back(value.data(), value.size()); + } + LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); +} + +void MergeTreeMetadataCache::shutdown() +{ + if (rocksdb) + { + rocksdb->Close(); + } +} + +} + +#endif diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h new file mode 100644 index 00000000000..00c783d881d --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -0,0 +1,39 @@ +#pragma once + + +#include +#include + +#include +#include +#include + +#include "config_core.h" +#if USE_ROCKSDB + +namespace DB +{ +class MergeTreeMetadataCache +{ +public: + using Status = rocksdb::Status; + + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; + MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; + + Status put(const String & key, const String & value); + Status del(const String & key); + Status get(const String & key, String & value); + void getByPrefix(const String & prefix, Strings & keys, Strings & values); + + void shutdown(); +private: + std::unique_ptr rocksdb; + Poco::Logger * log = &Poco::Logger::get("MergeTreeMetadataCache"); +}; + +using MergeTreeMetadataCachePtr = std::shared_ptr; +} + +#endif diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 6a394fa5baa..f1a669eeb2f 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -370,10 +370,10 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis std::unique_ptr file; #if USE_ROCKSDB + String _; if (metadata_cache) { - String v; - file = metadata_cache->readOrSet(disk, "partition.dat", v); + file = metadata_cache->readOrSet(disk, "partition.dat", _); } else { diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp index dcb6ce54378..eee04d24405 100644 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace ProfileEvents { diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h index 91440da014c..427b9524afd 100644 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -5,7 +5,6 @@ #if USE_ROCKSDB #include #include -#include namespace DB @@ -13,8 +12,12 @@ namespace DB class SeekableReadBuffer; class IMergeTreeDataPart; -class PartMetadataCache; -using PartMetadataCachePtr = std::shared_ptr; + +class MergeTreeMetadataCache; +using MergeTreeMetadataCachePtr = std::shared_ptr; + +class IDisk; +using DiskPtr = std::shared_ptr; class PartMetadataCache { @@ -45,5 +48,7 @@ private: const IMergeTreeDataPart * parent_part; }; +using PartMetadataCachePtr = std::shared_ptr; + } #endif diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index bf74f15e822..275365648f3 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include namespace DB diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.reference b/tests/queries/0_stateless/01233_check_part_meta_cache.reference index 914add905ce..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -63 0 -77 0 -84 0 -98 0 -122 0 -154 0 -122 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index 3c5d55a0069..c15c2883436 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -6,65 +6,65 @@ DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert third batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache; @@ -72,54 +72,54 @@ CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert third batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference index 95de1ef56a9..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -63 0 -77 0 -84 0 -98 0 -124 0 -150 0 -124 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index 7c78721f692..6bd8425a8ea 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -6,65 +6,65 @@ DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 30; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 60; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; @@ -72,55 +72,55 @@ CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- nsert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference index 2275537d212..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -7 0 -14 0 -21 0 -35 0 -51 0 -67 0 -0 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 6111f20b599..2c490c80d70 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -8,65 +8,65 @@ DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache ; @@ -74,55 +74,55 @@ CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference index 2275537d212..2bd6025fea2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference @@ -1,28 +1,28 @@ -0 0 -7 0 -14 0 -28 0 -42 0 -56 0 -70 0 -77 0 -7 0 -14 0 -21 0 -35 0 -51 0 -67 0 -0 0 -12 0 -24 0 -48 0 -72 0 -96 0 -120 0 -132 0 -108 0 -132 0 -144 0 -183 0 -235 0 -183 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index ee8ad61c97c..c7cc2a09899 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -5,124 +5,124 @@ set mutations_sync = 1; set replication_alter_partitions_sync = 2; DROP DATABASE IF EXISTS test_metadata_cache ; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache ; +DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete column. alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache ; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert second batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Update some data. alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); --Delete some data. alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Delete some data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Drop partitioin 201805 alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Optimize table. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add column. alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Add TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Modify TTL info. alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Truncate table. truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select count(1), countIf(info.5 = 0); +with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); From e5ff05c50dc0fac888ae961804c0ace95be59c5f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 29 Dec 2021 16:20:29 +0800 Subject: [PATCH 009/103] fix building --- src/Functions/checkPartMetadataCache.cpp | 1 - src/Storages/MergeTree/MergeTreeMetadataCache.h | 10 ++++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index 87d84297226..a57607d23f6 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -1,7 +1,6 @@ #include "config_core.h" #if USE_ROCKSDB - #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index 00c783d881d..f8d7c52cb06 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -1,15 +1,13 @@ #pragma once +#include "config_core.h" -#include -#include - +#if USE_ROCKSDB #include #include #include - -#include "config_core.h" -#if USE_ROCKSDB +#include +#include namespace DB { From 05dfc4eb25be9d0b761c798968fffc4c15da14c0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 31 Dec 2021 11:13:38 +0800 Subject: [PATCH 010/103] fix building and stateless test --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 13 ++++++------- src/Storages/MergeTree/IMergeTreeDataPart.h | 12 ++++++------ .../0_stateless/01233_check_part_meta_cache.sql | 2 +- .../01233_check_part_meta_cache_in_atomic.sql | 2 +- .../01233_check_part_meta_cache_replicated.sql | 2 +- ...3_check_part_meta_cache_replicated_in_atomic.sql | 2 +- 6 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index abcc52dd295..83e91e7ad27 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -907,7 +907,7 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() } } -void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) const +void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) { files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME); } @@ -1075,7 +1075,7 @@ void IMergeTreeDataPart::loadChecksums(bool require) } } -void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files) const +void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files) { files.push_back("checksums.txt"); } @@ -1215,7 +1215,7 @@ void IMergeTreeDataPart::loadRowsCount() } } -void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) const +void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) { files.push_back("count.txt"); } @@ -1269,7 +1269,7 @@ void IMergeTreeDataPart::loadTTLInfos() } -void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) const +void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) { files.push_back("ttl.txt"); } @@ -1307,7 +1307,7 @@ void IMergeTreeDataPart::loadUUID() } } -void IMergeTreeDataPart::appendFilesOfUUID(Strings & files) const +void IMergeTreeDataPart::appendFilesOfUUID(Strings & files) { files.push_back(UUID_FILE_NAME); } @@ -1388,7 +1388,7 @@ void IMergeTreeDataPart::loadColumns(bool require) setColumns(loaded_columns, infos); } -void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) const +void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) { files.push_back("columns.txt"); } @@ -1475,7 +1475,6 @@ void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool incl assert(use_metadata_cache); Strings files; - files.reserve(16); appendFilesOfColumnsChecksumsIndexes(files, include_projection); LOG_TRACE( storage.log, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a992d2dadd7..1457a643b18 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -503,17 +503,17 @@ private: /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); - void appendFilesOfUUID(Strings & files) const; + static void appendFilesOfUUID(Strings & files); /// Reads columns names and types from columns.txt void loadColumns(bool require); - void appendFilesOfColumns(Strings & files) const; + static void appendFilesOfColumns(Strings & files); /// If checksums.txt exists, reads file's checksums (and sizes) from it void loadChecksums(bool require); - void appendFilesOfChecksums(Strings & files) const; + static void appendFilesOfChecksums(Strings & files); /// Loads marks index granularity into memory virtual void loadIndexGranularity(); @@ -529,12 +529,12 @@ private: /// For the older format version calculates rows count from the size of a column with a fixed size. void loadRowsCount(); - void appendFilesOfRowsCount(Strings & files) const; + static void appendFilesOfRowsCount(Strings & files); /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); - void appendFilesOfTTLInfos(Strings & files) const; + static void appendFilesOfTTLInfos(Strings & files); void loadPartitionAndMinMaxIndex(); @@ -549,7 +549,7 @@ private: /// any specifial compression. void loadDefaultCompressionCodec(); - void appendFilesOfDefaultCompressionCodec(Strings & files) const; + static void appendFilesOfDefaultCompressionCodec(Strings & files); /// Found column without specific compression and return codec /// for this column with default parameters. diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index c15c2883436..70de4e0de9e 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -2,8 +2,8 @@ -- Create table under database with engine ordinary. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index 6bd8425a8ea..61452368b52 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -2,8 +2,8 @@ -- Create table under database with engine atomic. set mutations_sync = 1; -DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 2c490c80d70..5aff175f97e 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -4,8 +4,8 @@ -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_metadata_cache; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index c7cc2a09899..7a7846bab05 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -4,8 +4,8 @@ -- Create table under database with engine ordinary. set mutations_sync = 1; set replication_alter_partitions_sync = 2; -DROP DATABASE IF EXISTS test_metadata_cache ; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; +DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Atomic; CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); From 770ff591552bcc47461da06ef00650eda138351d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 31 Dec 2021 13:32:00 +0800 Subject: [PATCH 011/103] fix building --- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- src/Storages/MergeTree/MergeTreePartition.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index f1a669eeb2f..8a85e374062 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -465,7 +465,7 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & } -void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) const +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 644ddd5ba88..b1bf64550c6 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -54,7 +54,7 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); - void appendFiles(const MergeTreeData & storage, Strings & files) const; + static void appendFiles(const MergeTreeData & storage, Strings & files); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); From e3da4f07a9779308a6d9eef4fabde3f681f3f4fe Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 1 Jan 2022 08:36:54 +0800 Subject: [PATCH 012/103] fix stateless test --- tests/queries/0_stateless/01233_check_part_meta_cache.sql | 2 +- .../0_stateless/01233_check_part_meta_cache_in_atomic.sql | 2 +- .../0_stateless/01233_check_part_meta_cache_replicated.sql | 7 +++---- .../01233_check_part_meta_cache_replicated_in_atomic.sql | 7 +++---- 4 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql index 70de4e0de9e..6c1a1232cab 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache.sql @@ -1,4 +1,4 @@ ---Tags: no-fasttest +-- Tags: no-parallel, no-fasttest -- Create table under database with engine ordinary. set mutations_sync = 1; diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql index 61452368b52..af8d6f888a7 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-parallel, no-fasttest -- Create table under database with engine atomic. set mutations_sync = 1; diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 5aff175f97e..955c9b49957 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- Tags: zookeeper +-- Tags: no-fasttest, no-parallel, zookeeper -- Create table under database with engine ordinary. set mutations_sync = 1; @@ -7,7 +6,7 @@ set replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Ordinary; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. @@ -70,7 +69,7 @@ with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadat -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache ; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index 7a7846bab05..6943f721f70 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- Tags: zookeeper +-- Tags: no-fasttest, zookeeper, no-parallel -- Create table under database with engine ordinary. set mutations_sync = 1; @@ -7,7 +6,7 @@ set replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; DROP DATABASE IF EXISTS test_metadata_cache; CREATE DATABASE test_metadata_cache ENGINE = Atomic; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); -- Insert first batch of data. @@ -70,7 +69,7 @@ with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadat -- Recreate table with projection. drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; +CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; -- Insert first batch of data. INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); From eaf5a72fb79cfbdfb0f4fa227777deb9b5781ee1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 1 Jan 2022 09:06:08 +0800 Subject: [PATCH 013/103] fix stateless test --- .../0_stateless/01233_check_part_meta_cache_replicated.sql | 1 + .../01233_check_part_meta_cache_replicated_in_atomic.sql | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql index 955c9b49957..bdb43c4e905 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql @@ -1,4 +1,5 @@ -- Tags: no-fasttest, no-parallel, zookeeper +-- Tag no-parallel: static zk path -- Create table under database with engine ordinary. set mutations_sync = 1; diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql index 6943f721f70..4e491c301f2 100644 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql @@ -1,4 +1,5 @@ -- Tags: no-fasttest, zookeeper, no-parallel +-- Tag no-parallel: static zk path -- Create table under database with engine ordinary. set mutations_sync = 1; From e13e1f5d7e164aec7b275d4bd1b883d0c2d1cbf4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 11:44:10 +0800 Subject: [PATCH 014/103] add unit test && use enum_name --- programs/server/Server.cpp | 12 ++- src/Functions/checkPartMetadataCache.cpp | 4 +- src/Interpreters/Context.cpp | 7 +- .../tests/gtest_merge_tree_metadata_cache.cpp | 80 +++++++++++++++++++ src/Processors/examples/CMakeLists.txt | 4 - .../examples/merge_tree_metadata_cache.cpp | 51 ------------ src/Storages/MergeTree/IMergeTreeDataPart.h | 17 ++-- 7 files changed, 102 insertions(+), 73 deletions(-) create mode 100644 src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp delete mode 100644 src/Processors/examples/merge_tree_metadata_cache.cpp diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f4d388d559d..ed863271e11 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -823,7 +823,17 @@ if (ThreadFuzzer::instance().isEffective()) /// Initialize merge tree metadata cache { size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); - global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + + try + { + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + } + catch (...) + { + /// Rename rocksdb directory and reinitialize merge tree metadata cache + fs::rename(path / "rocksdb", path / "rocksdb.old"); + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + } } #endif diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index a57607d23f6..ddcb44f5372 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -39,9 +39,9 @@ public: static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } static constexpr DataPartStates part_states - = {DataPartState::Committed, + = {DataPartState::Active, DataPartState::Temporary, - DataPartState::PreCommitted, + DataPartState::PreActive, DataPartState::Outdated, DataPartState::Deleting, DataPartState::DeleteOnDestroy}; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3986b5bf822..0cb81ba4056 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2289,11 +2289,10 @@ void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) table_options.block_cache = cache; options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); + if (status != rocksdb::Status::OK()) - { - String message = "Fail to open rocksdb path at: " + dir + " status:" + status.ToString(); - throw Exception(message, ErrorCodes::SYSTEM_ERROR); - } + throw Exception(ErrorCodes::SYSTEM_ERROR, "Fail to open rocksdb path at: {} status:{}", dir, status.ToString()); + shared->merge_tree_metadata_cache = std::make_shared(db); } #endif diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp new file mode 100644 index 00000000000..0679cbf529a --- /dev/null +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -0,0 +1,80 @@ +#include + +#if USE_ROCKSDB +#include +#include +#include +#include + +using namespace DB; + +class MergeTreeMetadataCacheTest : public ::testing::Test +{ +public: + void SetUp() override + { + auto shared_context = Context::createShared(); + global_context = Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); + cache = global_context->getMergeTreeMetadataCache(); + } + + void TearDown() override + { + global_context->shutdown(); + } + + ContextMutablePtr global_context; + MergeTreeMetadataCachePtr cache; +}; + +TEST_F(MergeTreeMetadataCacheTest, testCommon) +{ + std::vector files + = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; + String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/"; + + for (const auto & file : files) + { + auto status = cache->put(prefix + file, prefix + file); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk); + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk); + ASSERT_EQ(value, prefix + file); + } + + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), files.size()); + ASSERT_EQ(values.size(), files.size()); + for (size_t i=0; i < files.size(); ++i) + { + ASSERT_EQ(values[i], prefix + keys[i]); + } + + for (const auto & file : files) + { + auto status = cache->del(prefix + file); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kOk); + } + + for (const auto & file : files) + { + String value; + auto status = cache->get(prefix + file, value); + ASSERT_EQ(status.code(), rocksdb::Status::Code::kNotFound); + } + + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), 0); + ASSERT_EQ(values.size(), 0); +} + +#endif diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 2b6b9128e4c..e69de29bb2d 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,4 +0,0 @@ -if (USE_ROCKSDB) - add_executable (merge_tree_metadata_cache merge_tree_metadata_cache.cpp) - target_link_libraries (merge_tree_metadata_cache PRIVATE dbms) -endif() \ No newline at end of file diff --git a/src/Processors/examples/merge_tree_metadata_cache.cpp b/src/Processors/examples/merge_tree_metadata_cache.cpp deleted file mode 100644 index c726eb7ce5a..00000000000 --- a/src/Processors/examples/merge_tree_metadata_cache.cpp +++ /dev/null @@ -1,51 +0,0 @@ -#include -#include - -int main() -{ - using namespace DB; - auto shared_context = Context::createShared(); - auto global_context = Context::createGlobal(shared_context.get()); - global_context->makeGlobalContext(); - global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - - auto cache = global_context->getMergeTreeMetadataCache(); - - std::vector files - = {"columns.txt", "checksums.txt", "primary.idx", "count.txt", "partition.dat", "minmax_p.idx", "default_compression_codec.txt"}; - String prefix = "data/test_metadata_cache/check_part_metadata_cache/201806_1_1_0_4/"; - - for (const auto & file : files) - { - auto status = cache->put(prefix + file, prefix + file); - std::cout << "put " << file << " " << status.ToString() << std::endl; - } - - for (const auto & file : files) - { - String value; - auto status = cache->get(prefix + file, value); - std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; - } - - - for (const auto & file : files) - { - auto status = cache->del(prefix + file); - std::cout << "del " << file << " " << status.ToString() << std::endl; - } - - for (const auto & file : files) - { - String value; - auto status = cache->get(prefix + file, value); - std::cout << "get " << file << " " << status.ToString() << " " << value << std::endl; - } - - Strings keys; - Strings values; - cache->getByPrefix(prefix, keys, values); - for (size_t i=0; i #include #include #include @@ -47,21 +48,15 @@ class IMergeTreeDataPart : public std::enable_shared_from_this Date: Tue, 4 Jan 2022 11:48:07 +0800 Subject: [PATCH 015/103] remove redudant judge --- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 5 +---- src/Storages/MergeTree/MergeTreeMetadataCache.h | 6 +++++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index d9dacadcead..7025a79018b 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -59,10 +59,7 @@ void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, void MergeTreeMetadataCache::shutdown() { - if (rocksdb) - { - rocksdb->Close(); - } + rocksdb->Close(); } } diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index f8d7c52cb06..286c7ebb08e 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -16,7 +16,11 @@ class MergeTreeMetadataCache public: using Status = rocksdb::Status; - explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { } + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} + { + assert(rocksdb); + } + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; From 92b3f198aa6932b1d7205c727ed7a0828fbe1e59 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 11:52:15 +0800 Subject: [PATCH 016/103] throw exception if clickhouse use merge tree metdata cache but not compiled with rocksdb --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fb04d406c98..d9d9388bdc2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -321,11 +321,7 @@ MergeTreeData::MergeTreeData( #if !USE_ROCKSDB if (use_metadata_cache) { - LOG_WARNING( - log, - "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." - "set use_metadata_cache to false forcely"); - use_metadata_cache = false; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." "set use_metadata_cache to false forcely"); } #endif From 94d1f7ccb1bc76c96d765f112286ecda1fafe0fa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 12:36:04 +0800 Subject: [PATCH 017/103] enable continue if cache data corrupted --- programs/server/Server.cpp | 23 +++++++++++++++-------- programs/server/config.xml | 7 +++++-- src/Core/SettingsEnums.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++---- 4 files changed, 25 insertions(+), 15 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ed863271e11..8641623ac07 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -812,17 +812,16 @@ if (ThreadFuzzer::instance().isEffective()) /// Directory with metadata of tables, which was marked as dropped by Atomic database fs::create_directories(path / "metadata_dropped/"); - -#if USE_ROCKSDB - fs::create_directories(path / "rocksdb/"); -#endif } #if USE_ROCKSDB /// Initialize merge tree metadata cache + if (config().has("merge_tree_metadata_cache")) { - size_t size = config().getUInt64("meta_file_cache_size", 256 << 20); + fs::create_directories(path / "rocksdb/"); + size_t size = config().getUInt64("merge_tree_metadata_cache.lru_cache_size", 256 << 20); + bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false); try { @@ -830,9 +829,17 @@ if (ThreadFuzzer::instance().isEffective()) } catch (...) { - /// Rename rocksdb directory and reinitialize merge tree metadata cache - fs::rename(path / "rocksdb", path / "rocksdb.old"); - global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + if (continue_if_corrupted) + { + /// Rename rocksdb directory and reinitialize merge tree metadata cache + time_t now = time(nullptr); + fs::rename(path / "rocksdb", path / ("rocksdb.old." + std::to_string(now))); + global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); + } + else + { + throw; + } } } #endif diff --git a/programs/server/config.xml b/programs/server/config.xml index 470c4dfa35f..c5f8e7eeb94 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1292,6 +1292,9 @@ --> - - 268435456 + + diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 106589f5d24..5d2640da319 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -171,5 +171,4 @@ DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) - } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d9d9388bdc2..4e0a0a162f6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -318,11 +318,12 @@ MergeTreeData::MergeTreeData( LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); -#if !USE_ROCKSDB if (use_metadata_cache) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb." "set use_metadata_cache to false forcely"); - } +#if USE_ROCKSDB + if (!getContext()->getMergeTreeMetadataCache()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if not config in config.xml"); +#else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb"); #endif common_assignee_trigger = [this] (bool delay) noexcept From 37c7c282d76ef77f204528bb0779ba17685e6fca Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 13:41:11 +0800 Subject: [PATCH 018/103] fix unit test --- src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp | 3 ++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 4 ++++ 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp index 0679cbf529a..8371d7ff6b9 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -13,7 +13,7 @@ class MergeTreeMetadataCacheTest : public ::testing::Test public: void SetUp() override { - auto shared_context = Context::createShared(); + shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); @@ -25,6 +25,7 @@ public: global_context->shutdown(); } + SharedContextHolder shared_context; ContextMutablePtr global_context; MergeTreeMetadataCachePtr cache; }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c0672f9e98f..2e71c22e144 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -700,7 +700,7 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b appendFilesOfColumns(files); appendFilesOfChecksums(files); appendFilesOfIndexGranularity(files); - appendFilesofIndex(files); + appendFilesOfIndex(files); appendFilesOfRowsCount(files); appendFilesOfPartitionAndMinMaxIndex(files); appendFilesOfTTLInfos(files); @@ -817,7 +817,7 @@ void IMergeTreeDataPart::loadIndex() } } -void IMergeTreeDataPart::appendFilesofIndex(Strings & files) const +void IMergeTreeDataPart::appendFilesOfIndex(Strings & files) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (parent_part) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 150c258a4b6..0463d44abee 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -518,7 +518,7 @@ private: /// Loads index file. void loadIndex(); - void appendFilesofIndex(Strings & files) const; + void appendFilesOfIndex(Strings & files) const; /// Load rows count for this part from disk (for the newer storage format version). /// For the older format version calculates rows count from the size of a column with a fixed size. diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 7025a79018b..fa3825fd3be 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -17,6 +17,8 @@ namespace DB MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); + options.sync = true; + options.disableWAL = false; auto status = rocksdb->Put(options, key, value); ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCachePut); return status; @@ -25,6 +27,8 @@ MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, c MergeTreeMetadataCache::Status MergeTreeMetadataCache::del(const String & key) { auto options = rocksdb::WriteOptions(); + options.sync = true; + options.disableWAL = false; auto status = rocksdb->Delete(options, key); ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheDelete); LOG_TRACE(log, "Delete key:{} from MergeTreeMetadataCache status:{}", key, status.ToString()); From abefefc719712141d394b301c02b52774370655b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 18:17:23 +0800 Subject: [PATCH 019/103] support multi disk --- .../tests/gtest_merge_tree_metadata_cache.cpp | 31 ++++++++++--------- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +-- src/Storages/MergeTree/PartMetadataCache.cpp | 31 ++++++++++++------- src/Storages/MergeTree/PartMetadataCache.h | 16 +++++++--- 4 files changed, 51 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp index 8371d7ff6b9..1005739098c 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -20,10 +20,7 @@ public: cache = global_context->getMergeTreeMetadataCache(); } - void TearDown() override - { - global_context->shutdown(); - } + void TearDown() override { global_context->shutdown(); } SharedContextHolder shared_context; ContextMutablePtr global_context; @@ -50,14 +47,16 @@ TEST_F(MergeTreeMetadataCacheTest, testCommon) ASSERT_EQ(value, prefix + file); } - Strings keys; - Strings values; - cache->getByPrefix(prefix, keys, values); - ASSERT_EQ(keys.size(), files.size()); - ASSERT_EQ(values.size(), files.size()); - for (size_t i=0; i < files.size(); ++i) { - ASSERT_EQ(values[i], prefix + keys[i]); + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), files.size()); + ASSERT_EQ(values.size(), files.size()); + for (size_t i = 0; i < files.size(); ++i) + { + ASSERT_EQ(values[i], keys[i]); + } } for (const auto & file : files) @@ -73,9 +72,13 @@ TEST_F(MergeTreeMetadataCacheTest, testCommon) ASSERT_EQ(status.code(), rocksdb::Status::Code::kNotFound); } - cache->getByPrefix(prefix, keys, values); - ASSERT_EQ(keys.size(), 0); - ASSERT_EQ(values.size(), 0); + { + Strings keys; + Strings values; + cache->getByPrefix(prefix, keys, values); + ASSERT_EQ(keys.size(), 0); + ASSERT_EQ(values.size(), 0); + } } #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 2e71c22e144..c49232397c7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -343,7 +343,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( #if USE_ROCKSDB if (use_metadata_cache) metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); + storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getPath(), storage.relative_data_path, relative_path, parent_part); #endif } @@ -375,7 +375,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( #if USE_ROCKSDB if (use_metadata_cache) metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), storage.relative_data_path, relative_path, parent_part); + storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getName(), storage.relative_data_path, relative_path, parent_part); #endif } diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp index eee04d24405..eff6d9cc0a9 100644 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ b/src/Storages/MergeTree/PartMetadataCache.cpp @@ -28,7 +28,8 @@ std::unique_ptr PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; - auto status = cache->get(file_path, value); + String key = getKey(file_path); + auto status = cache->get(key, value); if (!status.ok()) { ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); @@ -41,7 +42,7 @@ PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, Str if (in) { readStringUntilEOF(value, *in); - cache->put(file_path, value); + cache->put(key, value); } } else @@ -57,7 +58,8 @@ void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_name String read_value; for (const auto & file_name : file_names) { - const String file_path = fs::path(getFullRelativePath()) / file_name; + String file_path = fs::path(getFullRelativePath()) / file_name; + String key = getKey(file_path); if (!disk->exists(file_path)) continue; @@ -66,10 +68,10 @@ void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_name continue; readStringUntilEOF(text, *in); - auto status = cache->put(file_path, text); + auto status = cache->put(key, text); if (!status.ok()) { - status = cache->get(file_path, read_value); + status = cache->get(key, read_value); if (status.IsNotFound() || read_value == text) continue; throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); @@ -82,11 +84,12 @@ void PartMetadataCache::batchDelete(const Strings & file_names) for (const auto & file_name : file_names) { String file_path = fs::path(getFullRelativePath()) / file_name; - auto status = cache->del(file_path); + String key = getKey(file_path); + auto status = cache->del(key); if (!status.ok()) { String read_value; - status = cache->get(file_path, read_value); + status = cache->get(key, read_value); if (status.IsNotFound()) continue; throw Exception(ErrorCodes::LOGICAL_ERROR, "drop meta failed status:{}, file_path:{}", status.ToString(), file_path); @@ -97,15 +100,16 @@ void PartMetadataCache::batchDelete(const Strings & file_names) void PartMetadataCache::set(const String & file_name, const String & value) { String file_path = fs::path(getFullRelativePath()) / file_name; + String key = getKey(file_path); String read_value; - auto status = cache->get(file_path, read_value); + auto status = cache->get(key, read_value); if (status == rocksdb::Status::OK() && value == read_value) return; - status = cache->put(file_path, value); + status = cache->put(key, value); if (!status.ok()) { - status = cache->get(file_path, read_value); + status = cache->get(key, read_value); if (status.IsNotFound() || read_value == value) return; @@ -115,7 +119,7 @@ void PartMetadataCache::set(const String & file_name, const String & value) void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const { - String prefix = fs::path(getFullRelativePath()) / ""; + String prefix = getKey(fs::path(getFullRelativePath()) / ""); Strings values; cache->getByPrefix(prefix, files, values); size_t size = files.size(); @@ -132,5 +136,10 @@ String PartMetadataCache::getFullRelativePath() const return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; } +String PartMetadataCache::getKey(const String & file_path) const +{ + return disk_name + ":" + file_path; +} + } #endif diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h index 427b9524afd..c2904738b9c 100644 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ b/src/Storages/MergeTree/PartMetadataCache.h @@ -24,8 +24,14 @@ class PartMetadataCache public: using uint128 = CityHash_v1_0_2::uint128; - PartMetadataCache(const MergeTreeMetadataCachePtr & cache_, const String & relative_data_path_, const String & relative_path_, const IMergeTreeDataPart * parent_part_) + PartMetadataCache( + const MergeTreeMetadataCachePtr & cache_, + const String & disk_name_, + const String & relative_data_path_, + const String & relative_path_, + const IMergeTreeDataPart * parent_part_) : cache(cache_) + , disk_name(disk_name_) , relative_data_path(relative_data_path_) , relative_path(relative_path_) , parent_part(parent_part_) @@ -40,11 +46,13 @@ public: void getFilesAndCheckSums(Strings & files, std::vector & checksums) const; private: - std::string getFullRelativePath() const; + String getFullRelativePath() const; + String getKey(const String & file_path) const; MergeTreeMetadataCachePtr cache; - const String & relative_data_path; // relative path of table to disk - const String & relative_path; // relative path of part to table + const String & disk_name; + const String & relative_data_path; /// Relative path of table to disk + const String & relative_path; /// Relative path of part to table const IMergeTreeDataPart * parent_part; }; From c0a9c2b9161a994a0fc26e927c8d09ac8c0d596c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jan 2022 19:51:50 +0800 Subject: [PATCH 020/103] refactor metadatacache to reduce using of USE_ROCKSDB --- src/Functions/checkPartMetadataCache.cpp | 2 +- src/IO/CMakeLists.txt | 4 +- src/IO/ReadBufferFromString.h | 7 + src/IO/examples/read_buffer.cpp | 31 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 349 +++--------------- src/Storages/MergeTree/IMergeTreeDataPart.h | 46 +-- .../MergeTree/IPartMetadataManager.cpp | 11 + src/Storages/MergeTree/IPartMetadataManager.h | 42 +++ src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/MergeTree/MergeTreePartition.cpp | 15 +- src/Storages/MergeTree/MergeTreePartition.h | 11 +- src/Storages/MergeTree/PartMetadataCache.cpp | 145 -------- src/Storages/MergeTree/PartMetadataCache.h | 62 ---- .../MergeTree/PartMetadataManagerOrdinary.cpp | 33 ++ .../MergeTree/PartMetadataManagerOrdinary.h | 28 ++ .../PartMetadataManagerWithCache.cpp | 194 ++++++++++ .../MergeTree/PartMetadataManagerWithCache.h | 40 ++ 17 files changed, 451 insertions(+), 570 deletions(-) create mode 100644 src/Storages/MergeTree/IPartMetadataManager.cpp create mode 100644 src/Storages/MergeTree/IPartMetadataManager.h delete mode 100644 src/Storages/MergeTree/PartMetadataCache.cpp delete mode 100644 src/Storages/MergeTree/PartMetadataCache.h create mode 100644 src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp create mode 100644 src/Storages/MergeTree/PartMetadataManagerOrdinary.h create mode 100644 src/Storages/MergeTree/PartMetadataManagerWithCache.cpp create mode 100644 src/Storages/MergeTree/PartMetadataManagerWithCache.h diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp index ddcb44f5372..d65c14095f7 100644 --- a/src/Functions/checkPartMetadataCache.cpp +++ b/src/Functions/checkPartMetadataCache.cpp @@ -134,7 +134,7 @@ public: cache_checksums.reserve(file_number); disk_checksums.reserve(file_number); - part->checkMetadataCache(keys, cache_checksums, disk_checksums); + // part->checkMetadataCache(keys, cache_checksums, disk_checksums); for (size_t i = 0; i < keys.size(); ++i) { col_key.insert(keys[i]); diff --git a/src/IO/CMakeLists.txt b/src/IO/CMakeLists.txt index f676f415eea..970602896c9 100644 --- a/src/IO/CMakeLists.txt +++ b/src/IO/CMakeLists.txt @@ -1,3 +1,3 @@ -if (ENABLE_EXAMPLES) +#if (ENABLE_EXAMPLES) add_subdirectory (examples) -endif () +#endif () diff --git a/src/IO/ReadBufferFromString.h b/src/IO/ReadBufferFromString.h index 09646e9b41f..7ea6afc3543 100644 --- a/src/IO/ReadBufferFromString.h +++ b/src/IO/ReadBufferFromString.h @@ -15,4 +15,11 @@ public: explicit ReadBufferFromString(std::string_view s) : ReadBufferFromMemory(s.data(), s.size()) {} }; + +class ReadBufferFromOwnString : public String, public ReadBufferFromString +{ +public: + explicit ReadBufferFromOwnString(const String & s_): String(s_), ReadBufferFromString(*this) {} +}; + } diff --git a/src/IO/examples/read_buffer.cpp b/src/IO/examples/read_buffer.cpp index ea3da690ca5..85675c0d613 100644 --- a/src/IO/examples/read_buffer.cpp +++ b/src/IO/examples/read_buffer.cpp @@ -2,18 +2,15 @@ #include -#include -#include +#include #include +#include +#include - -int main(int, char **) +int readAndPrint(DB::ReadBuffer & in) { try { - std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; - DB::ReadBufferFromString in(s); - DB::Int64 a; DB::Float64 b; DB::String c, d; @@ -31,12 +28,32 @@ int main(int, char **) std::cout << a << ' ' << b << ' ' << c << '\t' << '\'' << d << '\'' << std::endl; std::cout << in.count() << std::endl; + return 0; } catch (const DB::Exception & e) { std::cerr << e.what() << ", " << e.displayText() << std::endl; return 1; } +} + +int main(int, char **) +{ + { + std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + DB::ReadBufferFromString in(s); + if (readAndPrint(in)) + std::cout << "readAndPrint from ReadBufferFromString failed" << std::endl; + } + + + std::shared_ptr in; + { + std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + in = std::make_shared(s); + } + if (readAndPrint(*in)) + std::cout << "readAndPrint from ReadBufferFromOwnString failed" << std::endl; return 0; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c49232397c7..4c88580e069 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include #include #include @@ -59,19 +61,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) -{ - size_t file_size = disk->getFileSize(path); - return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); -} - -#if USE_ROCKSDB void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const PartMetadataCachePtr & cache, const DiskPtr & disk, const String & part_path) -#else -void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const DiskPtr & disk, const String & part_path) -#endif + const MergeTreeData & data, const PartMetadataManagerPtr & manager) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -80,31 +71,11 @@ void IMergeTreeDataPart::MinMaxIndex::load( auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); - auto read_min_max_index = [&](size_t i) - { - String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); - auto file = openForReading(disk, file_name); - return file; - }; - hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - std::unique_ptr file; -#if USE_ROCKSDB - String _; - if (cache) - { - String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; - file = cache->readOrSet(disk, file_name, _); - } - else - { - file = read_min_max_index(i); - } -#else - file = read_min_max_index(i); -#endif + String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + auto file = manager->read(file_name); auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; @@ -340,11 +311,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( minmax_idx = std::make_shared(); -#if USE_ROCKSDB - if (use_metadata_cache) - metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getPath(), storage.relative_data_path, relative_path, parent_part); -#endif + initializePartMetadataManager(); } IMergeTreeDataPart::IMergeTreeDataPart( @@ -371,12 +338,8 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); - -#if USE_ROCKSDB - if (use_metadata_cache) - metadata_cache = std::make_shared( - storage.getContext()->getMergeTreeMetadataCache(), volume->getDisk()->getName(), storage.relative_data_path, relative_path, parent_part); -#endif + + initializePartMetadataManager(); } IMergeTreeDataPart::~IMergeTreeDataPart() @@ -774,23 +737,9 @@ void IMergeTreeDataPart::loadIndex() loaded_index[i]->reserve(index_granularity.getMarksCount()); } - String index_path = fs::path(getFullRelativePath()) / "primary.idx"; - - std::unique_ptr index_file; -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - index_file = metadata_cache->readOrSet(volume->getDisk(), "primary.idx", _); - } - else - { - index_file = openForReading(volume->getDisk(), index_path); - } -#else - index_file = openForReading(volume->getDisk(), index_path); -#endif - + String index_name = "primary.idx"; + String index_path = fs::path(getFullRelativePath()) / index_name; + auto index_file = metadata_manager->read(index_name); size_t marks_count = index_granularity.getMarksCount(); Serializations key_serializations(key_size); @@ -854,34 +803,14 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() } String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; - - bool exists = false; - std::unique_ptr file_buf; -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - file_buf = metadata_cache->readOrSet(volume->getDisk(), DEFAULT_COMPRESSION_CODEC_FILE_NAME, _); - exists = file_buf != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - file_buf = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - file_buf = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists(DEFAULT_COMPRESSION_CODEC_FILE_NAME); if (!exists) { default_codec = detectDefaultCompressionCodec(); } else { + auto file_buf = metadata_manager->read(DEFAULT_COMPRESSION_CODEC_FILE_NAME); String codec_line; readEscapedStringUntilEOL(codec_line, *file_buf); @@ -980,11 +909,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() { String path = getFullRelativePath(); if (!parent_part) -#if USE_ROCKSDB - partition.load(storage, metadata_cache, volume->getDisk(), path); -#else - partition.load(storage, volume->getDisk(), path); -#endif + partition.load(storage, metadata_manager); if (!isEmpty()) { @@ -992,11 +917,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() // projection parts don't have minmax_idx, and it's always initialized minmax_idx->initialized = true; else -#if USE_ROCKSDB - minmax_idx->load(storage, metadata_cache, volume->getDisk(), path); -#else - minmax_idx->load(storage, volume->getDisk(), path); -#endif + minmax_idx->load(storage, metadata_manager); } if (parent_part) return; @@ -1027,30 +948,10 @@ void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) c void IMergeTreeDataPart::loadChecksums(bool require) { const String path = fs::path(getFullRelativePath()) / "checksums.txt"; - bool exists = false; - std::unique_ptr buf; - -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - buf = metadata_cache->readOrSet(volume->getDisk(), "checksums.txt", _); - exists = buf != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - buf = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - buf = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists("checksums.txt"); if (exists) { + auto buf = metadata_manager->read("checksums.txt"); if (checksums.read(*buf)) { assertEOF(*buf); @@ -1092,7 +993,8 @@ void IMergeTreeDataPart::loadRowsCount() auto read_rows_count = [&]() { - auto buf = openForReading(volume->getDisk(), path); + // auto buf = openForReading(volume->getDisk(), path); + auto buf = metadata_manager->read("count.txt"); readIntText(rows_count, *buf); assertEOF(*buf); }; @@ -1103,30 +1005,11 @@ void IMergeTreeDataPart::loadRowsCount() } else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part) { -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - auto buf = metadata_cache->readOrSet(volume->getDisk(), "count.txt", _); - if (!buf) - throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - - readIntText(rows_count, *buf); - assertEOF(*buf); - } - else - { - if (!volume->getDisk()->exists(path)) - throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - - read_rows_count(); - } -#else - if (!volume->getDisk()->exists(path)) + bool exists = metadata_manager->exists("count.txt"); + if (!exists) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); read_rows_count(); -#endif #ifndef NDEBUG /// columns have to be loaded @@ -1228,31 +1111,10 @@ void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files) void IMergeTreeDataPart::loadTTLInfos() { - String path = fs::path(getFullRelativePath()) / "ttl.txt"; - bool exists = false; - std::unique_ptr in; - -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - in = metadata_cache->readOrSet(volume->getDisk(), "ttl.txt", _); - exists = in != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists("ttl.txt"); if (exists) { + auto in = metadata_manager->read("ttl.txt"); assertString("ttl format version: ", *in); size_t format_version; readText(format_version, *in); @@ -1282,31 +1144,10 @@ void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files) void IMergeTreeDataPart::loadUUID() { - String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; - bool exists = false; - std::unique_ptr in; - -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - in = metadata_cache->readOrSet(volume->getDisk(), UUID_FILE_NAME, _); - exists = in != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists(UUID_FILE_NAME); if (exists) { + auto in = metadata_manager->read(UUID_FILE_NAME); readText(uuid, *in); if (uuid == UUIDHelpers::Nil) throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR); @@ -1326,27 +1167,7 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; - bool exists = false; - std::unique_ptr in; -#if USE_ROCKSDB - String _; - if (use_metadata_cache) - { - in = metadata_cache->readOrSet(volume->getDisk(), "columns.txt", _); - exists = in != nullptr; - } - else - { - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); - } -#else - exists = volume->getDisk()->exists(path); - if (exists) - in = openForReading(volume->getDisk(), path); -#endif - + bool exists = metadata_manager->exists("columns.txt"); if (!exists) { /// We can get list of columns only from columns.txt in compact parts. @@ -1370,6 +1191,7 @@ void IMergeTreeDataPart::loadColumns(bool require) } else { + auto in = metadata_manager->read("columns.txt"); loaded_columns.readText(*in); for (const auto & column : loaded_columns) @@ -1449,24 +1271,12 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ } } -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::DROP, true); - assertMetadataCacheDropped(true); - } -#endif - + metadata_manager->deleteAll(true); + metadata_manager->assertAllDeleted(true); volume->getDisk()->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr))); volume->getDisk()->moveDirectory(from, to); relative_path = new_relative_path; - -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::PUT, true); - } -#endif + metadata_manager->updateAll(true); SyncGuardPtr sync_guard; if (storage.getSettings()->fsync_part_directory) @@ -1475,73 +1285,6 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } -#if USE_ROCKSDB -void IMergeTreeDataPart::modifyAllMetadataCaches(ModifyCacheType type, bool include_projection) const -{ - assert(use_metadata_cache); - - Strings files; - appendFilesOfColumnsChecksumsIndexes(files, include_projection); - LOG_TRACE( - storage.log, - "part name:{} path:{} {} keys:{}", - name, - getFullRelativePath(), - modifyCacheTypeToString(type), - boost::algorithm::join(files, ", ")); - - switch (type) - { - case ModifyCacheType::PUT: - metadata_cache->batchSet(volume->getDisk(), files); - break; - case ModifyCacheType::DROP: - metadata_cache->batchDelete(files); - break; - } -} - -void IMergeTreeDataPart::assertMetadataCacheDropped(bool include_projection) const -{ - assert(use_metadata_cache); - - Strings files; - std::vector _; - metadata_cache->getFilesAndCheckSums(files, _); - if (files.empty()) - return; - - for (const auto & file : files) - { - String file_name = fs::path(file).filename(); - /// file belongs to current part - if (fs::path(getFullRelativePath()) / file_name == file) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Data part {} with type {} with meta file {} still in cache", name, getType().toString(), file); - } - - /// file belongs to projection part of current part - if (!parent_part && include_projection) - { - for (const auto & [projection_name, projection_part] : projection_parts) - { - if (fs::path(projection_part->getFullRelativePath()) / file_name == file) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Data part {} with type {} with meta file {} with projection name still in cache", - name, - getType().toString(), - file, - projection_name); - } - } - } - } -} -#endif - std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { /// NOTE: It's needed for zero-copy replication @@ -1560,6 +1303,18 @@ std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const return {}; } +void IMergeTreeDataPart::initializePartMetadataManager() +{ +#if USE_ROCKSDB + if (use_metadata_cache) + metadata_manager = std::make_shared(this, storage.getContext()->getMergeTreeMetadataCache()); + else + metadata_manager = std::make_shared(this); +#else + metadata_manager = std::make_shared(this); +#endif +} + void IMergeTreeDataPart::remove() const { std::optional keep_shared_data = keepSharedDataInDecoupledStorage(); @@ -1579,13 +1334,8 @@ void IMergeTreeDataPart::remove() const return; } -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::DROP); - assertMetadataCacheDropped(); - } -#endif + metadata_manager->deleteAll(false); + metadata_manager->assertAllDeleted(false); /** Atomic directory removal: * - rename directory to temporary name; @@ -1690,13 +1440,8 @@ void IMergeTreeDataPart::remove() const void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_shared_data) const { -#if USE_ROCKSDB - if (use_metadata_cache) - { - modifyAllMetadataCaches(ModifyCacheType::DROP); - assertMetadataCacheDropped(); - } -#endif + metadata_manager->deleteAll(false); + metadata_manager->assertAllDeleted(false); String to = parent_to + "/" + relative_path; auto disk = volume->getDisk(); @@ -2037,6 +1782,7 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } +/* #if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { @@ -2107,6 +1853,7 @@ void IMergeTreeDataPart::checkMetadataCache(Strings & files, std::vector #include #include -#include +#include #include @@ -46,22 +46,6 @@ class UncompressedCache; class IMergeTreeDataPart : public std::enable_shared_from_this { public: - -#if USE_ROCKSDB - enum ModifyCacheType : uint8_t - { - PUT = 1, /// Override set - DROP = 2, /// Remove keys - }; - - static constexpr std::string_view modifyCacheTypeToString(ModifyCacheType type) - { - return magic_enum::enum_name(type); - } - - using uint128 = PartMetadataCache::uint128; -#endif - static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -78,6 +62,8 @@ public: using Type = MergeTreeDataPartType; + using uint128 = IPartMetadataManager::uint128; + IMergeTreeDataPart( const MergeTreeData & storage_, @@ -156,10 +142,6 @@ public: /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; -#if USE_ROCKSDB - void assertMetadataCacheDropped(bool include_projection = false) const; -#endif - void remove() const; void projectionRemove(const String & parent_to, bool keep_shared_data = false) const; @@ -320,11 +302,7 @@ public: { } -#if USE_ROCKSDB - void load(const MergeTreeData & data, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); -#else - void load(const MergeTreeData & data, const DiskPtr & disk, const String & part_path); -#endif + void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager); void store(const MergeTreeData & data, const DiskPtr & disk, const String & part_path, Checksums & checksums) const; void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const; @@ -393,9 +371,7 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; -#if USE_ROCKSDB - virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; -#endif + // virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; bool isProjectionPart() const { return parent_part != nullptr; } @@ -471,9 +447,7 @@ protected: /// Disabled when USE_ROCKSDB is OFF, or use_metadata_cache is set true in merge tree settings bool use_metadata_cache = false; -#if USE_ROCKSDB - mutable PartMetadataCachePtr metadata_cache; -#endif + mutable PartMetadataManagerPtr metadata_manager; void removeIfNeeded(); @@ -488,6 +462,8 @@ protected: std::optional keepSharedDataInDecoupledStorage() const; + void initializePartMetadataManager(); + private: /// In compact parts order of columns is necessary NameToNumber column_name_to_position; @@ -550,10 +526,8 @@ private: /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; -#if USE_ROCKSDB - void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; - IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; -#endif + // void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; + // IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/IPartMetadataManager.cpp b/src/Storages/MergeTree/IPartMetadataManager.cpp new file mode 100644 index 00000000000..5e24ac2c0e1 --- /dev/null +++ b/src/Storages/MergeTree/IPartMetadataManager.cpp @@ -0,0 +1,11 @@ +#include "IPartMetadataManager.h" + +#include +#include + +namespace DB +{ +IPartMetadataManager::IPartMetadataManager(const IMergeTreeDataPart * part_) : part(part_), disk(part->volume->getDisk()) +{ +} +} diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h new file mode 100644 index 00000000000..6db87e20f16 --- /dev/null +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IMergeTreeDataPart; + +class SeekableReadBuffer; + +class IDisk; +using DiskPtr = std::shared_ptr; + + +class IPartMetadataManager +{ +public: + using uint128 = CityHash_v1_0_2::uint128; + + explicit IPartMetadataManager(const IMergeTreeDataPart * part_); + + virtual ~IPartMetadataManager() = default; + + virtual std::unique_ptr read(const String & file_name) const = 0; + + virtual bool exists(const String & file_name) const = 0; + + virtual void deleteAll(bool include_projection) = 0; + + virtual void assertAllDeleted(bool include_projection) const = 0; + + virtual void updateAll(bool include_projection) = 0; + +protected: + const IMergeTreeDataPart * part; + const DiskPtr disk; +}; + +using PartMetadataManagerPtr = std::shared_ptr; +} diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4a108390a85..47dcecaa860 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -895,6 +895,7 @@ protected: friend class StorageReplicatedMergeTree; friend class MergeTreeDataWriter; friend class MergeTask; + friend class IPartMetadataManager; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 8a85e374062..3d933303142 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,11 +160,13 @@ namespace }; } +/* static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) { size_t file_size = disk->getFileSize(path); return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); } +*/ String MergeTreePartition::getID(const MergeTreeData & storage) const { @@ -355,21 +357,18 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } } -#if USE_ROCKSDB -void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path) -#else -void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path) -#endif +void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataManagerPtr & manager) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) return; const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; - auto partition_file_path = part_path + "partition.dat"; - std::unique_ptr file; +/* #if USE_ROCKSDB + std::unique_ptr file; + auto partition_file_path = part_path + "partition.dat"; String _; if (metadata_cache) { @@ -382,6 +381,8 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis #else file = openForReading(disk, partition_file_path); #endif +*/ + auto file = manager->read("partition.dat"); value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index b1bf64550c6..d408b7df6a1 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -4,12 +4,9 @@ #include #include #include +#include #include -#if USE_ROCKSDB -#include -#endif - namespace DB { @@ -41,11 +38,7 @@ public: void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; -#if USE_ROCKSDB - void load(const MergeTreeData & storage, const PartMetadataCachePtr & metadata_cache, const DiskPtr & disk, const String & part_path); -#else - void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path); -#endif + void load(const MergeTreeData & storage, const PartMetadataManagerPtr & manager); void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; diff --git a/src/Storages/MergeTree/PartMetadataCache.cpp b/src/Storages/MergeTree/PartMetadataCache.cpp deleted file mode 100644 index eff6d9cc0a9..00000000000 --- a/src/Storages/MergeTree/PartMetadataCache.cpp +++ /dev/null @@ -1,145 +0,0 @@ -#include "PartMetadataCache.h" - -#if USE_ROCKSDB -#include -#include -#include -#include -#include -#include -#include -#include - -namespace ProfileEvents -{ - extern const Event MergeTreeMetadataCacheHit; - extern const Event MergeTreeMetadataCacheMiss; -} - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -namespace DB -{ - -std::unique_ptr -PartMetadataCache::readOrSet(const DiskPtr & disk, const String & file_name, String & value) -{ - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - auto status = cache->get(key, value); - if (!status.ok()) - { - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); - if (!disk->exists(file_path)) - { - return nullptr; - } - - auto in = disk->readFile(file_path); - if (in) - { - readStringUntilEOF(value, *in); - cache->put(key, value); - } - } - else - { - ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); - } - return std::make_unique(value); -} - -void PartMetadataCache::batchSet(const DiskPtr & disk, const Strings & file_names) -{ - String text; - String read_value; - for (const auto & file_name : file_names) - { - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - if (!disk->exists(file_path)) - continue; - - auto in = disk->readFile(file_path); - if (!in) - continue; - - readStringUntilEOF(text, *in); - auto status = cache->put(key, text); - if (!status.ok()) - { - status = cache->get(key, read_value); - if (status.IsNotFound() || read_value == text) - continue; - throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); - } - } -} - -void PartMetadataCache::batchDelete(const Strings & file_names) -{ - for (const auto & file_name : file_names) - { - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - auto status = cache->del(key); - if (!status.ok()) - { - String read_value; - status = cache->get(key, read_value); - if (status.IsNotFound()) - continue; - throw Exception(ErrorCodes::LOGICAL_ERROR, "drop meta failed status:{}, file_path:{}", status.ToString(), file_path); - } - } -} - -void PartMetadataCache::set(const String & file_name, const String & value) -{ - String file_path = fs::path(getFullRelativePath()) / file_name; - String key = getKey(file_path); - String read_value; - auto status = cache->get(key, read_value); - if (status == rocksdb::Status::OK() && value == read_value) - return; - - status = cache->put(key, value); - if (!status.ok()) - { - status = cache->get(key, read_value); - if (status.IsNotFound() || read_value == value) - return; - - throw Exception(ErrorCodes::LOGICAL_ERROR, "set meta failed status:{}, file_path:{}", status.ToString(), file_path); - } -} - -void PartMetadataCache::getFilesAndCheckSums(Strings & files, std::vector & checksums) const -{ - String prefix = getKey(fs::path(getFullRelativePath()) / ""); - Strings values; - cache->getByPrefix(prefix, files, values); - size_t size = files.size(); - for (size_t i = 0; i < size; ++i) - { - ReadBufferFromString rbuf(values[i]); - HashingReadBuffer hbuf(rbuf); - checksums.push_back(hbuf.getHash()); - } -} - -String PartMetadataCache::getFullRelativePath() const -{ - return fs::path(relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -} - -String PartMetadataCache::getKey(const String & file_path) const -{ - return disk_name + ":" + file_path; -} - -} -#endif diff --git a/src/Storages/MergeTree/PartMetadataCache.h b/src/Storages/MergeTree/PartMetadataCache.h deleted file mode 100644 index c2904738b9c..00000000000 --- a/src/Storages/MergeTree/PartMetadataCache.h +++ /dev/null @@ -1,62 +0,0 @@ -#pragma once - -#include "config_core.h" - -#if USE_ROCKSDB -#include -#include - - -namespace DB -{ - -class SeekableReadBuffer; -class IMergeTreeDataPart; - -class MergeTreeMetadataCache; -using MergeTreeMetadataCachePtr = std::shared_ptr; - -class IDisk; -using DiskPtr = std::shared_ptr; - -class PartMetadataCache -{ -public: - using uint128 = CityHash_v1_0_2::uint128; - - PartMetadataCache( - const MergeTreeMetadataCachePtr & cache_, - const String & disk_name_, - const String & relative_data_path_, - const String & relative_path_, - const IMergeTreeDataPart * parent_part_) - : cache(cache_) - , disk_name(disk_name_) - , relative_data_path(relative_data_path_) - , relative_path(relative_path_) - , parent_part(parent_part_) - { - } - - std::unique_ptr - readOrSet(const DiskPtr & disk, const String & file_name, String & value); - void batchSet(const DiskPtr & disk, const Strings & file_names); - void batchDelete(const Strings & file_names); - void set(const String & file_name, const String & value); - void getFilesAndCheckSums(Strings & files, std::vector & checksums) const; - -private: - String getFullRelativePath() const; - String getKey(const String & file_path) const; - - MergeTreeMetadataCachePtr cache; - const String & disk_name; - const String & relative_data_path; /// Relative path of table to disk - const String & relative_path; /// Relative path of part to table - const IMergeTreeDataPart * parent_part; -}; - -using PartMetadataCachePtr = std::shared_ptr; - -} -#endif diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp new file mode 100644 index 00000000000..f12af590e7d --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp @@ -0,0 +1,33 @@ +#include "PartMetadataManagerOrdinary.h" + +#include +#include +#include + +namespace DB +{ + +static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) +{ + size_t file_size = disk->getFileSize(path); + return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); +} + +PartMetadataManagerOrdinary::PartMetadataManagerOrdinary(const IMergeTreeDataPart * part_) : IPartMetadataManager(part_) +{ +} + + +std::unique_ptr PartMetadataManagerOrdinary::read(const String & file_name) const +{ + String file_path = fs::path(part->getFullRelativePath() + "/" + file_name); + return openForReading(disk, file_path); +} + +bool PartMetadataManagerOrdinary::exists(const String & file_name) const +{ + return disk->exists(fs::path(part->getFullRelativePath()) / file_name); +} + + +} diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.h b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h new file mode 100644 index 00000000000..5f236f6271d --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +class PartMetadataManagerOrdinary : public IPartMetadataManager +{ +public: + explicit PartMetadataManagerOrdinary(const IMergeTreeDataPart * part_); + + ~PartMetadataManagerOrdinary() override = default; + + std::unique_ptr read(const String & file_name) const override; + + bool exists(const String & file_name) const override; + + void deleteAll(bool /*include_projection*/) override {} + + void assertAllDeleted(bool /*include_projection*/) const override {} + + void updateAll(bool /*include_projection*/) override {} + +}; + + +} diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp new file mode 100644 index 00000000000..ce10f7b625c --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -0,0 +1,194 @@ +#include "PartMetadataManagerWithCache.h" + +#if USE_ROCKSDB +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event MergeTreeMetadataCacheHit; + extern const Event MergeTreeMetadataCacheMiss; +} + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ +PartMetadataManagerWithCache::PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_) + : IPartMetadataManager(part_), cache(cache_) +{ +} + +String PartMetadataManagerWithCache::getKeyFromFilePath(const String & file_path) const +{ + return disk->getName() + ":" + file_path; +} + +String PartMetadataManagerWithCache::getFilePathFromKey(const String & key) const +{ + return key.substr(disk->getName().size() + 1); +} + +std::unique_ptr PartMetadataManagerWithCache::read(const String & file_name) const +{ + String file_path = fs::path(part->getFullRelativePath()) / file_name; + String key = getKeyFromFilePath(file_path); + String value; + auto status = cache->get(key, value); + if (!status.ok()) + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); + auto in = disk->readFile(file_path); + readStringUntilEOF(value, *in); + cache->put(key, value); + } + else + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); + } + return std::make_unique(value); +} + +bool PartMetadataManagerWithCache::exists(const String & file_name) const +{ + String file_path = fs::path(part->getFullRelativePath()) / file_name; + String key = getKeyFromFilePath(file_path); + String value; + auto status = cache->get(key, value); + if (status.ok()) + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheHit); + return true; + } + else + { + ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheMiss); + return disk->exists(fs::path(part->getFullRelativePath()) / file_name); + } +} + +void PartMetadataManagerWithCache::deleteAll(bool include_projection) +{ + Strings file_names; + part->appendFilesOfColumnsChecksumsIndexes(file_names, include_projection); + + String value; + for (const auto & file_name : file_names) + { + String file_path = fs::path(part->getFullRelativePath()) / file_name; + String key = getKeyFromFilePath(file_path); + auto status = cache->del(key); + if (!status.ok()) + { + status = cache->get(key, value); + if (status.IsNotFound()) + continue; + + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "deleteAll failed include_projection:{} status:{}, file_path:{}", + include_projection, + status.ToString(), + file_path); + } + } +} + +void PartMetadataManagerWithCache::updateAll(bool include_projection) +{ + Strings file_names; + part->appendFilesOfColumnsChecksumsIndexes(file_names, include_projection); + + String value; + String read_value; + for (const auto & file_name : file_names) + { + String file_path = fs::path(part->getFullRelativePath()) / file_name; + if (!disk->exists(file_path)) + continue; + auto in = disk->readFile(file_path); + readStringUntilEOF(value, *in); + + String key = getKeyFromFilePath(file_path); + auto status = cache->put(key, value); + if (!status.ok()) + { + status = cache->get(key, read_value); + if (status.IsNotFound() || read_value == value) + continue; + + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "updateAll failed include_projection:{} status:{}, file_path:{}", + include_projection, + status.ToString(), + file_path); + } + } +} + +void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) const +{ + Strings keys; + std::vector _; + getKeysAndCheckSums(keys, _); + if (keys.empty()) + return; + + String file_path; + String file_name; + for (const auto & key : keys) + { + file_path = getFilePathFromKey(key); + file_name = fs::path(file_path).filename(); + + /// Metadata file belongs to current part + if (fs::path(part->getFullRelativePath()) / file_name == file_path) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Data part {} with type {} with meta file {} still in cache", + part->name, + part->getType().toString(), + file_path); + + /// File belongs to projection part of current part + if (!part->isProjectionPart() && include_projection) + { + const auto & projection_parts = part->getProjectionParts(); + for (const auto & [projection_name, projection_part] : projection_parts) + { + if (fs::path(projection_part->getFullRelativePath()) / file_name == file_path) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Data part {} with type {} with meta file {} with projection name still in cache", + part->name, + part->getType().toString(), + file_path, + projection_name); + } + } + } + } +} + +void PartMetadataManagerWithCache::getKeysAndCheckSums(Strings & keys, std::vector & checksums) const +{ + String prefix = getKeyFromFilePath(fs::path(part->getFullRelativePath()) / ""); + Strings values; + cache->getByPrefix(prefix, keys, values); + size_t size = keys.size(); + for (size_t i = 0; i < size; ++i) + { + ReadBufferFromString rbuf(values[i]); + HashingReadBuffer hbuf(rbuf); + checksums.push_back(hbuf.getHash()); + } +} + +} +#endif diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.h b/src/Storages/MergeTree/PartMetadataManagerWithCache.h new file mode 100644 index 00000000000..76570b0684a --- /dev/null +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.h @@ -0,0 +1,40 @@ +#pragma once + +#include "config_core.h" + +#if USE_ROCKSDB +#include +#include + +namespace DB +{ + +class PartMetadataManagerWithCache : public IPartMetadataManager +{ +public: + PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_); + + ~PartMetadataManagerWithCache() override = default; + + std::unique_ptr read(const String & file_name) const override; + + bool exists(const String & file_name) const override; + + void deleteAll(bool include_projection) override; + + void assertAllDeleted(bool include_projection) const override; + + void updateAll(bool include_projection) override; + +private: + String getKeyFromFilePath(const String & file_path) const; + String getFilePathFromKey(const String & key) const; + + void getKeysAndCheckSums(Strings & keys, std::vector & checksums) const; + + + MergeTreeMetadataCachePtr cache; +}; + +} +#endif From b91e0a533c0328167cf812c4f853d3a18cdbac84 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jan 2022 20:05:22 +0800 Subject: [PATCH 021/103] fix style --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 4c88580e069..99072ebcd41 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -338,7 +338,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementTypeMetric(part_type); minmax_idx = std::make_shared(); - + initializePartMetadataManager(); } From 83d064c24adb5e3b509050e610120e7c5a872707 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 6 Jan 2022 11:41:24 +0800 Subject: [PATCH 022/103] fix unit test and build error --- src/IO/CMakeLists.txt | 4 ++-- src/IO/examples/write_buffer.cpp | 1 + .../tests/gtest_merge_tree_metadata_cache.cpp | 16 ++++++---------- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/IO/CMakeLists.txt b/src/IO/CMakeLists.txt index 970602896c9..f676f415eea 100644 --- a/src/IO/CMakeLists.txt +++ b/src/IO/CMakeLists.txt @@ -1,3 +1,3 @@ -#if (ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory (examples) -#endif () +endif () diff --git a/src/IO/examples/write_buffer.cpp b/src/IO/examples/write_buffer.cpp index 5587b8aa1a2..bca0be24b1a 100644 --- a/src/IO/examples/write_buffer.cpp +++ b/src/IO/examples/write_buffer.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp index 1005739098c..839c54c63b2 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp @@ -1,9 +1,11 @@ -#include +#include "config_core.h" #if USE_ROCKSDB #include #include #include +#include +#include #include using namespace DB; @@ -13,17 +15,11 @@ class MergeTreeMetadataCacheTest : public ::testing::Test public: void SetUp() override { - shared_context = Context::createShared(); - global_context = Context::createGlobal(shared_context.get()); - global_context->makeGlobalContext(); - global_context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - cache = global_context->getMergeTreeMetadataCache(); + const auto & context_holder = getContext(); + context_holder.context->initializeMergeTreeMetadataCache("./db/", 256 << 20); + cache = context_holder.context->getMergeTreeMetadataCache(); } - void TearDown() override { global_context->shutdown(); } - - SharedContextHolder shared_context; - ContextMutablePtr global_context; MergeTreeMetadataCachePtr cache; }; From 3803cc3d5eddf69fafc72f5181f25be65761c4e5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 6 Jan 2022 15:42:12 +0800 Subject: [PATCH 023/103] fix bug --- programs/server/Server.cpp | 4 ++-- programs/server/config.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8641623ac07..846a1960713 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -814,7 +814,6 @@ if (ThreadFuzzer::instance().isEffective()) fs::create_directories(path / "metadata_dropped/"); } - #if USE_ROCKSDB /// Initialize merge tree metadata cache if (config().has("merge_tree_metadata_cache")) @@ -822,9 +821,10 @@ if (ThreadFuzzer::instance().isEffective()) fs::create_directories(path / "rocksdb/"); size_t size = config().getUInt64("merge_tree_metadata_cache.lru_cache_size", 256 << 20); bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false); - try { + LOG_DEBUG( + log, "Initiailizing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted); global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); } catch (...) diff --git a/programs/server/config.xml b/programs/server/config.xml index c5f8e7eeb94..4f75e233b64 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1293,8 +1293,8 @@ --> - + From cf413f16a8166af61c236e054be650ce7ffa908a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 7 Jan 2022 18:37:08 +0800 Subject: [PATCH 024/103] remove function checkPartMetadataCache --- src/Functions/checkPartMetadataCache.cpp | 156 ------------------ .../registerFunctionsMiscellaneous.cpp | 8 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 51 +----- src/Storages/MergeTree/IMergeTreeDataPart.h | 12 +- src/Storages/MergeTree/IPartMetadataManager.h | 3 + .../MergeTree/PartMetadataManagerOrdinary.h | 1 + .../PartMetadataManagerWithCache.cpp | 83 +++++++++- .../MergeTree/PartMetadataManagerWithCache.h | 2 + .../ReplicatedMergeTreePartCheckThread.cpp | 1 + src/Storages/MergeTree/checkDataPart.cpp | 1 - src/Storages/StorageMergeTree.cpp | 3 + 11 files changed, 101 insertions(+), 220 deletions(-) delete mode 100644 src/Functions/checkPartMetadataCache.cpp diff --git a/src/Functions/checkPartMetadataCache.cpp b/src/Functions/checkPartMetadataCache.cpp deleted file mode 100644 index d65c14095f7..00000000000 --- a/src/Functions/checkPartMetadataCache.cpp +++ /dev/null @@ -1,156 +0,0 @@ -#include "config_core.h" - -#if USE_ROCKSDB -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; -} - -class FunctionCheckPartMetadataCache : public IFunction, WithContext -{ -public: - using uint128 = IMergeTreeDataPart::uint128; - using DataPartPtr = MergeTreeData::DataPartPtr; - using DataPartState = MergeTreeData::DataPartState; - using DataPartStates = MergeTreeData::DataPartStates; - - - static constexpr auto name = "checkPartMetadataCache"; - static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - - static constexpr DataPartStates part_states - = {DataPartState::Active, - DataPartState::Temporary, - DataPartState::PreActive, - DataPartState::Outdated, - DataPartState::Deleting, - DataPartState::DeleteOnDestroy}; - - explicit FunctionCheckPartMetadataCache(ContextPtr context_) : WithContext(context_) { } - - String getName() const override { return name; } - - bool isDeterministic() const override { return false; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - bool isDeterministicInScopeOfQuery() const override { return false; } - - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - for (const auto & argument : arguments) - { - if (!isString(argument)) - throw Exception("The argument of function " + getName() + " must have String type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - DataTypePtr key_type = std::make_unique(); - DataTypePtr state_type = std::make_unique(); - DataTypePtr cache_checksum_type = std::make_unique(32); - DataTypePtr disk_checksum_type = std::make_unique(32); - DataTypePtr match_type = std::make_unique(); - DataTypePtr tuple_type - = std::make_unique(DataTypes{key_type, state_type, cache_checksum_type, disk_checksum_type, match_type}); - return std::make_shared(tuple_type); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - /// Get database name - const auto * arg_database = arguments[0].column.get(); - const ColumnString * column_database = checkAndGetColumnConstData(arg_database); - if (!column_database) - throw Exception("The first argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); - String database_name = column_database->getDataAt(0).toString(); - - /// Get table name - const auto * arg_table = arguments[1].column.get(); - const ColumnString * column_table = checkAndGetColumnConstData(arg_table); - if (!column_table) - throw Exception("The second argument of function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN); - String table_name = column_table->getDataAt(0).toString(); - - /// Get storage - StorageID storage_id(database_name, table_name); - auto storage = DatabaseCatalog::instance().getTable(storage_id, getContext()); - auto data = std::dynamic_pointer_cast(storage); - if (!data || !data->getSettings()->use_metadata_cache) - throw Exception("The table in function " + getName() + " must be in MergeTree Family", ErrorCodes::ILLEGAL_COLUMN); - - /// Fill in checking results. - auto col_result = result_type->createColumn(); - auto & col_arr = assert_cast(*col_result); - auto & col_tuple = assert_cast(col_arr.getData()); - col_tuple.reserve(data->fileNumberOfDataParts(part_states)); - auto & col_key = assert_cast(col_tuple.getColumn(0)); - auto & col_state = assert_cast(col_tuple.getColumn(1)); - auto & col_cache_checksum = assert_cast(col_tuple.getColumn(2)); - auto & col_disk_checksum = assert_cast(col_tuple.getColumn(3)); - auto & col_match = assert_cast(col_tuple.getColumn(4)); - auto parts = data->getDataParts(part_states); - for (const auto & part : parts) - executePart(part, col_key, col_state, col_cache_checksum, col_disk_checksum, col_match); - col_arr.getOffsets().push_back(col_tuple.size()); - return result_type->createColumnConst(input_rows_count, col_arr[0]); - } - - static void executePart( - const DataPartPtr & part, - ColumnString & col_key, - ColumnString & col_state, - ColumnFixedString & col_cache_checksum, - ColumnFixedString & col_disk_checksum, - ColumnUInt8 & col_match) - { - Strings keys; - auto state_view = part->stateString(); - String state(state_view.data(), state_view.size()); - std::vector cache_checksums; - std::vector disk_checksums; - uint8_t match = 0; - size_t file_number = part->fileNumberOfColumnsChecksumsIndexes(); - keys.reserve(file_number); - cache_checksums.reserve(file_number); - disk_checksums.reserve(file_number); - - // part->checkMetadataCache(keys, cache_checksums, disk_checksums); - for (size_t i = 0; i < keys.size(); ++i) - { - col_key.insert(keys[i]); - col_state.insert(state); - col_cache_checksum.insert(getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); - col_disk_checksum.insert(getHexUIntUppercase(disk_checksums[i].first) + getHexUIntUppercase(disk_checksums[i].second)); - - match = cache_checksums[i] == disk_checksums[i] ? 1 : 0; - col_match.insertValue(match); - } - } -}; - -void registerFunctionCheckPartMetadataCache(FunctionFactory & factory) -{ - factory.registerFunction(); -} -} -#endif diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 297e6dfb452..76d61ce509a 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -81,10 +81,6 @@ void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); void registerFunctionGetOSKernelVersion(FunctionFactory &); -#if USE_ROCKSDB -void registerFunctionCheckPartMetadataCache(FunctionFactory &); -#endif - #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); #endif @@ -171,10 +167,6 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionZooKeeperSessionUptime(factory); registerFunctionGetOSKernelVersion(factory); -#if USE_ROCKSDB - registerFunctionCheckPartMetadataCache(factory); -#endif - #if USE_ICU registerFunctionConvertCharset(factory); #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 99072ebcd41..909c6e42c9e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -61,8 +61,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -void IMergeTreeDataPart::MinMaxIndex::load( - const MergeTreeData & data, const PartMetadataManagerPtr & manager) +void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const PartMetadataManagerPtr & manager) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); const auto & partition_key = metadata_snapshot->getPartitionKey(); @@ -1782,8 +1781,6 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID() const return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); } -/* -#if USE_ROCKSDB IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const { assert(use_metadata_cache); @@ -1808,52 +1805,10 @@ IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const St return in_hash.getHash(); } -void IMergeTreeDataPart::checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const +std::unordered_map IMergeTreeDataPart::checkMetadata() const { - assert(use_metadata_cache); - - /// Only applies for normal part - if (isProjectionPart()) - return; - - /// the directory of projection part is under the directory of its parent part - const auto filenames_without_checksums = getFileNamesWithoutChecksums(); - metadata_cache->getFilesAndCheckSums(files, cache_checksums); - for (const auto & file : files) - { - // std::cout << "check key:" << file << std::endl; - String file_name = fs::path(file).filename(); - - /// file belongs to normal part - if (fs::path(getFullRelativePath()) / file_name == file) - { - auto disk_checksum = getActualChecksumByFile(file); - disk_checksums.push_back(disk_checksum); - continue; - } - - /// file belongs to projection part - String proj_dir_name = fs::path(file).parent_path().filename(); - auto pos = proj_dir_name.find_last_of('.'); - if (pos == String::npos) - { - disk_checksums.push_back({}); - continue; - } - String proj_name = proj_dir_name.substr(0, pos); - auto it = projection_parts.find(proj_name); - if (it == projection_parts.end()) - { - disk_checksums.push_back({}); - continue; - } - - auto disk_checksum = it->second->getActualChecksumByFile(file); - disk_checksums.push_back(disk_checksum); - } + return metadata_manager->check(); } -#endif -*/ bool isCompactPart(const MergeTreeDataPartPtr & data_part) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2bb6f570b6a..ecbc2a32aa4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -371,8 +371,6 @@ public: String getRelativePathForPrefix(const String & prefix, bool detached = false) const; - // virtual void checkMetadataCache(Strings & files, std::vector & cache_checksums, std::vector & disk_checksums) const; - bool isProjectionPart() const { return parent_part != nullptr; } const IMergeTreeDataPart * getParentPart() const { return parent_part; } @@ -418,6 +416,12 @@ public: /// Required for distinguish different copies of the same part on S3 String getUniqueId() const; + /// Get checksums of metadata file in part directory + IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; + + /// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true) + std::unordered_map checkMetadata() const; + protected: /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk @@ -464,6 +468,7 @@ protected: void initializePartMetadataManager(); + private: /// In compact parts order of columns is necessary NameToNumber column_name_to_position; @@ -526,9 +531,6 @@ private: /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; - // void modifyAllMetadataCaches(ModifyCacheType type, bool include_projection = false) const; - // IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_path) const; - mutable State state{State::Temporary}; }; diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h index 6db87e20f16..17786c90761 100644 --- a/src/Storages/MergeTree/IPartMetadataManager.h +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -33,6 +34,8 @@ public: virtual void updateAll(bool include_projection) = 0; + virtual std::unordered_map check() const = 0; + protected: const IMergeTreeDataPart * part; const DiskPtr disk; diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.h b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h index 5f236f6271d..a655431296a 100644 --- a/src/Storages/MergeTree/PartMetadataManagerOrdinary.h +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.h @@ -22,6 +22,7 @@ public: void updateAll(bool /*include_projection*/) override {} + std::unordered_map check() const override { return {}; } }; diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index ce10f7b625c..b088f63c0d0 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -1,6 +1,8 @@ #include "PartMetadataManagerWithCache.h" #if USE_ROCKSDB +#include +#include #include #include #include @@ -11,13 +13,16 @@ namespace ProfileEvents extern const Event MergeTreeMetadataCacheMiss; } +namespace DB +{ + namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int CORRUPTED_DATA; + extern const int NO_SUCH_PROJECTION_IN_TABLE; } -namespace DB -{ PartMetadataManagerWithCache::PartMetadataManagerWithCache(const IMergeTreeDataPart * part_, const MergeTreeMetadataCachePtr & cache_) : IPartMetadataManager(part_), cache(cache_) { @@ -190,5 +195,79 @@ void PartMetadataManagerWithCache::getKeysAndCheckSums(Strings & keys, std::vect } } +std::unordered_map PartMetadataManagerWithCache::check() const +{ + /// Only applies for normal part stored on disk + if (part->isProjectionPart() || !part->isStoredOnDisk()) + return {}; + + /// the directory of projection part is under the directory of its parent part + const auto filenames_without_checksums = part->getFileNamesWithoutChecksums(); + + std::unordered_map results; + Strings keys; + std::vector cache_checksums; + std::vector disk_checksums; + getKeysAndCheckSums(keys, cache_checksums); + for (size_t i = 0; i < keys.size(); ++i) + { + const auto & key = keys[i]; + String file_path = getFilePathFromKey(key); + String file_name = fs::path(file_path).filename(); + results.emplace(file_name, cache_checksums[i]); + + /// File belongs to normal part + if (fs::path(part->getFullRelativePath()) / file_name == file_path) + { + auto disk_checksum = part->getActualChecksumByFile(file_path); + if (disk_checksum != cache_checksums[i]) + throw Exception( + ErrorCodes::CORRUPTED_DATA, + "Checksums doesn't match in part {}. Expected: {}. Found {}.", + part->name, + getHexUIntUppercase(disk_checksum.first) + getHexUIntUppercase(disk_checksum.second), + getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + + disk_checksums.push_back(disk_checksum); + continue; + } + + /// File belongs to projection part + String proj_dir_name = fs::path(file_path).parent_path().filename(); + auto pos = proj_dir_name.find_last_of('.'); + if (pos == String::npos) + { + throw Exception( + ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, + "There is no projection in part: {} contains file: {} with directory name: {}", + part->name, + file_path, + proj_dir_name); + } + + String proj_name = proj_dir_name.substr(0, pos); + const auto & projection_parts = part->getProjectionParts(); + auto it = projection_parts.find(proj_name); + if (it == projection_parts.end()) + { + throw Exception( + ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, + "There is no projection {} in part: {} contains file: {}", + proj_name, part->name, file_path); + } + + auto disk_checksum = it->second->getActualChecksumByFile(file_path); + if (disk_checksum != cache_checksums[i]) + throw Exception( + ErrorCodes::CORRUPTED_DATA, + "Checksums doesn't match in projection part {} {}. Expected: {}. Found {}.", + part->name, proj_name, + getHexUIntUppercase(disk_checksum.first) + getHexUIntUppercase(disk_checksum.second), + getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + disk_checksums.push_back(disk_checksum); + } + return results; +} + } #endif diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.h b/src/Storages/MergeTree/PartMetadataManagerWithCache.h index 76570b0684a..8b1472f5457 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.h +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.h @@ -26,6 +26,8 @@ public: void updateAll(bool include_projection) override; + std::unordered_map check() const override; + private: String getKeyFromFilePath(const String & file_path) const; String getFilePathFromKey(const String & key) const; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 8fcaee66007..85d929b5ea4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -399,6 +399,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na LOG_WARNING(log, "We have part {} covering part {}", part->name, part_name); } + part->checkMetadata(); return {part_name, true, ""}; } diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index eabd901eb24..0f35e30c5d0 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -270,7 +270,6 @@ IMergeTreeDataPart::Checksums checkDataPart( if (require_checksums || !checksums_txt.files.empty()) checksums_txt.checkEqual(checksums_data, check_uncompressed); - return checksums_data; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 11815d9ceef..03a789a7725 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1612,6 +1612,8 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ auto out = disk->writeFile(tmp_checksums_path, 4096); part->checksums.write(*out); disk->moveFile(tmp_checksums_path, checksums_path); + + part->checkMetadata(); results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } catch (const Exception & ex) @@ -1628,6 +1630,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ try { checkDataPart(part, true); + part->checkMetadata(); results.emplace_back(part->name, true, ""); } catch (const Exception & ex) From 3ed13e789612010ad55dcb10586e6d226c33ba27 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 7 Jan 2022 21:06:10 +0800 Subject: [PATCH 025/103] refactor stateless test and add move part/partition integration test --- programs/server/config.xml | 4 +- .../configs/logs_config.xml | 4 + tests/integration/test_multiple_disks/test.py | 17 +- ..._check_table_with_metadata_cache.reference | 672 ++++++++++++++++++ .../01233_check_table_with_metadata_cache.sh | 90 +++ 5 files changed, 780 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference create mode 100644 tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh diff --git a/programs/server/config.xml b/programs/server/config.xml index 4f75e233b64..ec49d516849 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1293,8 +1293,8 @@ --> - + diff --git a/tests/integration/test_multiple_disks/configs/logs_config.xml b/tests/integration/test_multiple_disks/configs/logs_config.xml index 2ee8bb55f38..b0643c8bdad 100644 --- a/tests/integration/test_multiple_disks/configs/logs_config.xml +++ b/tests/integration/test_multiple_disks/configs/logs_config.xml @@ -14,4 +14,8 @@ part_log
500 + + 268435456 + true + diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index db541edde9c..cfce95fab69 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -632,12 +632,13 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): return paths.strip().split('\n') -@pytest.mark.parametrize("name,engine", [ - pytest.param("altering_mt", "MergeTree()", id="mt"), +@pytest.mark.parametrize("name,engine,use_metadata_cache", [ + pytest.param("altering_mt", "MergeTree()", "false", id="mt"), + pytest.param("altering_mt", "MergeTree()", "true", id="mt_use_metadata_cache"), # ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",), # SYSTEM STOP MERGES doesn't disable merges assignments ]) -def test_alter_move(start_cluster, name, engine): +def test_alter_move(start_cluster, name, engine, use_metadata_cache): try: node1.query(""" CREATE TABLE IF NOT EXISTS {name} ( @@ -646,8 +647,8 @@ def test_alter_move(start_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy='jbods_with_external' - """.format(name=name, engine=engine)) + SETTINGS storage_policy='jbods_with_external', use_metadata_cache={use_metadata_cache} + """.format(name=name, engine=engine, use_metadata_cache=use_metadata_cache)) node1.query("SYSTEM STOP MERGES {}".format(name)) # to avoid conflicts @@ -655,6 +656,8 @@ def test_alter_move(start_cluster, name, engine): node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 66)".format(name)) node1.query("INSERT INTO {} VALUES(toDate('2019-04-10'), 42)".format(name)) node1.query("INSERT INTO {} VALUES(toDate('2019-04-11'), 43)".format(name)) + assert node1.query("CHECK TABLE " + name) == "1\n" + used_disks = get_used_disks_for_table(node1, name) assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods" @@ -664,6 +667,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, first_part)) + assert node1.query("CHECK TABLE " + name) == "1\n" disk = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip() @@ -672,6 +676,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PART '{}' TO DISK 'jbod1'".format(name, first_part)) + assert node1.query("CHECK TABLE " + name) == "1\n" disk = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, first_part)).strip() @@ -680,6 +685,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO VOLUME 'external'".format(name)) + assert node1.query("CHECK TABLE " + name) == "1\n" disks = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format( name)).strip().split('\n') @@ -690,6 +696,7 @@ def test_alter_move(start_cluster, name, engine): time.sleep(1) node1.query("ALTER TABLE {} MOVE PARTITION 201904 TO DISK 'jbod2'".format(name)) + assert node1.query("CHECK TABLE " + name) == "1\n" disks = node1.query( "SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201904' and active = 1".format( name)).strip().split('\n') diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference new file mode 100644 index 00000000000..5957d23fe82 --- /dev/null +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference @@ -0,0 +1,672 @@ +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh new file mode 100644 index 00000000000..5a3fd98c3be --- /dev/null +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -0,0 +1,90 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +table_engines="ReplicatedMergeTree" +database_engines="Ordinary Atomic" +use_metadata_caches="false true" +use_projections="false true" + +for table_engine in $table_engines; do + for database_engine in $database_engines; do + for use_metadata_cache in $use_metadata_caches; do + for use_projection in $use_projections; do + echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}" + + ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;" + ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;" + ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};" + + table_engine_clause="" + if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then + table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/test_metadata_cache/check_part_metadata_cache', 'r1')" + elif [[ "$table_engine" == "MergeTree" ]]; then + table_engine_clause="ENGINE MergeTree()" + fi + + projection_clause="" + if [[ "$use_projection" == "true" ]]; then + projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" + fi + ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache};" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Insert first batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Insert second batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # First update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Second update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # First delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Second delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Insert third batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Drop one partition. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Add column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Delete column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Add TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Modify TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Truncate table. + ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + done + done + done +done From 3d0e960bc0d6250c634811172d6cc019d36ae9ab Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 7 Jan 2022 21:07:06 +0800 Subject: [PATCH 026/103] fix stateless test --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index 5a3fd98c3be..017c7977745 100644 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -table_engines="ReplicatedMergeTree" +table_engines="MergeTree ReplicatedMergeTree" database_engines="Ordinary Atomic" use_metadata_caches="false true" use_projections="false true" From fdea2bc9caa2f2b1757d93768e0afe9ac2f30a15 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 8 Jan 2022 09:55:02 +0800 Subject: [PATCH 027/103] remove old stateless tests --- .../01233_check_part_meta_cache.reference | 28 ---- .../01233_check_part_meta_cache.sql | 125 ----------------- ..._check_part_meta_cache_in_atomic.reference | 28 ---- .../01233_check_part_meta_cache_in_atomic.sql | 126 ----------------- ...check_part_meta_cache_replicated.reference | 28 ---- ...01233_check_part_meta_cache_replicated.sql | 128 ------------------ ..._meta_cache_replicated_in_atomic.reference | 28 ---- ...k_part_meta_cache_replicated_in_atomic.sql | 128 ------------------ 8 files changed, 619 deletions(-) delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache.sql delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference delete mode 100644 tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.reference b/tests/queries/0_stateless/01233_check_part_meta_cache.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache.sql b/tests/queries/0_stateless/01233_check_part_meta_cache.sql deleted file mode 100644 index 6c1a1232cab..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache.sql +++ /dev/null @@ -1,125 +0,0 @@ --- Tags: no-parallel, no-fasttest - --- Create table under database with engine ordinary. -set mutations_sync = 1; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Ordinary; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert third batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert third batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql deleted file mode 100644 index af8d6f888a7..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_in_atomic.sql +++ /dev/null @@ -1,126 +0,0 @@ --- Tags: no-parallel, no-fasttest - --- Create table under database with engine atomic. -set mutations_sync = 1; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Atomic; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 30; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + 60; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; -CREATE TABLE test_metadata_cache.check_part_metadata_cache( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE MergeTree() PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- nsert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql deleted file mode 100644 index bdb43c4e905..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated.sql +++ /dev/null @@ -1,128 +0,0 @@ --- Tags: no-fasttest, no-parallel, zookeeper --- Tag no-parallel: static zk path - --- Create table under database with engine ordinary. -set mutations_sync = 1; -set replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Ordinary; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache ; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference deleted file mode 100644 index 2bd6025fea2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.reference +++ /dev/null @@ -1,28 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql b/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql deleted file mode 100644 index 4e491c301f2..00000000000 --- a/tests/queries/0_stateless/01233_check_part_meta_cache_replicated_in_atomic.sql +++ /dev/null @@ -1,128 +0,0 @@ --- Tags: no-fasttest, zookeeper, no-parallel --- Tag no-parallel: static zk path - --- Create table under database with engine ordinary. -set mutations_sync = 1; -set replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC; -DROP DATABASE IF EXISTS test_metadata_cache; -CREATE DATABASE test_metadata_cache ENGINE = Atomic; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete column. -alter table test_metadata_cache.check_part_metadata_cache drop column v1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v2, v3) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Recreate table with projection. -drop table if exists test_metadata_cache.check_part_metadata_cache SYNC; -CREATE TABLE test_metadata_cache.check_part_metadata_cache ( p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_metadata_cache/check_part_metadata_cache', '{replica}') PARTITION BY toYYYYMM(p) ORDER BY k TTL p + INTERVAL 15 YEAR settings use_metadata_cache = 1; - --- Insert first batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Insert second batch of data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Update some data. -alter table test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - ---Delete some data. -alter table test_metadata_cache.check_part_metadata_cache delete where k = 1; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - -alter table test_metadata_cache.check_part_metadata_cache delete where k = 8; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Delete some data. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Drop partitioin 201805 -alter table test_metadata_cache.check_part_metadata_cache drop partition 201805; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Optimize table. -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -optimize table test_metadata_cache.check_part_metadata_cache FINAL; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add column. -alter table test_metadata_cache.check_part_metadata_cache add column v3 UInt64; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Add TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Modify TTL info. -alter table test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR; -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - --- Truncate table. -truncate table test_metadata_cache.check_part_metadata_cache; -with arrayJoin(checkPartMetadataCache('test_metadata_cache', 'check_part_metadata_cache')) as info select countIf(info.5 = 0); - From 51e544b447cf23c5dc0a4a01136dda3364c6a364 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 8 Jan 2022 10:11:21 +0800 Subject: [PATCH 028/103] fix stateless test --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) mode change 100644 => 100755 tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh old mode 100644 new mode 100755 index 017c7977745..e4b8e2f6792 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: setting use_metadata_cache=true is not supported in fasttest, because clickhouse binary in fasttest is build without RocksDB. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -21,7 +23,7 @@ for table_engine in $table_engines; do table_engine_clause="" if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then - table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/test_metadata_cache/check_part_metadata_cache', 'r1')" + table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')" elif [[ "$table_engine" == "MergeTree" ]]; then table_engine_clause="ENGINE MergeTree()" fi From 5bff268f84d029261a9e75bccf2111450299723e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 11:45:06 +0800 Subject: [PATCH 029/103] optimize code as advices --- src/Interpreters/Context.cpp | 6 +++++- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/IPartMetadataManager.h | 16 +++++++++++++++- src/Storages/MergeTree/MergeTreePartition.cpp | 8 -------- .../MergeTree/PartMetadataManagerWithCache.h | 14 ++++++++++++++ 5 files changed, 34 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0cb81ba4056..d51da66872f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2291,7 +2291,11 @@ void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); if (status != rocksdb::Status::OK()) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Fail to open rocksdb path at: {} status:{}", dir, status.ToString()); + throw Exception( + ErrorCodes::SYSTEM_ERROR, + "Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).", + dir, + status.ToString()); shared->merge_tree_metadata_cache = std::make_shared(db); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7e222607cb6..c8b693727da 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/MergeTree/IPartMetadataManager.h b/src/Storages/MergeTree/IPartMetadataManager.h index 17786c90761..876000de412 100644 --- a/src/Storages/MergeTree/IPartMetadataManager.h +++ b/src/Storages/MergeTree/IPartMetadataManager.h @@ -14,7 +14,12 @@ class SeekableReadBuffer; class IDisk; using DiskPtr = std::shared_ptr; - +/// Interface for managing metadata of merge tree part. +/// IPartMetadataManager has two implementations: +/// - PartMetadataManagerOrdinary: manage metadata from disk directly. deleteAll/assertAllDeleted/updateAll/check +/// are all empty implementations because they are not needed for PartMetadataManagerOrdinary(those operations +/// are done implicitly when removing or renaming part directory). +/// - PartMetadataManagerWithCache: manage metadata from RocksDB cache and disk. class IPartMetadataManager { public: @@ -24,16 +29,25 @@ public: virtual ~IPartMetadataManager() = default; + /// Read metadata content and return SeekableReadBuffer object. virtual std::unique_ptr read(const String & file_name) const = 0; + /// Return true if metadata exists in part. virtual bool exists(const String & file_name) const = 0; + /// Delete all metadatas in part. + /// If include_projection is true, also delete metadatas in projection parts. virtual void deleteAll(bool include_projection) = 0; + /// Assert that all metadatas in part are deleted. + /// If include_projection is true, also assert that all metadatas in projection parts are deleted. virtual void assertAllDeleted(bool include_projection) const = 0; + /// Update all metadatas in part. + /// If include_projection is true, also update metadatas in projection parts. virtual void updateAll(bool include_projection) = 0; + /// Check all metadatas in part. virtual std::unordered_map check() const = 0; protected: diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 3d933303142..c16840ed1bf 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -160,14 +160,6 @@ namespace }; } -/* -static std::unique_ptr openForReading(const DiskPtr & disk, const String & path) -{ - size_t file_size = disk->getFileSize(path); - return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size); -} -*/ - String MergeTreePartition::getID(const MergeTreeData & storage) const { return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.h b/src/Storages/MergeTree/PartMetadataManagerWithCache.h index 8b1472f5457..06e7a85ba2b 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.h +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.h @@ -9,6 +9,8 @@ namespace DB { +/// PartMetadataManagerWithCache stores metadatas of part in RocksDB as cache layer to speed up +/// loading process of merge tree table. class PartMetadataManagerWithCache : public IPartMetadataManager { public: @@ -16,22 +18,34 @@ public: ~PartMetadataManagerWithCache() override = default; + /// First read the metadata from RocksDB cache, then from disk. std::unique_ptr read(const String & file_name) const override; + /// First judge existence of the metadata in RocksDB cache, then in disk. bool exists(const String & file_name) const override; + /// Delete all metadatas in part from RocksDB cache. void deleteAll(bool include_projection) override; + /// Assert all metadatas in part from RocksDB cache are deleted. void assertAllDeleted(bool include_projection) const override; + /// Update all metadatas in part from RocksDB cache. + /// Need to be called after part directory is renamed. void updateAll(bool include_projection) override; + /// Check if all metadatas in part from RocksDB cache are up to date. std::unordered_map check() const override; private: + /// Get cache key from path of metadata file. + /// Format: :relative/full/path/of/metadata/file String getKeyFromFilePath(const String & file_path) const; + + /// Get metadata file path from cache key. String getFilePathFromKey(const String & key) const; + /// Get cache keys and checksums of corresponding metadata in a part(including projection parts) void getKeysAndCheckSums(Strings & keys, std::vector & checksums) const; From 3d409a4526b3ba7331e61460d320783c8371edb1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 12:20:39 +0800 Subject: [PATCH 030/103] add metrics MergeTreeMetadataCacheSize --- src/Interpreters/AsynchronousMetrics.cpp | 11 +++++++++++ src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 7 +++++++ src/Storages/MergeTree/MergeTreeMetadataCache.h | 1 + 3 files changed, 19 insertions(+) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d1c5fbebbc7..451014755b8 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -573,6 +574,15 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } } +#if USE_ROCKSDB + { + if (auto metadata_cache = getContext()->getMergeTreeMetadataCache()) + { + new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); + } + } +#endif + #if USE_EMBEDDED_COMPILER { if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) @@ -583,6 +593,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } #endif + new_values["Uptime"] = getContext()->getUptimeSeconds(); /// Process process memory usage according to OS diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index fa3825fd3be..5df9850b318 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -61,6 +61,13 @@ void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); } +uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const +{ + uint64_t keys = 0; + rocksdb->GetAggregatedIntProperty("rocksdb.estimate-num-keys", &keys); + return keys; +} + void MergeTreeMetadataCache::shutdown() { rocksdb->Close(); diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index 286c7ebb08e..a389d9a14ad 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -28,6 +28,7 @@ public: Status del(const String & key); Status get(const String & key, String & value); void getByPrefix(const String & prefix, Strings & keys, Strings & values); + uint64_t getEstimateNumKeys() const; void shutdown(); private: From 7c7f45bce23a73052fddd1b83efa64e0cf253c3a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 12:32:38 +0800 Subject: [PATCH 031/103] fix comment --- src/Storages/MergeTree/PartMetadataManagerWithCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index b088f63c0d0..3d68497f5b0 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -201,7 +201,7 @@ std::unordered_map PartMetadataManagerWit if (part->isProjectionPart() || !part->isStoredOnDisk()) return {}; - /// the directory of projection part is under the directory of its parent part + /// The directory of projection part is under the directory of its parent part const auto filenames_without_checksums = part->getFileNamesWithoutChecksums(); std::unordered_map results; From 2cae8d552cf2ddc05a8db66359f4e014ddf95f4e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 19:17:12 +0800 Subject: [PATCH 032/103] remove useless code --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 ------- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 9 --------- src/Storages/MergeTree/MergeTreeData.h | 2 -- 4 files changed, 19 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c8995309c4b..a2cceb540af 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -681,13 +681,6 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b } } -size_t IMergeTreeDataPart::fileNumberOfColumnsChecksumsIndexes() const -{ - Strings files; - appendFilesOfColumnsChecksumsIndexes(files, true); - return files.size(); -} - void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index c8b693727da..c0c61c6ca0a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -149,7 +149,6 @@ public: /// Load checksums from checksums.txt if exists. Load index if required. void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const; - size_t fileNumberOfColumnsChecksumsIndexes() const; String getMarksFileExtension() const { return index_granularity_info.marks_file_extension; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7ee8e8f196b..a18b7125e54 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1341,15 +1341,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); } -size_t MergeTreeData::fileNumberOfDataParts(const DataPartStates & states) const -{ - size_t result = 0; - auto parts = getDataParts(states); - for (const auto & part : parts) - result += part->fileNumberOfColumnsChecksumsIndexes(); - return result; -} - /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. /// (Only files on the first level of nesting are considered). diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 80d0f0a9bff..cc6b1c78b74 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -427,8 +427,6 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); - size_t fileNumberOfDataParts(const DataPartStates & states) const; - String getLogName() const { return log_name; } Int64 getMaxBlockNumber() const; From 7e227040cb0e25f6ac8eaba688e43708080123f0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 10 Jan 2022 19:52:54 +0800 Subject: [PATCH 033/103] optimize getMergeTreeMetadataCache --- src/Interpreters/Context.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 5 +---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d51da66872f..d8bdb024462 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2029,6 +2029,10 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const #if USE_ROCKSDB MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { + if (!shared->merge_tree_metadata_cache) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart"); return shared->merge_tree_metadata_cache; } #endif diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a18b7125e54..1168335647f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -319,11 +319,8 @@ MergeTreeData::MergeTreeData( LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); +#if !USE_ROCKSDB if (use_metadata_cache) -#if USE_ROCKSDB - if (!getContext()->getMergeTreeMetadataCache()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if not config in config.xml"); -#else throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't use merge tree metadata cache if clickhouse was compiled without rocksdb"); #endif From 90a92dd14a7bae2b780bfad82dd327bc3f8a5bf7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 11 Jan 2022 11:09:52 +0800 Subject: [PATCH 034/103] fix stateless test --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- src/Interpreters/Context.cpp | 8 +++++++- src/Interpreters/Context.h | 1 + tests/queries/0_stateless/01161_all_system_tables.sh | 2 +- 4 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 451014755b8..4b9bcd42e51 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -576,7 +576,7 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti #if USE_ROCKSDB { - if (auto metadata_cache = getContext()->getMergeTreeMetadataCache()) + if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) { new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d8bdb024462..925f3048262 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2029,10 +2029,16 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const #if USE_ROCKSDB MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { - if (!shared->merge_tree_metadata_cache) + auto cache = tryGetMergeTreeMetadataCache(); + if (!cache) throw Exception( ErrorCodes::LOGICAL_ERROR, "Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart"); + return cache; +} + +MergeTreeMetadataCachePtr Context::tryGetMergeTreeMetadataCache() const +{ return shared->merge_tree_metadata_cache; } #endif diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b8616e8b634..1d618ba9ac6 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -686,6 +686,7 @@ public: #if USE_ROCKSDB MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; + MergeTreeMetadataCachePtr tryGetMergeTreeMetadataCache() const; #endif #if USE_NURAFT diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index 9b19cc97d16..1a653763ad3 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -12,7 +12,7 @@ function run_selects() { thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables - WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' + WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num") for t in "${tables_arr[@]}" From d31871d253f70ee79b3cae3e382b1f7ec02aee10 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 11 Jan 2022 15:17:00 +0800 Subject: [PATCH 035/103] commit again --- src/Common/tests/gtest_global_context.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 9bd7c2490d6..1b7eacf6f11 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -18,8 +18,11 @@ struct ContextHolder ContextHolder(ContextHolder &&) = default; }; -inline const ContextHolder & getContext() +const ContextHolder & getContext() { - static ContextHolder holder; - return holder; + static ContextHolder * holder; + static std::once_flag once; + std::call_once(once, [&]() { holder = new ContextHolder(); }); + return *holder; } + From 2b8853db279bc1b55a10ff21b2b07a3a08fc1276 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 11 Jan 2022 20:30:40 -0600 Subject: [PATCH 036/103] Update tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh Co-authored-by: Azat Khuzhin --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index aedecb0328f..efa94e9775a 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -1,7 +1,8 @@ #!/usr/bin/env bash # Tags: no-fasttest # Tag no-fasttest: setting use_metadata_cache=true is not supported in fasttest, because clickhouse binary in fasttest is build without RocksDB. - +# To suppress Warning messages from CHECK TABLE +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 6567cd7abc185d0496fd850c5d193efeb90d2109 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 13 Jan 2022 15:27:41 +0800 Subject: [PATCH 037/103] fix all the stateless test --- src/Common/tests/gtest_global_context.cpp | 17 ++++++++--- src/Common/tests/gtest_global_context.h | 11 +++++++ src/Interpreters/Context.cpp | 21 +------------- .../MergeTree/MergeTreeMetadataCache.cpp | 29 +++++++++++++++++++ .../MergeTree/MergeTreeMetadataCache.h | 2 ++ .../tests/gtest_merge_tree_metadata_cache.cpp | 10 +++++-- 6 files changed, 63 insertions(+), 27 deletions(-) rename src/{Interpreters => Storages/MergeTree}/tests/gtest_merge_tree_metadata_cache.cpp (90%) diff --git a/src/Common/tests/gtest_global_context.cpp b/src/Common/tests/gtest_global_context.cpp index 9b51e1bb99c..ec86c953c5b 100644 --- a/src/Common/tests/gtest_global_context.cpp +++ b/src/Common/tests/gtest_global_context.cpp @@ -2,8 +2,17 @@ const ContextHolder & getContext() { - static ContextHolder * holder; - static std::once_flag once; - std::call_once(once, [&]() { holder = new ContextHolder(); }); - return *holder; + return getMutableContext(); +} + +ContextHolder & getMutableContext() +{ + static ContextHolder holder; + return holder; +} + +void destroyContext() +{ + auto & holder = getMutableContext(); + return holder.destroy(); } diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 7756be7ce9b..f846a0dbe4f 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -16,6 +16,17 @@ struct ContextHolder } ContextHolder(ContextHolder &&) = default; + + void destroy() + { + context->shutdown(); + context.reset(); + shared_context.reset(); + } }; const ContextHolder & getContext(); + +ContextHolder & getMutableContext(); + +void destroyContext(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 925f3048262..111ffc9b3f8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -131,7 +131,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_READ_METHOD; - extern const int SYSTEM_ERROR; } @@ -2289,25 +2288,7 @@ void Context::initializeTraceCollector() #if USE_ROCKSDB void Context::initializeMergeTreeMetadataCache(const String & dir, size_t size) { - rocksdb::Options options; - rocksdb::BlockBasedTableOptions table_options; - rocksdb::DB * db; - - options.create_if_missing = true; - options.statistics = rocksdb::CreateDBStatistics(); - auto cache = rocksdb::NewLRUCache(size); - table_options.block_cache = cache; - options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); - rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); - - if (status != rocksdb::Status::OK()) - throw Exception( - ErrorCodes::SYSTEM_ERROR, - "Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).", - dir, - status.ToString()); - - shared->merge_tree_metadata_cache = std::make_shared(db); + shared->merge_tree_metadata_cache = MergeTreeMetadataCache::create(dir, size); } #endif diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 5df9850b318..555838204ff 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -14,6 +14,33 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +std::unique_ptr MergeTreeMetadataCache::create(const String & dir, size_t size) +{ + assert(size != 0); + rocksdb::Options options; + rocksdb::BlockBasedTableOptions table_options; + rocksdb::DB * db; + + options.create_if_missing = true; + options.statistics = rocksdb::CreateDBStatistics(); + auto cache = rocksdb::NewLRUCache(size); + table_options.block_cache = cache; + options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + rocksdb::Status status = rocksdb::DB::Open(options, dir, &db); + if (status != rocksdb::Status::OK()) + throw Exception( + ErrorCodes::SYSTEM_ERROR, + "Fail to open rocksdb path at: {} status:{}. You can try to remove the cache (this will not affect any table data).", + dir, + status.ToString()); + return std::make_unique(db); +} + MergeTreeMetadataCache::Status MergeTreeMetadataCache::put(const String & key, const String & value) { auto options = rocksdb::WriteOptions(); @@ -59,6 +86,7 @@ void MergeTreeMetadataCache::getByPrefix(const String & prefix, Strings & keys, } LOG_TRACE(log, "Seek with prefix:{} from MergeTreeMetadataCache items:{}", prefix, keys.size()); ProfileEvents::increment(ProfileEvents::MergeTreeMetadataCacheSeek); + delete it; } uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const @@ -71,6 +99,7 @@ uint64_t MergeTreeMetadataCache::getEstimateNumKeys() const void MergeTreeMetadataCache::shutdown() { rocksdb->Close(); + rocksdb.reset(); } } diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index a389d9a14ad..8e40eaf7310 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -21,6 +21,8 @@ public: assert(rocksdb); } + static std::unique_ptr create(const String & dir, size_t size); + MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; diff --git a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp similarity index 90% rename from src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp rename to src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp index 839c54c63b2..f513d1b2553 100644 --- a/src/Interpreters/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp @@ -15,9 +15,13 @@ class MergeTreeMetadataCacheTest : public ::testing::Test public: void SetUp() override { - const auto & context_holder = getContext(); - context_holder.context->initializeMergeTreeMetadataCache("./db/", 256 << 20); - cache = context_holder.context->getMergeTreeMetadataCache(); + cache = MergeTreeMetadataCache::create("./db/", 268435456); + } + + void TearDown() override + { + cache->shutdown(); + cache.reset(); } MergeTreeMetadataCachePtr cache; From cfb18e7ea0972101c52ee5861de40a065d53aad5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jan 2022 10:05:36 +0800 Subject: [PATCH 038/103] remove useless header --- src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp index f513d1b2553..33a82845545 100644 --- a/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp +++ b/src/Storages/MergeTree/tests/gtest_merge_tree_metadata_cache.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include using namespace DB; From e64ba110ef8e0ac6cb719e2ca4a9a253b54d4549 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jan 2022 19:55:44 +0800 Subject: [PATCH 039/103] fix path error --- src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp index f12af590e7d..184521cb6cf 100644 --- a/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerOrdinary.cpp @@ -20,7 +20,7 @@ PartMetadataManagerOrdinary::PartMetadataManagerOrdinary(const IMergeTreeDataPar std::unique_ptr PartMetadataManagerOrdinary::read(const String & file_name) const { - String file_path = fs::path(part->getFullRelativePath() + "/" + file_name); + String file_path = fs::path(part->getFullRelativePath()) / file_name; return openForReading(disk, file_path); } From 55382ccb1edb7682e78f3794a8e3ae019ee65d4b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 18 Jan 2022 10:37:37 +0800 Subject: [PATCH 040/103] fix stateless ests --- .../01233_check_table_with_metadata_cache.sh | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index efa94e9775a..07833f4cb39 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -12,6 +12,13 @@ database_engines=(Ordinary Atomic) use_metadata_caches=(false true) use_projections=(false true) +function materialize_projection_if_needed() +{ + if [[ "$use_projection" == "true" ]]; then + ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache MATERIALIZE PROJECTION p1" + fi +} + for table_engine in "${table_engines[@]}"; do for database_engine in "${database_engines[@]}"; do for use_metadata_cache in "${use_metadata_caches[@]}"; do @@ -34,58 +41,72 @@ for table_engine in "${table_engines[@]}"; do projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" fi ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache};" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Insert first batch of data. ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Insert second batch of data. ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # First update. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Second update. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # First delete. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Second delete. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Insert third batch of data. ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Drop one partition. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Add column. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Delete column. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Add TTL. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Modify TTL. ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" # Truncate table. ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" + materialize_projection_if_needed ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" done done From d73d4e831e20d44b31a24e10fd0df4f5cfcf5240 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 19 Jan 2022 17:08:01 +0800 Subject: [PATCH 041/103] fix bug of check table when create data part with wide format and projection --- src/Storages/MergeTree/checkDataPart.cpp | 2 +- .../0_stateless/01710_projection_part_check.reference | 1 + tests/queries/0_stateless/01710_projection_part_check.sql | 8 ++++++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 0f35e30c5d0..075e9e9fbc8 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -153,7 +153,7 @@ IMergeTreeDataPart::Checksums checkDataPart( [&](const ISerialization::SubstreamPath & substream_path) { String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin"; - checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name); + projection_checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name); }); } } diff --git a/tests/queries/0_stateless/01710_projection_part_check.reference b/tests/queries/0_stateless/01710_projection_part_check.reference index 2f7ad3359c0..813e663bdfc 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.reference +++ b/tests/queries/0_stateless/01710_projection_part_check.reference @@ -1,2 +1,3 @@ all_1_1_0 1 all_2_2_0 1 +201805_1_1_0 1 diff --git a/tests/queries/0_stateless/01710_projection_part_check.sql b/tests/queries/0_stateless/01710_projection_part_check.sql index 39fb6a89fc8..142b1363d3c 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.sql +++ b/tests/queries/0_stateless/01710_projection_part_check.sql @@ -8,3 +8,11 @@ insert into tp select number, number from numbers(5); check table tp settings check_query_single_value_result=0; drop table if exists tp; + +CREATE TABLE tp (`p` Date, `k` UInt64, `v1` UInt64, `v2` Int64, PROJECTION p1 ( SELECT p, sum(k), sum(v1), sum(v2) GROUP BY p) ) ENGINE = MergeTree PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tp (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); + +CHECK TABLE tp settings check_query_single_value_result=0; + +DROP TABLE if exists tp; \ No newline at end of file From 9253ed8836bb274cc82685f0e6ad6160b80e1d96 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 19 Jan 2022 18:28:09 +0800 Subject: [PATCH 042/103] fix bug of checkPartData --- ..._check_table_with_metadata_cache.reference | 704 +++++++++++++++++- .../01233_check_table_with_metadata_cache.sh | 139 ++-- 2 files changed, 751 insertions(+), 92 deletions(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference index 5957d23fe82..c3206dcb9cb 100644 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference @@ -1,4 +1,4 @@ -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -40,7 +40,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -82,7 +82,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -124,7 +124,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -166,7 +166,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -208,7 +208,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -250,7 +250,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -292,7 +292,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true +database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -334,7 +334,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -376,7 +376,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -418,7 +418,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -460,7 +460,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -502,7 +502,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -544,7 +544,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -586,7 +586,7 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); @@ -628,7 +628,679 @@ CHECK TABLE test_metadata_cache.check_part_metadata_cache; TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 -database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true +database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; +CHECK TABLE test_metadata_cache.check_part_metadata_cache; +1 +database engine:Atomic; table engine:ReplicatedMergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index 07833f4cb39..468a259ac66 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -11,103 +11,90 @@ table_engines=(MergeTree ReplicatedMergeTree) database_engines=(Ordinary Atomic) use_metadata_caches=(false true) use_projections=(false true) - -function materialize_projection_if_needed() -{ - if [[ "$use_projection" == "true" ]]; then - ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache MATERIALIZE PROJECTION p1" - fi -} +use_compact_data_parts=(false true) for table_engine in "${table_engines[@]}"; do for database_engine in "${database_engines[@]}"; do for use_metadata_cache in "${use_metadata_caches[@]}"; do for use_projection in "${use_projections[@]}"; do - echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}" + for use_compact_data_part in "${use_compact_data_parts[@]}"; do + echo "database engine:${database_engine}; table engine:${table_engine}; use metadata cache:${use_metadata_cache}; use projection:${use_projection}; use_compact_data_part:${use_compact_data_part}" - ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;" - ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;" - ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};" + ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_metadata_cache.check_part_metadata_cache SYNC;" + ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS test_metadata_cache;" + ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE test_metadata_cache ENGINE = ${database_engine};" - table_engine_clause="" - if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then - table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')" - elif [[ "$table_engine" == "MergeTree" ]]; then - table_engine_clause="ENGINE MergeTree()" - fi + table_engine_clause="" + if [[ "$table_engine" == "ReplicatedMergeTree" ]]; then + table_engine_clause="ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_metadata_cache/check_part_metadata_cache', 'r1')" + elif [[ "$table_engine" == "MergeTree" ]]; then + table_engine_clause="ENGINE MergeTree()" + fi - projection_clause="" - if [[ "$use_projection" == "true" ]]; then - projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" - fi - ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache};" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + projection_clause="" + if [[ "$use_projection" == "true" ]]; then + projection_clause=", projection p1 (select p, sum(k), sum(v1), sum(v2) group by p)" + fi - # Insert first batch of data. - ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + compact_data_part_clause=", min_bytes_for_wide_part = 10485760" + if [[ $use_compact_data_part == "true" ]]; then + compact_data_part_clause=", min_bytes_for_wide_part = 0" + fi + ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test_metadata_cache.check_part_metadata_cache (p Date, k UInt64, v1 UInt64, v2 Int64${projection_clause}) $table_engine_clause PARTITION BY toYYYYMM(p) ORDER BY k settings use_metadata_cache = ${use_metadata_cache} ${compact_data_part_clause}" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Insert second batch of data. - ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Insert first batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # First update. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Insert second batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Second update. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # First update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # First delete. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Second update. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Second delete. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # First delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Insert third batch of data. - ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Second delete. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Drop one partition. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Insert third batch of data. + ${CLICKHOUSE_CLIENT} --echo --query "INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000);" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Add column. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Drop one partition. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Delete column. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Add column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Add TTL. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Delete column. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Modify TTL. - ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Add TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" - # Truncate table. - ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" - materialize_projection_if_needed - ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + # Modify TTL. + ${CLICKHOUSE_CLIENT} --echo --query "ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + + # Truncate table. + ${CLICKHOUSE_CLIENT} --echo --query "TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache;" + ${CLICKHOUSE_CLIENT} --echo --query "CHECK TABLE test_metadata_cache.check_part_metadata_cache;" + done done done done From dfd199b047ec1e100b1a8ba6654535a1d68c3b2f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 24 Jan 2022 11:51:11 +0800 Subject: [PATCH 043/103] fix integration test failed --- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 1 - src/Storages/MergeTree/MergeTreeMetadataCache.h | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 555838204ff..40e84deb259 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -27,7 +27,6 @@ std::unique_ptr MergeTreeMetadataCache::create(const Str rocksdb::DB * db; options.create_if_missing = true; - options.statistics = rocksdb::CreateDBStatistics(); auto cache = rocksdb::NewLRUCache(size); table_options.block_cache = cache; options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.h b/src/Storages/MergeTree/MergeTreeMetadataCache.h index 8e40eaf7310..65c5eada200 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.h +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.h @@ -16,14 +16,15 @@ class MergeTreeMetadataCache public: using Status = rocksdb::Status; + static std::unique_ptr create(const String & dir, size_t size); + explicit MergeTreeMetadataCache(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} { assert(rocksdb); } - static std::unique_ptr create(const String & dir, size_t size); - MergeTreeMetadataCache(const MergeTreeMetadataCache &) = delete; + MergeTreeMetadataCache & operator=(const MergeTreeMetadataCache &) = delete; Status put(const String & key, const String & value); From 5398af7c054ef1cfcf170e7c8857a3816040dbf8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jan 2022 18:59:29 +0800 Subject: [PATCH 044/103] update again --- src/Storages/MergeTree/MergeTreeMetadataCache.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp index 40e84deb259..0f67346810f 100644 --- a/src/Storages/MergeTree/MergeTreeMetadataCache.cpp +++ b/src/Storages/MergeTree/MergeTreeMetadataCache.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } + std::unique_ptr MergeTreeMetadataCache::create(const String & dir, size_t size) { assert(size != 0); From 3d3b6d19566b4f22a8a5545cd38fcb7b055b810f Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Thu, 10 Mar 2022 07:23:49 -0800 Subject: [PATCH 045/103] clang-tidy report issues with Medium priority --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/SystemLog.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 4 ++-- src/Storages/StorageDistributed.h | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 16d6e2d0652..3d0bab1bb68 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1062,7 +1062,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) QualifiedTableName qualified_name{database_name, create.getTable()}; TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); if (!loading_dependencies.empty()) - DatabaseCatalog::instance().addLoadingDependencies(std::move(qualified_name), std::move(loading_dependencies)); + DatabaseCatalog::instance().addLoadingDependencies(qualified_name, std::move(loading_dependencies)); return fillTableIfNeeded(create); } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f43d108de9e..e3db0f01f86 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -185,7 +185,7 @@ Chain InterpreterInsertQuery::buildChain( std::atomic_uint64_t * elapsed_counter_ms) { auto sample = getSampleBlock(columns, table, metadata_snapshot); - return buildChainImpl(table, metadata_snapshot, std::move(sample) , thread_status, elapsed_counter_ms); + return buildChainImpl(table, metadata_snapshot, sample, thread_status, elapsed_counter_ms); } Chain InterpreterInsertQuery::buildChainImpl( diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index ec6fd98010d..201d2848dc8 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -332,7 +332,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, for (const auto & name_and_type : log_element_names_and_types) log_element_columns.emplace_back(name_and_type.type, name_and_type.name); - Block block(std::move(log_element_columns)); + Block block(log_element_columns); MutableColumns columns = block.mutateColumns(); for (const auto & elem : to_flush) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 8b08f5f28dd..3957e0ce60d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -122,7 +122,7 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->default_expression) { default_desc.kind = columnDefaultKindFromString(col_ast->default_specifier); - default_desc.expression = std::move(col_ast->default_expression); + default_desc.expression = col_ast->default_expression; } if (col_ast->comment) @@ -309,7 +309,7 @@ void ColumnsDescription::flattenNested() continue; } - ColumnDescription column = std::move(*it); + ColumnDescription column = *it; removeSubcolumns(column.name); it = columns.get<0>().erase(it); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index e47e0fddd6c..06a038f44c6 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -43,7 +43,7 @@ class StorageDistributed final : public shared_ptr_helper, p friend class StorageSystemDistributionQueue; public: - ~StorageDistributed() override; + virtual ~StorageDistributed() override; std::string getName() const override { return "Distributed"; } From a17e5efea204968d02d17b841fe6ef226ccf9298 Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Thu, 10 Mar 2022 07:31:06 -0800 Subject: [PATCH 046/103] adding destructors --- src/IO/ReadBufferFromFileDescriptor.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 188cdd709b5..4cfd064720a 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -38,6 +38,10 @@ public: , fd(fd_) { } + + virutal ~ReadBufferFromFileDescriptor() + { + } int getFD() const { @@ -80,6 +84,9 @@ public: { use_pread = true; } + virtual ~ReadBufferFromFileDescriptorPRead() + { + } }; } From 81ad3ca220d0341a603f8643f78a0daca42e2662 Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Thu, 10 Mar 2022 10:19:03 -0800 Subject: [PATCH 047/103] syle check --- src/IO/ReadBufferFromFileDescriptor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 4cfd064720a..a7d564f689b 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -38,7 +38,7 @@ public: , fd(fd_) { } - + virutal ~ReadBufferFromFileDescriptor() { } From 0de950c3f882c1b00e6433714af4beb505e062e7 Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Thu, 10 Mar 2022 10:29:57 -0800 Subject: [PATCH 048/103] style correction --- src/IO/ReadBufferFromFileDescriptor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index a7d564f689b..d075cef9003 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -39,7 +39,7 @@ public: { } - virutal ~ReadBufferFromFileDescriptor() + virtual ~ReadBufferFromFileDescriptor() { } From 137e4b79c195c816fba54e5e11317ada9886f51e Mon Sep 17 00:00:00 2001 From: Rajkumar Date: Thu, 10 Mar 2022 12:46:32 -0800 Subject: [PATCH 049/103] override needed --- src/IO/ReadBufferFromFileDescriptor.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index d075cef9003..28da16a0bf2 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -39,7 +39,7 @@ public: { } - virtual ~ReadBufferFromFileDescriptor() + virtual ~ReadBufferFromFileDescriptor() override { } @@ -84,7 +84,7 @@ public: { use_pread = true; } - virtual ~ReadBufferFromFileDescriptorPRead() + virtual ~ReadBufferFromFileDescriptorPRead() override { } }; From 1e0ad94d6731f1c65dbc5a522451d8d08de5ddda Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Mar 2022 14:08:30 +0100 Subject: [PATCH 050/103] Add a stand alone clickhouse-keeper package --- docker/packager/packager | 1 + packages/clickhouse-keeper.yaml | 38 +++++++++++++++++++++++++++++++++ 2 files changed, 39 insertions(+) create mode 100644 packages/clickhouse-keeper.yaml diff --git a/docker/packager/packager b/docker/packager/packager index a5763273f5f..18f56ee93ad 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -163,6 +163,7 @@ def parse_env_variables( cmake_flags.append("-DCMAKE_INSTALL_PREFIX=/usr") cmake_flags.append("-DCMAKE_INSTALL_SYSCONFDIR=/etc") cmake_flags.append("-DCMAKE_INSTALL_LOCALSTATEDIR=/var") + cmake_flags.append("-DBUILD_STANDALONE_KEEPER=ON") if is_release_build(build_type, package_type, sanitizer, split_binary): cmake_flags.append("-DINSTALL_STRIPPED_BINARIES=ON") diff --git a/packages/clickhouse-keeper.yaml b/packages/clickhouse-keeper.yaml new file mode 100644 index 00000000000..e717ba79c5b --- /dev/null +++ b/packages/clickhouse-keeper.yaml @@ -0,0 +1,38 @@ +# package sources should be placed in ${PWD}/root +# nfpm should run from the same directory with a config +name: "clickhouse-keeper" +arch: "${DEB_ARCH}" # amd64, arm64 +platform: "linux" +version: "${CLICKHOUSE_VERSION_STRING}" +vendor: "ClickHouse Inc." +homepage: "https://clickhouse.com" +license: "Apache" +section: "database" +priority: "optional" + +conflicts: +- clickhouse-server +depends: +- adduser + +maintainer: "ClickHouse Dev Team " +description: | + Static clickhouse-keeper binary + A stand-alone clickhouse-keeper package + + +contents: +- src: root/etc/clickhouse-keeper + dst: /etc/clickhouse-keeper + type: config +- src: root/usr/bin/clickhouse-keeper + dst: /usr/bin/clickhouse-keeper +# docs +- src: ../AUTHORS + dst: /usr/share/doc/clickhouse-keeper/AUTHORS +- src: ../CHANGELOG.md + dst: /usr/share/doc/clickhouse-keeper/CHANGELOG.md +- src: ../LICENSE + dst: /usr/share/doc/clickhouse-keeper/LICENSE +- src: ../README.md + dst: /usr/share/doc/clickhouse-keeper/README.md From a4ab73619f012acf2db96b64342829b3be4b804c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 16 Mar 2022 11:37:18 +0100 Subject: [PATCH 051/103] Fix UBSan build --- programs/keeper/CMakeLists.txt | 6 ------ src/Compression/CompressionFactory.cpp | 17 ++++++++++++++--- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 92bb5dc45a3..b1132e3aaea 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -71,17 +71,11 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressedReadBufferFromFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressedWriteBuffer.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecDelta.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecDoubleDelta.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecEncrypted.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecGorilla.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecLZ4.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecMultiple.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecNone.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecT64.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecZSTD.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionFactory.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/getCompressionCodecForFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/ICompressionCodec.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/LZ4_decompress_faster.cpp diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index ca5e5176d13..8dfc894e15b 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -165,25 +165,36 @@ void registerCodecNone(CompressionCodecFactory & factory); void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); +void registerCodecMultiple(CompressionCodecFactory & factory); + + +/// Keeper use only general-purpose codes, so we don't need these special codecs +/// in standalone build +#ifndef KEEPER_STANDALONE_BUILD + void registerCodecDelta(CompressionCodecFactory & factory); void registerCodecT64(CompressionCodecFactory & factory); void registerCodecDoubleDelta(CompressionCodecFactory & factory); void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecEncrypted(CompressionCodecFactory & factory); -void registerCodecMultiple(CompressionCodecFactory & factory); + +#endif CompressionCodecFactory::CompressionCodecFactory() { - registerCodecLZ4(*this); registerCodecNone(*this); + registerCodecLZ4(*this); registerCodecZSTD(*this); registerCodecLZ4HC(*this); + registerCodecMultiple(*this); + +#ifndef KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); registerCodecDoubleDelta(*this); registerCodecGorilla(*this); registerCodecEncrypted(*this); - registerCodecMultiple(*this); +#endif default_codec = get("LZ4", {}); } From 6fe4c3d17eb1b98bf89b071111f16e019814b84a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 23 Mar 2022 12:13:42 +0800 Subject: [PATCH 052/103] some minor changes --- src/Processors/examples/arrow_dict.cpp | 28 +++++++++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 13 +++++---- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 5 ---- .../MergeTree/MergeTreeDataPartCompact.h | 2 -- .../MergeTree/MergeTreeDataPartInMemory.cpp | 5 ---- .../MergeTree/MergeTreeDataPartInMemory.h | 2 -- .../MergeTree/MergeTreeDataPartWide.cpp | 5 ---- .../MergeTree/MergeTreeDataPartWide.h | 2 -- src/Storages/MergeTree/MergeTreePartition.cpp | 18 ------------ 10 files changed, 37 insertions(+), 45 deletions(-) create mode 100644 src/Processors/examples/arrow_dict.cpp diff --git a/src/Processors/examples/arrow_dict.cpp b/src/Processors/examples/arrow_dict.cpp new file mode 100644 index 00000000000..d83549d5814 --- /dev/null +++ b/src/Processors/examples/arrow_dict.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + + +void test() +{ + auto value_field = arrow::field("value", arrow::utf8(), false); + auto map_field = arrow::field("evenInfo", arrow::map(arrow::utf8(), value_field), false); + auto map_type = std::dynamic_pointer_cast(map_field->type()); + std::cout << "field:" << map_field->ToString() << std::endl; + std::cout << "key_type: " << map_type->key_type()->ToString() << std::endl; + std::cout << "val_type: " << map_type->value_type()->ToString() << std::endl; + std::cout << "item_type: " << map_type->item_type()->ToString() << std::endl; + + arrow::MemoryPool * pool = arrow::default_memory_pool(); + std::unique_ptr array_builder; + arrow::Status status = arrow::MakeBuilder(pool, map_field->type(), &array_builder); + std::shared_ptr arrow_array; + status = array_builder->Finish(&arrow_array); + std::cout << status.ToString() << std::endl; +} + +int main() +{ + test(); + return 0; +} \ No newline at end of file diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b845d869487..8201b5078f5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -701,9 +701,9 @@ void IMergeTreeDataPart::loadIndexGranularity() throw Exception("Method 'loadIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); } +/// Currently we don't cache mark files of part, because cache other meta files is enough to speed up loading. void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) const { - throw Exception("Method 'appendFilesOfIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); } void IMergeTreeDataPart::loadIndex() @@ -985,7 +985,6 @@ void IMergeTreeDataPart::loadRowsCount() auto read_rows_count = [&]() { - // auto buf = openForReading(volume->getDisk(), path); auto buf = metadata_manager->read("count.txt"); readIntText(rows_count, *buf); assertEOF(*buf); @@ -1201,9 +1200,12 @@ void IMergeTreeDataPart::loadColumns(bool require) }; SerializationInfoByName infos(loaded_columns, settings); - path = getFullRelativePath() + SERIALIZATION_FILE_NAME; - if (volume->getDisk()->exists(path)) - infos.readJSON(*volume->getDisk()->readFile(path)); + exists = metadata_manager->exists(SERIALIZATION_FILE_NAME); + if (exists) + { + auto in = metadata_manager->read(SERIALIZATION_FILE_NAME); + infos.readJSON(*in); + } setColumns(loaded_columns); setSerializationInfos(infos); @@ -1212,6 +1214,7 @@ void IMergeTreeDataPart::loadColumns(bool require) void IMergeTreeDataPart::appendFilesOfColumns(Strings & files) { files.push_back("columns.txt"); + files.push_back(SERIALIZATION_FILE_NAME); } bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 01adb98c129..47739fef8ed 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -467,7 +467,7 @@ protected: std::map> projection_parts; - /// Disabled when USE_ROCKSDB is OFF, or use_metadata_cache is set true in merge tree settings + /// Disabled when USE_ROCKSDB is OFF or use_metadata_cache is set to false in merge tree settings bool use_metadata_cache = false; mutable PartMetadataManagerPtr metadata_manager; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 6a747960a40..f4da730b1f0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -192,9 +192,4 @@ MergeTreeDataPartCompact::~MergeTreeDataPartCompact() removeIfNeeded(); } -/// We don't cache mark file, because cache other metadata files is enough to speed up loading. -void MergeTreeDataPartCompact::appendFilesOfIndexGranularity(Strings& /* files */) const -{ -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index b96afc4b972..38bfa11652a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -72,8 +72,6 @@ private: /// Compact parts doesn't support per column size, only total size void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; - - void appendFilesOfIndexGranularity(Strings& files) const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 08417ab8f2c..e4a174a7d29 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -166,11 +166,6 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() return checksum; } -/// No mark files for part in memory -void MergeTreeDataPartInMemory::appendFilesOfIndexGranularity(Strings& /* files */) const -{ -} - DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) { return std::dynamic_pointer_cast(part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 068b15d2bdc..c5ee9ebd01f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -62,8 +62,6 @@ private: /// Calculates uncompressed sizes in memory. void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; - - void appendFilesOfIndexGranularity(Strings & files) const override; }; using DataPartInMemoryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 1582a7f3274..b279c1aba6a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -268,9 +268,4 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col } } -/// We don't cache mark files of part, because cache other meta files is enough to speed up loading. -void MergeTreeDataPartWide::appendFilesOfIndexGranularity(Strings& /* files */) const -{ -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 2b11fc4eb02..078dda394ee 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -66,8 +66,6 @@ private: /// Loads marks index granularity into memory void loadIndexGranularity() override; - void appendFilesOfIndexGranularity(Strings & files) const override; - ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set * processed_substreams) const; void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 8c7ef2fcb53..4955a63387c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -375,25 +375,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; -/* -#if USE_ROCKSDB - std::unique_ptr file; - auto partition_file_path = part_path + "partition.dat"; - String _; - if (metadata_cache) - { - file = metadata_cache->readOrSet(disk, "partition.dat", _); - } - else - { - file = openForReading(disk, partition_file_path); - } -#else - file = openForReading(disk, partition_file_path); -#endif -*/ auto file = manager->read("partition.dat"); - value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file); From 07bf891c8590e921902522897d3e1356055855b8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 23 Mar 2022 21:56:01 +0800 Subject: [PATCH 053/103] rm useless file --- src/Processors/examples/arrow_dict.cpp | 28 -------------------------- 1 file changed, 28 deletions(-) delete mode 100644 src/Processors/examples/arrow_dict.cpp diff --git a/src/Processors/examples/arrow_dict.cpp b/src/Processors/examples/arrow_dict.cpp deleted file mode 100644 index d83549d5814..00000000000 --- a/src/Processors/examples/arrow_dict.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include -#include -#include - - -void test() -{ - auto value_field = arrow::field("value", arrow::utf8(), false); - auto map_field = arrow::field("evenInfo", arrow::map(arrow::utf8(), value_field), false); - auto map_type = std::dynamic_pointer_cast(map_field->type()); - std::cout << "field:" << map_field->ToString() << std::endl; - std::cout << "key_type: " << map_type->key_type()->ToString() << std::endl; - std::cout << "val_type: " << map_type->value_type()->ToString() << std::endl; - std::cout << "item_type: " << map_type->item_type()->ToString() << std::endl; - - arrow::MemoryPool * pool = arrow::default_memory_pool(); - std::unique_ptr array_builder; - arrow::Status status = arrow::MakeBuilder(pool, map_field->type(), &array_builder); - std::shared_ptr arrow_array; - status = array_builder->Finish(&arrow_array); - std::cout << status.ToString() << std::endl; -} - -int main() -{ - test(); - return 0; -} \ No newline at end of file From abd587cb23fcfd424d3b5c2eb155193770115c08 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 23 Mar 2022 16:17:44 +0000 Subject: [PATCH 054/103] Better tests --- .../test.py | 4 ++-- .../test_backup_with_other_granularity/test.py | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py b/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py index 08fb6e53e7b..ead5ab7f18e 100644 --- a/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py @@ -12,6 +12,7 @@ TABLE_NAME = "blob_storage_table" CONTAINER_NAME = "cont" CLUSTER_NAME = "test_cluster" +drop_table_statement = f"DROP TABLE {TABLE_NAME} ON CLUSTER {CLUSTER_NAME} SYNC" @pytest.fixture(scope="module") def cluster(): @@ -47,11 +48,9 @@ def create_table(node, table_name, replica, **additional_settings): ORDER BY id SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}""" - node.query(f"DROP TABLE IF EXISTS {table_name}") node.query(create_table_statement) assert node.query(f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" - def get_large_objects_count(blob_container_client, large_size_threshold=100): return sum(blob['size'] > large_size_threshold for blob in blob_container_client.list_blobs()) @@ -81,3 +80,4 @@ def test_zero_copy_replication(cluster): assert node1.query(f"SELECT * FROM {TABLE_NAME} order by id FORMAT Values") == values1 + "," + values2 assert get_large_objects_count(blob_container_client) == 2 + node1.query(drop_table_statement) diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index 0f35c0f849e..de61bfdb1d8 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -60,6 +60,9 @@ def test_backup_from_old_version(started_cluster): assert node1.query("CHECK TABLE dest_table") == "1\n" + node1.query("DROP TABLE source_table") + node1.query("DROP TABLE dest_table") + def test_backup_from_old_version_setting(started_cluster): node2.query("CREATE TABLE source_table(A Int64, B String) Engine = MergeTree order by tuple()") @@ -97,6 +100,9 @@ def test_backup_from_old_version_setting(started_cluster): assert node2.query("CHECK TABLE dest_table") == "1\n" + node2.query("DROP TABLE source_table") + node2.query("DROP TABLE dest_table") + def test_backup_from_old_version_config(started_cluster): node3.query("CREATE TABLE source_table(A Int64, B String) Engine = MergeTree order by tuple()") @@ -138,6 +144,8 @@ def test_backup_from_old_version_config(started_cluster): assert node3.query("CHECK TABLE dest_table") == "1\n" + node3.query("DROP TABLE source_table") + node3.query("DROP TABLE dest_table") def test_backup_and_alter(started_cluster): node4.query("CREATE DATABASE test ENGINE=Ordinary") # Different path in shadow/ with Atomic @@ -160,3 +168,6 @@ def test_backup_and_alter(started_cluster): assert node4.query("SELECT sum(A) FROM test.backup_table") == "2\n" assert node4.query("SELECT B + 2 FROM test.backup_table") == "4\n" + + node4.query("DROP TABLE test.backup_table") + node4.query("DROP DATABASE test") From e6cc5364e090d9c27a23d177725a890d10c5f3d3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 24 Mar 2022 11:22:35 +0800 Subject: [PATCH 055/103] add long tag for test --- .../0_stateless/01233_check_table_with_metadata_cache.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index 468a259ac66..38f48372035 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, long # Tag no-fasttest: setting use_metadata_cache=true is not supported in fasttest, because clickhouse binary in fasttest is build without RocksDB. # To suppress Warning messages from CHECK TABLE CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error From db34ca57c2ca1db07a798f9b80aaef3fbeae3047 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 24 Mar 2022 14:58:45 +0800 Subject: [PATCH 056/103] reformat test.py --- tests/integration/test_multiple_disks/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index a81dbee1941..367948e8751 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1129,7 +1129,12 @@ def produce_alter_move(node, name): "name,engine,use_metadata_cache", [ pytest.param("concurrently_altering_mt", "MergeTree()", "false", id="mt"), - pytest.param("concurrently_altering_mt", "MergeTree()", "true", id="mt_use_metadata_cache"), + pytest.param( + "concurrently_altering_mt", + "MergeTree()", + "true", + id="mt_use_metadata_cache", + ), pytest.param( "concurrently_altering_replicated_mt", "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", From a75ded9da3f611567a5a4209f7e5fcb1a8d1184b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 24 Mar 2022 15:55:01 +0800 Subject: [PATCH 057/103] fix stateless tests --- ..._check_table_with_metadata_cache.reference | 672 ------------------ .../01233_check_table_with_metadata_cache.sh | 2 +- 2 files changed, 1 insertion(+), 673 deletions(-) diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference index c3206dcb9cb..b773fc49ec3 100644 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.reference @@ -1,675 +1,3 @@ -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Ordinary; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:false; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:false; use projection:true; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:false; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:false -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -database engine:Atomic; table engine:MergeTree; use metadata cache:true; use projection:true; use_compact_data_part:true -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v1 = 2001 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache update v2 = 4002 where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 1 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache delete where k = 8 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -INSERT INTO test_metadata_cache.check_part_metadata_cache (p, k, v1, v2) VALUES ('2018-06-15', 5, 1000, 2000), ('2018-06-16', 6, 3000, 4000), ('2018-06-17', 7, 5000, 6000), ('2018-06-18', 8, 7000, 8000); -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop partition 201805 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache add column v3 UInt64 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache drop column v3 settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 10 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -ALTER TABLE test_metadata_cache.check_part_metadata_cache modify TTL p + INTERVAL 15 YEAR settings mutations_sync = 1, replication_alter_partitions_sync = 1; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 -TRUNCATE TABLE test_metadata_cache.check_part_metadata_cache; -CHECK TABLE test_metadata_cache.check_part_metadata_cache; -1 database engine:Ordinary; table engine:ReplicatedMergeTree; use metadata cache:false; use projection:false; use_compact_data_part:false CHECK TABLE test_metadata_cache.check_part_metadata_cache; 1 diff --git a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh index 38f48372035..c73ea97a148 100755 --- a/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh +++ b/tests/queries/0_stateless/01233_check_table_with_metadata_cache.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -table_engines=(MergeTree ReplicatedMergeTree) +table_engines=(ReplicatedMergeTree) database_engines=(Ordinary Atomic) use_metadata_caches=(false true) use_projections=(false true) From d9d9f3bc6dfea4f8a357f84de2b5f10f0b170dd7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 24 Mar 2022 19:09:13 +0100 Subject: [PATCH 058/103] Fix --- .../ExternalDataSourceConfiguration.cpp | 5 + tests/config/config.d/named_collection.xml | 8 + .../02244_url_engine_headers_test.python | 141 ++++++++++++++++++ .../02244_url_engine_headers_test.reference | 1 + .../02244_url_engine_headers_test.sh | 7 + 5 files changed, 162 insertions(+) create mode 100644 tests/queries/0_stateless/02244_url_engine_headers_test.python create mode 100644 tests/queries/0_stateless/02244_url_engine_headers_test.reference create mode 100755 tests/queries/0_stateless/02244_url_engine_headers_test.sh diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 5549a816a06..abd20e6e5fd 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -325,6 +325,7 @@ void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration compression_method = conf.compression_method; structure = conf.structure; http_method = conf.http_method; + headers = conf.headers; } @@ -364,6 +365,10 @@ std::optional getURLBasedDataSourceConfiguration(const { configuration.structure = config.getString(config_prefix + ".structure", ""); } + else if (key == "compression_method") + { + configuration.compression_method = config.getString(config_prefix + ".compression_method", ""); + } else if (key == "headers") { Poco::Util::AbstractConfiguration::Keys header_keys; diff --git a/tests/config/config.d/named_collection.xml b/tests/config/config.d/named_collection.xml index f3b7074e1ce..7d60533525e 100644 --- a/tests/config/config.d/named_collection.xml +++ b/tests/config/config.d/named_collection.xml @@ -14,5 +14,13 @@ default s
+ + +
+ test_header + test_header_clickhouse +
+
+
diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.python b/tests/queries/0_stateless/02244_url_engine_headers_test.python new file mode 100644 index 00000000000..f9c594233a0 --- /dev/null +++ b/tests/queries/0_stateless/02244_url_engine_headers_test.python @@ -0,0 +1,141 @@ +#!/usr/bin/env python3 + +from http.server import SimpleHTTPRequestHandler,HTTPServer +import socket +import sys +import threading +import os +import traceback +import urllib.request +import subprocess + + +def is_ipv6(host): + try: + socket.inet_aton(host) + return False + except: + return True + +def get_local_port(host, ipv6): + if ipv6: + family = socket.AF_INET6 + else: + family = socket.AF_INET + + with socket.socket(family) as fd: + fd.bind((host, 0)) + return fd.getsockname()[1] + +CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', 'localhost') +CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') + +# Server returns this JSON response. +SERVER_JSON_RESPONSE = \ +'''{ + "login": "ClickHouse", + "id": 54801242, + "name": "ClickHouse", + "company": null +}''' + +EXPECTED_ANSWER = \ +'''{\\n\\t"login": "ClickHouse",\\n\\t"id": 54801242,\\n\\t"name": "ClickHouse",\\n\\t"company": null\\n}''' + +##################################################################################### +# This test starts an HTTP server and serves data to clickhouse url-engine based table. +# The objective of this test is to check the ClickHouse server provides a User-Agent +# with HTTP requests. +# In order for it to work ip+port of http server (given below) should be +# accessible from clickhouse server. +##################################################################################### + +# IP-address of this host accessible from the outside world. Get the first one +HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] +IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) +HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) + +# IP address and port of the HTTP server started from this script. +HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) +if IS_IPV6: + HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" +else: + HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" + + +def get_ch_answer(query): + host = CLICKHOUSE_HOST + if IS_IPV6: + host = f'[{host}]' + + url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) + return urllib.request.urlopen(url, data=query.encode()).read().decode() + +def check_answers(query, answer): + ch_answer = get_ch_answer(query) + if ch_answer.strip() != answer.strip(): + print("FAIL on query:", query, file=sys.stderr) + print("Expected answer:", answer, file=sys.stderr) + print("Fetched answer :", ch_answer, file=sys.stderr) + raise Exception("Fail on query") + +class HttpProcessor(SimpleHTTPRequestHandler): + def _set_headers(self): + test_header = self.headers.get('test_header') + if test_header and test_header.startswith('test_header_clickhouse'): + self.send_response(200) + else: + self.send_response(403) + + self.send_header('Content-Type', 'text/csv') + self.end_headers() + + def do_GET(self): + self._set_headers() + self.wfile.write(SERVER_JSON_RESPONSE.encode()) + + def log_message(self, format, *args): + return + +class HTTPServerV6(HTTPServer): + address_family = socket.AF_INET6 + +def start_server(requests_amount): + if IS_IPV6: + httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) + else: + httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) + + def real_func(): + for i in range(requests_amount): + httpd.handle_request() + + t = threading.Thread(target=real_func) + return t + +##################################################################### +# Testing area. +##################################################################### + +def test_select(): + global HTTP_SERVER_URL_STR + query = 'SELECT * FROM url(url_with_headers, url=\'{}\', format=\'JSONAsString\');'.format(HTTP_SERVER_URL_STR) + check_answers(query, EXPECTED_ANSWER) + +def main(): + t = start_server(2) + t.start() + test_select() + t.join() + print("PASSED") + +if __name__ == "__main__": + try: + main() + except Exception as ex: + exc_type, exc_value, exc_traceback = sys.exc_info() + traceback.print_tb(exc_traceback, file=sys.stderr) + print(ex, file=sys.stderr) + sys.stderr.flush() + + os._exit(1) diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.reference b/tests/queries/0_stateless/02244_url_engine_headers_test.reference new file mode 100644 index 00000000000..53cdf1e9393 --- /dev/null +++ b/tests/queries/0_stateless/02244_url_engine_headers_test.reference @@ -0,0 +1 @@ +PASSED diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.sh b/tests/queries/0_stateless/02244_url_engine_headers_test.sh new file mode 100755 index 00000000000..37d89907d79 --- /dev/null +++ b/tests/queries/0_stateless/02244_url_engine_headers_test.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +python3 "$CURDIR"/02244_url_engine_headers_test.python From 86dcbe100c94eb802318ef29de2da3487881fed6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Mar 2022 11:25:42 +0100 Subject: [PATCH 059/103] Add dbg symbols --- packages/clickhouse-keeper-dbg.yaml | 32 +++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 packages/clickhouse-keeper-dbg.yaml diff --git a/packages/clickhouse-keeper-dbg.yaml b/packages/clickhouse-keeper-dbg.yaml new file mode 100644 index 00000000000..e5d9419eb89 --- /dev/null +++ b/packages/clickhouse-keeper-dbg.yaml @@ -0,0 +1,32 @@ +# package sources should be placed in ${PWD}/root +# nfpm should run from the same directory with a config +name: "clickhouse-keeper-dbg" +arch: "${DEB_ARCH}" # amd64, arm64 +platform: "linux" +version: "${CLICKHOUSE_VERSION_STRING}" +vendor: "ClickHouse Inc." +homepage: "https://clickhouse.com" +license: "Apache" +section: "database" +priority: "optional" + +conflicts: +- clickhouse-server + +maintainer: "ClickHouse Dev Team " +description: | + debugging symbols for clickhouse-keeper + This package contains the debugging symbols for clickhouse-keeper. + +contents: +- src: root/usr/lib/debug/clickhouse-keeper.debug + dst: /usr/lib/debug/clickhouse-keeper.debug +# docs +- src: ../AUTHORS + dst: /usr/share/doc/clickhouse-keeper-dbg/AUTHORS +- src: ../CHANGELOG.md + dst: /usr/share/doc/clickhouse-keeper-dbg/CHANGELOG.md +- src: ../LICENSE + dst: /usr/share/doc/clickhouse-keeper-dbg/LICENSE +- src: ../README.md + dst: /usr/share/doc/clickhouse-keeper-dbg/README.md From 200abb51dca03501abea27d8015dd10ef1f1b674 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 25 Mar 2022 09:53:56 +0100 Subject: [PATCH 060/103] Better test --- tests/config/config.d/named_collection.xml | 4 +- .../02244_url_engine_headers_test.python | 141 ------------------ .../02244_url_engine_headers_test.reference | 2 +- .../02244_url_engine_headers_test.sh | 7 - .../02244_url_engine_headers_test.sql | 1 + 5 files changed, 4 insertions(+), 151 deletions(-) delete mode 100644 tests/queries/0_stateless/02244_url_engine_headers_test.python delete mode 100755 tests/queries/0_stateless/02244_url_engine_headers_test.sh create mode 100644 tests/queries/0_stateless/02244_url_engine_headers_test.sql diff --git a/tests/config/config.d/named_collection.xml b/tests/config/config.d/named_collection.xml index 7d60533525e..bc75461d664 100644 --- a/tests/config/config.d/named_collection.xml +++ b/tests/config/config.d/named_collection.xml @@ -17,8 +17,8 @@
- test_header - test_header_clickhouse + X-ClickHouse-Format + JSONEachRow
diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.python b/tests/queries/0_stateless/02244_url_engine_headers_test.python deleted file mode 100644 index f9c594233a0..00000000000 --- a/tests/queries/0_stateless/02244_url_engine_headers_test.python +++ /dev/null @@ -1,141 +0,0 @@ -#!/usr/bin/env python3 - -from http.server import SimpleHTTPRequestHandler,HTTPServer -import socket -import sys -import threading -import os -import traceback -import urllib.request -import subprocess - - -def is_ipv6(host): - try: - socket.inet_aton(host) - return False - except: - return True - -def get_local_port(host, ipv6): - if ipv6: - family = socket.AF_INET6 - else: - family = socket.AF_INET - - with socket.socket(family) as fd: - fd.bind((host, 0)) - return fd.getsockname()[1] - -CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', 'localhost') -CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') - -# Server returns this JSON response. -SERVER_JSON_RESPONSE = \ -'''{ - "login": "ClickHouse", - "id": 54801242, - "name": "ClickHouse", - "company": null -}''' - -EXPECTED_ANSWER = \ -'''{\\n\\t"login": "ClickHouse",\\n\\t"id": 54801242,\\n\\t"name": "ClickHouse",\\n\\t"company": null\\n}''' - -##################################################################################### -# This test starts an HTTP server and serves data to clickhouse url-engine based table. -# The objective of this test is to check the ClickHouse server provides a User-Agent -# with HTTP requests. -# In order for it to work ip+port of http server (given below) should be -# accessible from clickhouse server. -##################################################################################### - -# IP-address of this host accessible from the outside world. Get the first one -HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip().split()[0] -IS_IPV6 = is_ipv6(HTTP_SERVER_HOST) -HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST, IS_IPV6) - -# IP address and port of the HTTP server started from this script. -HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) -if IS_IPV6: - HTTP_SERVER_URL_STR = 'http://' + f'[{str(HTTP_SERVER_ADDRESS[0])}]:{str(HTTP_SERVER_ADDRESS[1])}' + "/" -else: - HTTP_SERVER_URL_STR = 'http://' + f'{str(HTTP_SERVER_ADDRESS[0])}:{str(HTTP_SERVER_ADDRESS[1])}' + "/" - - -def get_ch_answer(query): - host = CLICKHOUSE_HOST - if IS_IPV6: - host = f'[{host}]' - - url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) - return urllib.request.urlopen(url, data=query.encode()).read().decode() - -def check_answers(query, answer): - ch_answer = get_ch_answer(query) - if ch_answer.strip() != answer.strip(): - print("FAIL on query:", query, file=sys.stderr) - print("Expected answer:", answer, file=sys.stderr) - print("Fetched answer :", ch_answer, file=sys.stderr) - raise Exception("Fail on query") - -class HttpProcessor(SimpleHTTPRequestHandler): - def _set_headers(self): - test_header = self.headers.get('test_header') - if test_header and test_header.startswith('test_header_clickhouse'): - self.send_response(200) - else: - self.send_response(403) - - self.send_header('Content-Type', 'text/csv') - self.end_headers() - - def do_GET(self): - self._set_headers() - self.wfile.write(SERVER_JSON_RESPONSE.encode()) - - def log_message(self, format, *args): - return - -class HTTPServerV6(HTTPServer): - address_family = socket.AF_INET6 - -def start_server(requests_amount): - if IS_IPV6: - httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor) - else: - httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor) - - def real_func(): - for i in range(requests_amount): - httpd.handle_request() - - t = threading.Thread(target=real_func) - return t - -##################################################################### -# Testing area. -##################################################################### - -def test_select(): - global HTTP_SERVER_URL_STR - query = 'SELECT * FROM url(url_with_headers, url=\'{}\', format=\'JSONAsString\');'.format(HTTP_SERVER_URL_STR) - check_answers(query, EXPECTED_ANSWER) - -def main(): - t = start_server(2) - t.start() - test_select() - t.join() - print("PASSED") - -if __name__ == "__main__": - try: - main() - except Exception as ex: - exc_type, exc_value, exc_traceback = sys.exc_info() - traceback.print_tb(exc_traceback, file=sys.stderr) - print(ex, file=sys.stderr) - sys.stderr.flush() - - os._exit(1) diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.reference b/tests/queries/0_stateless/02244_url_engine_headers_test.reference index 53cdf1e9393..7b96f09d82e 100644 --- a/tests/queries/0_stateless/02244_url_engine_headers_test.reference +++ b/tests/queries/0_stateless/02244_url_engine_headers_test.reference @@ -1 +1 @@ -PASSED +{"12":12}\n diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.sh b/tests/queries/0_stateless/02244_url_engine_headers_test.sh deleted file mode 100755 index 37d89907d79..00000000000 --- a/tests/queries/0_stateless/02244_url_engine_headers_test.sh +++ /dev/null @@ -1,7 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -python3 "$CURDIR"/02244_url_engine_headers_test.python diff --git a/tests/queries/0_stateless/02244_url_engine_headers_test.sql b/tests/queries/0_stateless/02244_url_engine_headers_test.sql new file mode 100644 index 00000000000..e71a933346c --- /dev/null +++ b/tests/queries/0_stateless/02244_url_engine_headers_test.sql @@ -0,0 +1 @@ +select * from url(url_with_headers, url='http://127.0.0.1:8123?query=select+12', format='RawBLOB'); From ecc57b436c161547637938984b2cddc2bda4f844 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 25 Mar 2022 13:30:45 +0000 Subject: [PATCH 061/103] Check remote_url_allow_hosts before schema inference in URL engine --- src/Storages/StorageURL.cpp | 2 ++ tests/integration/test_allowed_url_from_config/test.py | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 5c8a7ea2be5..c30e1bdbd10 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -560,6 +560,8 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( const std::optional & format_settings, ContextPtr context) { + context->getRemoteHostFilter().checkURL(Poco::URI(uri)); + Poco::Net::HTTPBasicCredentials credentials; std::vector urls_to_check; diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 4f4f02fffdc..3306d26da6c 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -276,3 +276,9 @@ def test_HDFS(start_cluster): assert "not allowed" in node7.query_and_get_error( "SELECT * FROM hdfs('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'word String')" ) + + +def test_schema_inference(start_cluster): + error = node7.query_and_get_error("desc url('http://test.com`, 'TSVRaw'')") + assert(error.find('ReadWriteBufferFromHTTPBase') == -1) + From 543c46d2fda34d2c61720a3ba6455f09e19cbcb4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Mar 2022 17:29:15 +0100 Subject: [PATCH 062/103] Fix dbg package --- packages/clickhouse-keeper-dbg.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/packages/clickhouse-keeper-dbg.yaml b/packages/clickhouse-keeper-dbg.yaml index e5d9419eb89..685bff11080 100644 --- a/packages/clickhouse-keeper-dbg.yaml +++ b/packages/clickhouse-keeper-dbg.yaml @@ -19,8 +19,8 @@ description: | This package contains the debugging symbols for clickhouse-keeper. contents: -- src: root/usr/lib/debug/clickhouse-keeper.debug - dst: /usr/lib/debug/clickhouse-keeper.debug +- src: root/usr/lib/debug/usr/bin/clickhouse-keeper.debug + dst: /usr/lib/debug/usr/bin/clickhouse-keeper.debug # docs - src: ../AUTHORS dst: /usr/share/doc/clickhouse-keeper-dbg/AUTHORS From b3d80c7822cc47104012b0409d8d8ce091592b3b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Mar 2022 20:12:19 +0100 Subject: [PATCH 063/103] test From 6da2c48b7acca8f926013f455452730672718d5b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 14 Mar 2022 15:45:09 +0300 Subject: [PATCH 064/103] Add missing __builtin_unreachable for switch/case for gcc Signed-off-by: Azat Khuzhin --- src/Common/FileSegment.cpp | 1 + src/Disks/IO/CachedReadBufferFromRemoteFS.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index ceb16249c66..ff4d114c71b 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -437,6 +437,7 @@ String FileSegment::stateToString(FileSegment::State state) case FileSegment::State::SKIP_CACHE: return "SKIP_CACHE"; } + __builtin_unreachable(); } String FileSegmentsHolder::toString() diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.h b/src/Disks/IO/CachedReadBufferFromRemoteFS.h index 3d03debcd01..0e9dcc1033f 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.h +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.h @@ -96,6 +96,7 @@ private: case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: return "REMOTE_FS_READ_AND_PUT_IN_CACHE"; } + __builtin_unreachable(); } size_t first_offset = 0; }; From a847b71611156af796a6fe1166f6a5538f26f2f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 21 Mar 2022 07:03:43 +0300 Subject: [PATCH 065/103] Fix SubcolumnsTree declaration Signed-off-by: Azat Khuzhin --- src/Columns/ColumnObject.cpp | 6 +++--- src/Columns/ColumnObject.h | 10 +++++----- src/DataTypes/Serializations/SerializationObject.cpp | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 54439edb93d..64c7a84c263 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -521,7 +521,7 @@ ColumnObject::ColumnObject(bool is_nullable_) { } -ColumnObject::ColumnObject(SubcolumnsTree && subcolumns_, bool is_nullable_) +ColumnObject::ColumnObject(Subcolumns && subcolumns_, bool is_nullable_) : is_nullable(is_nullable_) , subcolumns(std::move(subcolumns_)) , num_rows(subcolumns.empty() ? 0 : (*subcolumns.begin())->data.size()) @@ -696,7 +696,7 @@ const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & ke ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) { if (const auto * node = subcolumns.findLeaf(key)) - return const_cast(node)->data; + return const_cast(node)->data; throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObject", key.getPath()); } @@ -794,7 +794,7 @@ bool ColumnObject::isFinalized() const void ColumnObject::finalize() { size_t old_size = size(); - SubcolumnsTree new_subcolumns; + Subcolumns new_subcolumns; for (auto && entry : subcolumns) { const auto & least_common_type = entry->data.getLeastCommonType(); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index fe8763f3777..9d61b165042 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -138,20 +138,20 @@ public: size_t num_of_defaults_in_prefix = 0; }; - using SubcolumnsTree = SubcolumnsTree; + using Subcolumns = SubcolumnsTree; private: /// If true then all subcolumns are nullable. const bool is_nullable; - SubcolumnsTree subcolumns; + Subcolumns subcolumns; size_t num_rows; public: static constexpr auto COLUMN_NAME_DUMMY = "_dummy"; explicit ColumnObject(bool is_nullable_); - ColumnObject(SubcolumnsTree && subcolumns_, bool is_nullable_); + ColumnObject(Subcolumns && subcolumns_, bool is_nullable_); /// Checks that all subcolumns have consistent sizes. void checkConsistency() const; @@ -173,8 +173,8 @@ public: /// It cares about consistency of sizes of Nested arrays. void addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size); - const SubcolumnsTree & getSubcolumns() const { return subcolumns; } - SubcolumnsTree & getSubcolumns() { return subcolumns; } + const Subcolumns & getSubcolumns() const { return subcolumns; } + Subcolumns & getSubcolumns() { return subcolumns; } PathsInData getKeys() const; /// Finalizes all subcolumns. diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 60b196092ed..fb8ccd2e5e6 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -63,12 +63,12 @@ private: size_t num_dimensions_to_keep; }; -using Node = typename ColumnObject::SubcolumnsTree::Node; +using Node = typename ColumnObject::Subcolumns::Node; /// Finds a subcolumn from the same Nested type as @entry and inserts /// an array with default values with consistent sizes as in Nested type. bool tryInsertDefaultFromNested( - const std::shared_ptr & entry, const ColumnObject::SubcolumnsTree & subcolumns) + const std::shared_ptr & entry, const ColumnObject::Subcolumns & subcolumns) { if (!entry->path.hasNested()) return false; From be4fea335c13f374688de2cf79878ae033804845 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 21 Mar 2022 19:22:12 +0300 Subject: [PATCH 066/103] Fix non-existing include dir (rapidjson) gcc reports: Mar 21 16:10:33 cc1plus: error: ../src/ch_contrib::rapidjson: No such file or directory [-Werror=missing-include-dirs] Signed-off-by: Azat Khuzhin --- src/CMakeLists.txt | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b24181625d3..145015ad0f2 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -573,10 +573,6 @@ if (ENABLE_TESTS) target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::simdjson) endif() - if(TARGET ch_contrib::rapidjson) - target_include_directories(unit_tests_dbms PRIVATE ch_contrib::rapidjson) - endif() - if (TARGET ch_contrib::yaml_cpp) target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::yaml_cpp) endif() From 55e9f604489049959ee796361ff1b20252602c8a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 21 Mar 2022 21:32:55 +0300 Subject: [PATCH 067/103] Fix building of SerializationObject gcc11 reports: ld.lld: error: undefined symbol: void DB::readEscapedStringInto, std::__1::allocator > >(std::__1::basic_string, std::__1::allocator >&, DB::ReadBuffer&) >>> referenced by SerializationObject.cpp:201 (./src/DataTypes/Serializations/SerializationObject.cpp:201) >>> SerializationObject.cpp.o:(void DB::SerializationObject >::deserializeTextImpl >::deserializeTextEscaped(DB::IColumn&, DB::ReadBuffer&, DB::FormatSettings const&) const::'lambda'(std::__1::basic_string, std::__1::allocator >&)>(DB::IColumn&, DB::SerializationObject >::deserializeTextEscaped(DB::IColumn&, DB::ReadBuffer&, DB::FormatSettings const&) const::'lambda'(std::__1::basic_string, std::__1::allocator >&)&&) const) in archive src/libdbms.a collect2: error: ld returned 1 exit status ninja: build stopped: subcommand failed. Signed-off-by: Azat Khuzhin --- src/DataTypes/Serializations/SerializationObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index fb8ccd2e5e6..64d79d107e1 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -198,7 +198,7 @@ void SerializationObject::deserializeWholeText(IColumn & column, ReadBuf template void SerializationObject::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - deserializeTextImpl(column, [&](String & s) { readEscapedStringInto(s, istr); }); + deserializeTextImpl(column, [&](String & s) { readEscapedString(s, istr); }); } template From d51861452457fac64beb06ee39725c288e36fd93 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 26 Mar 2022 23:41:03 +0300 Subject: [PATCH 068/103] Fix multi-line comment in threadPoolCallbackRunner.cpp --- src/Interpreters/threadPoolCallbackRunner.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/threadPoolCallbackRunner.cpp b/src/Interpreters/threadPoolCallbackRunner.cpp index fadad235039..c580d352c49 100644 --- a/src/Interpreters/threadPoolCallbackRunner.cpp +++ b/src/Interpreters/threadPoolCallbackRunner.cpp @@ -25,7 +25,7 @@ CallbackRunner threadPoolCallbackRunner(ThreadPool & pool) /// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed. /// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well. /// When, finally, we destroy the thread (and the ThreadStatus), - /// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,\ + /// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory, /// and by this time user-level memory tracker may be already destroyed. /// /// As a work-around, reset memory tracker to total, which is always alive. From 6a3986f41bd31f28660e7c9e352aa03aeaa331a6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 26 Mar 2022 23:41:29 +0300 Subject: [PATCH 069/103] Fix redundant-move in ReadWriteBufferFromHTTP.h --- src/IO/ReadWriteBufferFromHTTP.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 061dd772212..40a15ace4ad 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -663,7 +663,7 @@ public: Range range{from, to}; from = to; - return std::move(range); + return range; } private: From 806a4bcdee181200783946da8071ca0323babc1d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 11 Mar 2022 10:29:14 +0300 Subject: [PATCH 070/103] Add build with GCC v2: Add gcc build to "ClickHouse special build check (actions)" Signed-off-by: Azat Khuzhin --- .github/workflows/master.yml | 81 ++++++++++++++++++++++++++++++ .github/workflows/pull_request.yml | 77 ++++++++++++++++++++++++++++ tests/ci/ci_config.py | 1 + 3 files changed, 159 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5816a58081d..73cffc1d5d3 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -360,6 +360,52 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinGCC: + needs: [DockerHubPush] + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_gcc + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'true' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebAsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -918,6 +964,7 @@ jobs: - BuilderDebRelease - BuilderDebAarch64 - BuilderBinRelease + - BuilderBinGCC - BuilderDebAsan - BuilderDebTsan - BuilderDebUBsan @@ -2608,6 +2655,40 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" + UnitTestsReleaseGCC: + needs: [BuilderBinGCC] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-gcc, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" UnitTestsTsan: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index d50a2151f2f..87a31b9683c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -370,6 +370,48 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinGCC: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_gcc + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'true' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" BuilderDebAarch64: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -963,6 +1005,7 @@ jobs: - BuilderDebRelease - BuilderDebAarch64 - BuilderBinRelease + - BuilderBinGCC - BuilderDebAsan - BuilderDebTsan - BuilderDebUBsan @@ -2808,6 +2851,40 @@ jobs: docker kill "$(docker ps -q)" ||: docker rm -f "$(docker ps -a -q)" ||: sudo rm -fr "$TEMP_PATH" + UnitTestsReleaseGCC: + needs: [BuilderBinGCC] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-gcc, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill "$(docker ps -q)" ||: + docker rm -f "$(docker ps -a -q)" ||: + sudo rm -fr "$TEMP_PATH" UnitTestsTsan: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5c63b3f1ad1..7a183fd0bd6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -206,6 +206,7 @@ CI_CONFIG = { "binary_freebsd", "binary_darwin_aarch64", "binary_ppc64le", + "binary_gcc", ], }, "tests_config": { From 6780defb42014348eab758f2fb4cfffa93564584 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 26 Mar 2022 22:45:45 +0100 Subject: [PATCH 071/103] Fix build report check --- tests/ci/build_report_check.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 5afe2991073..5946c00cc8a 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -148,6 +148,13 @@ if __name__ == "__main__": build_name, ) + some_builds_are_missing = len(build_reports_map) < len(reports_order) + + if some_builds_are_missing: + logging.info("Expected to get %s build results, got %s", len(reports_order), len(build_reports_map)) + else: + logging.info("Got exactly %s builds", len(build_reports_map)) + build_reports = [ build_reports_map[build_name] for build_name in reports_order @@ -219,10 +226,10 @@ if __name__ == "__main__": if build_result.status == "success": ok_builds += 1 - if ok_builds == 0: + if ok_builds == 0 or some_builds_are_missing: summary_status = "error" - description = "{}/{} builds are OK".format(ok_builds, total_builds) + description = f"{ok_builds}/{total_builds} builds are OK" print("::notice ::Report url: {}".format(url)) From 6151e9b0599a1c188fe4d69e528ec0eb5d83ed27 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 27 Mar 2022 11:15:43 +0300 Subject: [PATCH 072/103] Avoid using std::result_of (deprecated in c++ > 17) Fixes: #34906 Signed-off-by: Azat Khuzhin --- contrib/boost-cmake/CMakeLists.txt | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/boost-cmake/CMakeLists.txt b/contrib/boost-cmake/CMakeLists.txt index 0215c68e683..3d66bc97971 100644 --- a/contrib/boost-cmake/CMakeLists.txt +++ b/contrib/boost-cmake/CMakeLists.txt @@ -27,7 +27,11 @@ target_include_directories (_boost_headers_only SYSTEM BEFORE INTERFACE ${LIBRAR # asio -target_compile_definitions (_boost_headers_only INTERFACE BOOST_ASIO_STANDALONE=1) +target_compile_definitions (_boost_headers_only INTERFACE + BOOST_ASIO_STANDALONE=1 + # Avoid using of deprecated in c++ > 17 std::result_of + BOOST_ASIO_HAS_STD_INVOKE_RESULT=1 +) # iostreams From 13f2b2c0a30576f2a0e135f4126eb52b4b6cd970 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 27 Mar 2022 11:32:51 +0300 Subject: [PATCH 073/103] Use proper type for RadixSortTraits in RowRefs.cpp Signed-off-by: Azat Khuzhin --- src/Interpreters/RowRefs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/RowRefs.cpp b/src/Interpreters/RowRefs.cpp index 2b791f5a189..97feed54c08 100644 --- a/src/Interpreters/RowRefs.cpp +++ b/src/Interpreters/RowRefs.cpp @@ -192,7 +192,7 @@ private: using Result = Element; static TKey & extractKey(Element & elem) { return elem.value; } - static Element extractResult(Element & elem) { return elem; } + static Result extractResult(Element & elem) { return elem; } }; if constexpr (is_descending) From 53eb49a2e2ea90dc3bf436130308dea8b9563980 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 27 Mar 2022 14:09:45 +0300 Subject: [PATCH 074/103] avro: create symlink in binary dir (over source dir) Signed-off-by: Azat Khuzhin --- contrib/avro-cmake/CMakeLists.txt | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/contrib/avro-cmake/CMakeLists.txt b/contrib/avro-cmake/CMakeLists.txt index d91ce40dd54..c5bda41782d 100644 --- a/contrib/avro-cmake/CMakeLists.txt +++ b/contrib/avro-cmake/CMakeLists.txt @@ -69,9 +69,10 @@ endif () target_compile_options(_avrocpp PRIVATE ${SUPPRESS_WARNINGS}) # create a symlink to include headers with +set(AVRO_INCLUDE_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") ADD_CUSTOM_TARGET(avro_symlink_headers ALL - COMMAND ${CMAKE_COMMAND} -E make_directory "${AVROCPP_ROOT_DIR}/include" - COMMAND ${CMAKE_COMMAND} -E create_symlink "${AVROCPP_ROOT_DIR}/api" "${AVROCPP_ROOT_DIR}/include/avro" + COMMAND ${CMAKE_COMMAND} -E make_directory "${AVRO_INCLUDE_DIR}" + COMMAND ${CMAKE_COMMAND} -E create_symlink "${AVROCPP_ROOT_DIR}/api" "${AVRO_INCLUDE_DIR}/avro" ) add_dependencies(_avrocpp avro_symlink_headers) -target_include_directories(_avrocpp SYSTEM BEFORE PUBLIC "${AVROCPP_ROOT_DIR}/include") +target_include_directories(_avrocpp SYSTEM BEFORE PUBLIC "${AVRO_INCLUDE_DIR}") From 4ddfff3084374afe9d186b1354fa494aef899f77 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Mar 2022 00:43:43 +0000 Subject: [PATCH 075/103] fix usage of quota with async inserts --- src/Interpreters/executeQuery.cpp | 15 +++++---- .../02246_async_insert_quota.reference | 2 ++ .../0_stateless/02246_async_insert_quota.sh | 32 +++++++++++++++++++ 3 files changed, 42 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02246_async_insert_quota.reference create mode 100755 tests/queries/0_stateless/02246_async_insert_quota.sh diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index c1606700540..caf2083391d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -607,6 +607,14 @@ static std::tuple executeQueryImpl( if (async_insert) { + quota = context->getQuota(); + if (quota) + { + quota->used(QuotaType::QUERY_INSERTS, 1); + quota->used(QuotaType::QUERIES, 1); + quota->checkExceeded(QuotaType::ERRORS); + } + queue->push(ast, context); if (settings.wait_for_async_insert) @@ -617,13 +625,6 @@ static std::tuple executeQueryImpl( res.pipeline = QueryPipeline(Pipe(std::move(source))); } - quota = context->getQuota(); - if (quota) - { - quota->used(QuotaType::QUERY_INSERTS, 1); - quota->used(QuotaType::QUERIES, 1); - } - const auto & table_id = insert_query->table_id; if (!table_id.empty()) context->setInsertionTable(table_id); diff --git a/tests/queries/0_stateless/02246_async_insert_quota.reference b/tests/queries/0_stateless/02246_async_insert_quota.reference new file mode 100644 index 00000000000..99b8e471635 --- /dev/null +++ b/tests/queries/0_stateless/02246_async_insert_quota.reference @@ -0,0 +1,2 @@ +QUOTA_EXPIRED +2 diff --git a/tests/queries/0_stateless/02246_async_insert_quota.sh b/tests/queries/0_stateless/02246_async_insert_quota.sh new file mode 100755 index 00000000000..9fc4df4660c --- /dev/null +++ b/tests/queries/0_stateless/02246_async_insert_quota.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts_02246" +${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02246" +${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02246" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02246" + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts_02246(a UInt32, s String) ENGINE = Memory" + +${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02246" +${CLICKHOUSE_CLIENT} -q "CREATE USER u02246" +${CLICKHOUSE_CLIENT} -q "GRANT INSERT ON async_inserts_02246 TO r02246" +${CLICKHOUSE_CLIENT} -q "GRANT r02246 to u02246" +${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02246 FOR INTERVAL 1 HOUR MAX QUERY INSERTS = 2 TO r02246" + +${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (1, 'a')" +${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (2, 'b')" +${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (3, 'c')" 2>&1 | grep -m1 -o QUOTA_EXPIRED + +sleep 1.0 + +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM async_inserts_02246" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts_02246" +${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02246" +${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02246" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02246" From c4832f7633ac7d47369bd274e6251d3675790193 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 28 Mar 2022 09:28:02 +0200 Subject: [PATCH 076/103] Fix --- .../MaterializedPostgreSQLConsumer.cpp | 40 +++++++++++++++---- .../MaterializedPostgreSQLConsumer.h | 2 + .../PostgreSQLReplicationHandler.cpp | 16 ++++---- .../PostgreSQL/PostgreSQLReplicationHandler.h | 3 +- 4 files changed, 43 insertions(+), 18 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 5b963a544c8..e3aa4ff82a5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -98,8 +98,24 @@ MaterializedPostgreSQLConsumer::StorageData::Buffer::Buffer( } +void MaterializedPostgreSQLConsumer::assertCorrectInsertion(StorageData::Buffer & buffer, size_t column_idx) +{ + if (column_idx >= buffer.description.sample_block.columns() + || column_idx >= buffer.description.types.size() + || column_idx >= buffer.columns.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to insert into buffer at position: {}, but block columns size is {}, types size: {}, columns size: {}, buffer structure: {}", + column_idx, + buffer.description.sample_block.columns(), buffer.description.types.size(), buffer.columns.size(), + buffer.description.sample_block.dumpStructure()); +} + + void MaterializedPostgreSQLConsumer::insertValue(StorageData::Buffer & buffer, const std::string & value, size_t column_idx) { + assertCorrectInsertion(buffer, column_idx); + const auto & sample = buffer.description.sample_block.getByPosition(column_idx); bool is_nullable = buffer.description.types[column_idx].second; @@ -134,6 +150,8 @@ void MaterializedPostgreSQLConsumer::insertValue(StorageData::Buffer & buffer, c void MaterializedPostgreSQLConsumer::insertDefaultValue(StorageData::Buffer & buffer, size_t column_idx) { + assertCorrectInsertion(buffer, column_idx); + const auto & sample = buffer.description.sample_block.getByPosition(column_idx); insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column); } @@ -515,13 +533,14 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl void MaterializedPostgreSQLConsumer::syncTables() { - try + for (const auto & table_name : tables_to_sync) { - for (const auto & table_name : tables_to_sync) - { - auto & storage_data = storages.find(table_name)->second; - Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); + auto & storage_data = storages.find(table_name)->second; + Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); + storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); + try + { if (result_rows.rows()) { auto storage = storage_data.storage; @@ -543,13 +562,18 @@ void MaterializedPostgreSQLConsumer::syncTables() CompletedPipelineExecutor executor(io.pipeline); executor.execute(); - - storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); } } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } - LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); + LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); + try + { auto tx = std::make_shared(connection->getRef()); current_lsn = advanceLSN(tx); tables_to_sync.clear(); diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index a01f9394190..5193feee708 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -122,6 +122,8 @@ private: void markTableAsSkipped(Int32 relation_id, const String & relation_name); + static void assertCorrectInsertion(StorageData::Buffer & buffer, size_t column_idx); + /// lsn - log sequnce nuumber, like wal offset (64 bit). static Int64 getLSNValue(const std::string & lsn) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 876ba9b1698..fa90295bcd6 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -64,8 +64,8 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( bool is_attach_, const MaterializedPostgreSQLSettings & replication_settings, bool is_materialized_postgresql_database_) - : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) - , context(context_) + : WithContext(context_->getGlobalContext()) + , log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , is_attach(is_attach_) , postgres_database(postgres_database_) , postgres_schema(replication_settings.materialized_postgresql_schema) @@ -94,9 +94,9 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } publication_name = fmt::format("{}_ch_publication", replication_identifier); - startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); - consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); - cleanup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ cleanupFunc(); }); + startup_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); + consumer_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); + cleanup_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ cleanupFunc(); }); } @@ -296,7 +296,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) /// (Apart from the case, when shutdownFinal is called). /// Handler uses it only for loadFromSnapshot and shutdown methods. consumer = std::make_shared( - context, + getContext(), std::move(tmp_connection), replication_slot, publication_name, @@ -921,9 +921,9 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras (); - auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + auto materialized_table_lock = materialized_storage->lockForShare(String(), getContext()->getSettingsRef().lock_acquire_timeout); /// If for some reason this temporary table already exists - also drop it. auto temp_materialized_storage = materialized_storage->createTemporary(); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 263095ec9c2..16e531f5247 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -13,7 +13,7 @@ namespace DB class StorageMaterializedPostgreSQL; struct SettingChange; -class PostgreSQLReplicationHandler +class PostgreSQLReplicationHandler : WithContext { friend class TemporaryReplicationSlot; @@ -98,7 +98,6 @@ private: std::pair getSchemaAndTableName(const String & table_name) const; Poco::Logger * log; - ContextPtr context; /// If it is not attach, i.e. a create query, then if publication already exists - always drop it. bool is_attach; From 98ad78f0a9b737123ca1a1bbcfd3f6513492538a Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 28 Mar 2022 09:41:56 +0200 Subject: [PATCH 077/103] Ping CI --- .../1_stateful/02226_s3_with_cache.reference | 2 + .../1_stateful/02226_s3_with_cache.sql | 42 +++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 tests/queries/1_stateful/02226_s3_with_cache.reference create mode 100644 tests/queries/1_stateful/02226_s3_with_cache.sql diff --git a/tests/queries/1_stateful/02226_s3_with_cache.reference b/tests/queries/1_stateful/02226_s3_with_cache.reference new file mode 100644 index 00000000000..c5149d11cab --- /dev/null +++ b/tests/queries/1_stateful/02226_s3_with_cache.reference @@ -0,0 +1,2 @@ +SELECT 1, * FROM test LIMIT 10 FORMAT Null; 2097152 2097152 0 +SELECT 2, * FROM test LIMIT 10 FORMAT Null; 0 2097152 0 diff --git a/tests/queries/1_stateful/02226_s3_with_cache.sql b/tests/queries/1_stateful/02226_s3_with_cache.sql new file mode 100644 index 00000000000..e30088f8dba --- /dev/null +++ b/tests/queries/1_stateful/02226_s3_with_cache.sql @@ -0,0 +1,42 @@ +# Tags: no-parallel, no-fasttest + +-- CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; +-- INSERT INTO test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 100000000; + +SET max_memory_usage='15G'; + +SELECT 1, * FROM test LIMIT 10 FORMAT Null; + +SYSTEM FLUSH LOGS; +SELECT query, + ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read, + ProfileEvents['RemoteFSCacheReadBytes'] > 0 as remote_fs_cache_read, + ProfileEvents['RemoteFSCacheDownloadedBytes'] > 0 as remote_fs_read_and_download +FROM system.query_log +WHERE query LIKE 'SELECT 1, * FROM test LIMIT%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC +LIMIT 1; + +SELECT * FROM test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; +SELECT * FROM test ORDER BY value LIMIT 10 FORMAT Null; +SELECT * FROM test WHERE value LIKE '%dba%' ORDER BY value LIMIT 10 FORMAT Null; + +SET remote_filesystem_read_method='read'; + +SELECT 2, * FROM test LIMIT 10 FORMAT Null; + +SYSTEM FLUSH LOGS; +SELECT query, + ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read, + ProfileEvents['RemoteFSCacheReadBytes'] > 0 as remote_fs_cache_read, + ProfileEvents['RemoteFSCacheDownloadedBytes'] > 0 as remote_fs_read_and_download +FROM system.query_log +WHERE query LIKE 'SELECT 2, * FROM test LIMIT%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC +LIMIT 1; + +-- DROP TABLE test NO DELAY; From 08db628cc0269501d222d9c89bfa9b96213ca261 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 11:18:11 +0200 Subject: [PATCH 078/103] Update packages/clickhouse-keeper.yaml Co-authored-by: Mikhail f. Shiryaev --- packages/clickhouse-keeper.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/packages/clickhouse-keeper.yaml b/packages/clickhouse-keeper.yaml index e717ba79c5b..e99ac30f944 100644 --- a/packages/clickhouse-keeper.yaml +++ b/packages/clickhouse-keeper.yaml @@ -14,6 +14,8 @@ conflicts: - clickhouse-server depends: - adduser +suggests: +- clickhouse-keeper-dbg maintainer: "ClickHouse Dev Team " description: | From 37e1160c678bfacf220880aaf560b71a49d4d7fa Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 11:18:16 +0200 Subject: [PATCH 079/103] Update packages/clickhouse-keeper-dbg.yaml Co-authored-by: Mikhail f. Shiryaev --- packages/clickhouse-keeper-dbg.yaml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/packages/clickhouse-keeper-dbg.yaml b/packages/clickhouse-keeper-dbg.yaml index 685bff11080..2c70b7ad4aa 100644 --- a/packages/clickhouse-keeper-dbg.yaml +++ b/packages/clickhouse-keeper-dbg.yaml @@ -9,10 +9,6 @@ homepage: "https://clickhouse.com" license: "Apache" section: "database" priority: "optional" - -conflicts: -- clickhouse-server - maintainer: "ClickHouse Dev Team " description: | debugging symbols for clickhouse-keeper From 77e700b1cfa293ad95f06e3ab4a3e2da040a616c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 11:25:08 +0200 Subject: [PATCH 080/103] Update src/Compression/CompressionFactory.cpp --- src/Compression/CompressionFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 8dfc894e15b..abf5e38a8c3 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -168,7 +168,7 @@ void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); -/// Keeper use only general-purpose codes, so we don't need these special codecs +/// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build #ifndef KEEPER_STANDALONE_BUILD From 47517970cef7cc23dea58e5051d99196d27fe169 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 11:56:37 +0200 Subject: [PATCH 081/103] Bump CI From c7fc1bd4dd386bcc89c9b1077e49b6f5e9f25f2f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 12:16:41 +0200 Subject: [PATCH 082/103] Fix flaky 01307_multiple_leaders --- tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh index 2a41d90cd3a..db986e74b6b 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh @@ -20,7 +20,8 @@ function thread() REPLICA=$1 ITERATIONS=$2 - $CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 0 --min_insert_block_size_bytes 0 --query "INSERT INTO r$REPLICA SELECT number * $NUM_REPLICAS + $REPLICA FROM numbers($ITERATIONS)" + # It's legal to fetch something before insert finished + $CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 0 --min_insert_block_size_bytes 0 --query "INSERT INTO r$REPLICA SELECT number * $NUM_REPLICAS + $REPLICA FROM numbers($ITERATIONS)" 2>&1 | grep -v -F "Tried to commit obsolete part" } for REPLICA in $SEQ; do From dbeba322fdb77bf81e7e85bbf3e42ff50348d6d2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Mar 2022 12:51:26 +0200 Subject: [PATCH 083/103] Delete 02226_s3_with_cache.sql --- .../1_stateful/02226_s3_with_cache.sql | 42 ------------------- 1 file changed, 42 deletions(-) delete mode 100644 tests/queries/1_stateful/02226_s3_with_cache.sql diff --git a/tests/queries/1_stateful/02226_s3_with_cache.sql b/tests/queries/1_stateful/02226_s3_with_cache.sql deleted file mode 100644 index e30088f8dba..00000000000 --- a/tests/queries/1_stateful/02226_s3_with_cache.sql +++ /dev/null @@ -1,42 +0,0 @@ -# Tags: no-parallel, no-fasttest - --- CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; --- INSERT INTO test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 100000000; - -SET max_memory_usage='15G'; - -SELECT 1, * FROM test LIMIT 10 FORMAT Null; - -SYSTEM FLUSH LOGS; -SELECT query, - ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read, - ProfileEvents['RemoteFSCacheReadBytes'] > 0 as remote_fs_cache_read, - ProfileEvents['RemoteFSCacheDownloadedBytes'] > 0 as remote_fs_read_and_download -FROM system.query_log -WHERE query LIKE 'SELECT 1, * FROM test LIMIT%' -AND type = 'QueryFinish' -AND current_database = currentDatabase() -ORDER BY query_start_time DESC -LIMIT 1; - -SELECT * FROM test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; -SELECT * FROM test ORDER BY value LIMIT 10 FORMAT Null; -SELECT * FROM test WHERE value LIKE '%dba%' ORDER BY value LIMIT 10 FORMAT Null; - -SET remote_filesystem_read_method='read'; - -SELECT 2, * FROM test LIMIT 10 FORMAT Null; - -SYSTEM FLUSH LOGS; -SELECT query, - ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read, - ProfileEvents['RemoteFSCacheReadBytes'] > 0 as remote_fs_cache_read, - ProfileEvents['RemoteFSCacheDownloadedBytes'] > 0 as remote_fs_read_and_download -FROM system.query_log -WHERE query LIKE 'SELECT 2, * FROM test LIMIT%' -AND type = 'QueryFinish' -AND current_database = currentDatabase() -ORDER BY query_start_time DESC -LIMIT 1; - --- DROP TABLE test NO DELAY; From 3887fefb6b3ec6576f5dd2bcb9252836c43478ab Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Mar 2022 12:51:35 +0200 Subject: [PATCH 084/103] Delete 02226_s3_with_cache.reference --- tests/queries/1_stateful/02226_s3_with_cache.reference | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 tests/queries/1_stateful/02226_s3_with_cache.reference diff --git a/tests/queries/1_stateful/02226_s3_with_cache.reference b/tests/queries/1_stateful/02226_s3_with_cache.reference deleted file mode 100644 index c5149d11cab..00000000000 --- a/tests/queries/1_stateful/02226_s3_with_cache.reference +++ /dev/null @@ -1,2 +0,0 @@ -SELECT 1, * FROM test LIMIT 10 FORMAT Null; 2097152 2097152 0 -SELECT 2, * FROM test LIMIT 10 FORMAT Null; 0 2097152 0 From 1122db89dbdc835d964841587aabf5fd1c4369d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Mar 2022 11:26:42 +0000 Subject: [PATCH 085/103] Use float devision for avg after optimize_fuse_sum_count_avg --- src/Interpreters/TreeRewriter.cpp | 5 ++++- tests/queries/0_stateless/02244_issue_35598_fuse.reference | 2 ++ tests/queries/0_stateless/02244_issue_35598_fuse.sql | 5 +++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02244_issue_35598_fuse.reference create mode 100644 tests/queries/0_stateless/02244_issue_35598_fuse.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 78e7ed33f8f..929e516f687 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -345,7 +345,10 @@ void replaceWithSumCount(String column_name, ASTFunction & func) { /// Rewrite "avg" to sumCount().1 / sumCount().2 auto new_arg1 = makeASTFunction("tupleElement", func_base, std::make_shared(UInt8(1))); - auto new_arg2 = makeASTFunction("tupleElement", func_base, std::make_shared(UInt8(2))); + auto new_arg2 = makeASTFunction("CAST", + makeASTFunction("tupleElement", func_base, std::make_shared(UInt8(2))), + std::make_shared("Float64")); + func.name = "divide"; exp_list->children.push_back(new_arg1); exp_list->children.push_back(new_arg2); diff --git a/tests/queries/0_stateless/02244_issue_35598_fuse.reference b/tests/queries/0_stateless/02244_issue_35598_fuse.reference new file mode 100644 index 00000000000..6ce84b402a3 --- /dev/null +++ b/tests/queries/0_stateless/02244_issue_35598_fuse.reference @@ -0,0 +1,2 @@ +0 0 nan +0 0 nan diff --git a/tests/queries/0_stateless/02244_issue_35598_fuse.sql b/tests/queries/0_stateless/02244_issue_35598_fuse.sql new file mode 100644 index 00000000000..a590854eb6c --- /dev/null +++ b/tests/queries/0_stateless/02244_issue_35598_fuse.sql @@ -0,0 +1,5 @@ +SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0)) +SETTINGS optimize_syntax_fuse_functions = 0, optimize_fuse_sum_count_avg = 0; + +SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0)) +SETTINGS optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1; From 13831120da7a266444d3405e714c5a75876ac784 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 14:32:53 +0200 Subject: [PATCH 086/103] Fix restart repliacs test --- ...estart_replicas_rename_deadlock_zookeeper.sh | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh index abd5c0d6a4f..7b065e87f8a 100755 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh @@ -38,27 +38,18 @@ function restart_replicas_loop() done sleep 0.$RANDOM } -function restart_thread_1() -{ - restart_replicas_loop -} - -function restart_thread_2() -{ - restart_replicas_loop -} export -f rename_thread_1 export -f rename_thread_2 -export -f restart_thread_1 -export -f restart_thread_2 +export -f restart_replicas_loop +export -f restart_replicas_loop TIMEOUT=10 clickhouse_client_loop_timeout $TIMEOUT rename_thread_1 2> /dev/null & clickhouse_client_loop_timeout $TIMEOUT rename_thread_2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT restart_thread_1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT restart_thread_2 2> /dev/null & +clickhouse_client_loop_timeout $TIMEOUT restart_replicas_loop 2> /dev/null & +clickhouse_client_loop_timeout $TIMEOUT restart_replicas_loop 2> /dev/null & wait From 8acb5de5b5ef3512ad1e0bc1fd2ccd663380c5d9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 14:43:15 +0200 Subject: [PATCH 087/103] Fix build report black --- tests/ci/build_report_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 5946c00cc8a..592e905bcb5 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -151,7 +151,11 @@ if __name__ == "__main__": some_builds_are_missing = len(build_reports_map) < len(reports_order) if some_builds_are_missing: - logging.info("Expected to get %s build results, got %s", len(reports_order), len(build_reports_map)) + logging.info( + "Expected to get %s build results, got %s", + len(reports_order), + len(build_reports_map), + ) else: logging.info("Got exactly %s builds", len(build_reports_map)) From 51ef88e5dd01f1e81082b0ce149504a104e09c6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 15:29:13 +0200 Subject: [PATCH 088/103] Fix 00385 --- .../00385_storage_file_and_clickhouse-local_app_long.sh | 4 ++-- tests/queries/0_stateless/01268_procfs_metrics.sh | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.sh b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.sh index d77955a51bc..a78cdd445cb 100755 --- a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.sh +++ b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app_long.sh @@ -19,11 +19,11 @@ function pack_unpack_compare() ${CLICKHOUSE_CLIENT} --query "CREATE TABLE buf_00385 ENGINE = Memory AS $1" local res_orig - res_orig=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM buf_00385") + res_orig=$(${CLICKHOUSE_CLIENT} --max_block_size=65505 --max_threads=1 --query "SELECT $TABLE_HASH FROM buf_00385") ${CLICKHOUSE_CLIENT} --max_threads=1 --query "CREATE TABLE buf_file ENGINE = File($3) AS SELECT * FROM buf_00385" local res_db_file - res_db_file=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM buf_file") + res_db_file=$(${CLICKHOUSE_CLIENT} --max_block_size=65505 --max_threads=1 --query "SELECT $TABLE_HASH FROM buf_file") ${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT * FROM buf_00385 FORMAT $3" > "$buf_file" local res_ch_local1 diff --git a/tests/queries/0_stateless/01268_procfs_metrics.sh b/tests/queries/0_stateless/01268_procfs_metrics.sh index d5bd99724ca..acf282ffd67 100755 --- a/tests/queries/0_stateless/01268_procfs_metrics.sh +++ b/tests/queries/0_stateless/01268_procfs_metrics.sh @@ -41,4 +41,7 @@ timeout $TIMEOUT bash -c show_processes_func & wait +# otherwise it can be alive after test +$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='SELECT * FROM numbers(600000000) FORMAT Null SETTINGS max_threads = 1'" 2> /dev/null 1> /dev/null + echo "Test OK" From e75a0a594d4516c23f5348c1e99b93a94f4a61fd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 15:39:02 +0200 Subject: [PATCH 089/103] Better fix for procfs_metrics --- tests/queries/0_stateless/01268_procfs_metrics.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01268_procfs_metrics.sh b/tests/queries/0_stateless/01268_procfs_metrics.sh index acf282ffd67..c1697edd632 100755 --- a/tests/queries/0_stateless/01268_procfs_metrics.sh +++ b/tests/queries/0_stateless/01268_procfs_metrics.sh @@ -42,6 +42,12 @@ timeout $TIMEOUT bash -c show_processes_func & wait # otherwise it can be alive after test -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='SELECT * FROM numbers(600000000) FORMAT Null SETTINGS max_threads = 1'" 2> /dev/null 1> /dev/null +query_alive=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.processes WHERE query ILIKE 'SELECT * FROM numbers(600000000)%'") +while [[ $query_alive != 0 ]] +do + $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query ilike '%SELECT * FROM numbers(600000000)%'" 2> /dev/null 1> /dev/null + sleep 0.5 + query_alive=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.processes WHERE query ILIKE 'SELECT * FROM numbers(600000000)%'") +done echo "Test OK" From b838a7dcb0300877c121654ceb6d6b2c8f53c567 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 15:53:22 +0200 Subject: [PATCH 090/103] Remove outdated links from CI --- CMakeLists.txt | 2 +- base/glibc-compatibility/CMakeLists.txt | 2 +- cmake/version.cmake | 2 +- docker/packager/packager | 2 +- docker/test/fuzzer/run-fuzzer.sh | 2 +- docker/test/keeper-jepsen/run.sh | 4 ++-- docker/test/performance-comparison/compare.sh | 6 +++--- .../test/performance-comparison/download.sh | 19 +++++-------------- .../test/performance-comparison/entrypoint.sh | 6 ++---- docker/test/stateful/Dockerfile | 2 +- docker/test/stateful/s3downloader | 2 +- docker/test/stress/Dockerfile | 2 +- 12 files changed, 20 insertions(+), 31 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index deef582c790..5157f0f9903 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -261,7 +261,7 @@ endif () # Add a section with the hash of the compiled machine code for integrity checks. # Only for official builds, because adding a section can be time consuming (rewrite of several GB). # And cross compiled binaries are not supported (since you cannot execute clickhouse hash-binary) -if (OBJCOPY_PATH AND YANDEX_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE)) +if (OBJCOPY_PATH AND CLICKHOUSE_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE)) set (USE_BINARY_HASH 1) endif () diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index ddec09121e1..ef7ec6d7fc0 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -51,6 +51,6 @@ if (GLIBC_COMPATIBILITY) message (STATUS "Some symbols from glibc will be replaced for compatibility") -elseif (YANDEX_OFFICIAL_BUILD) +elseif (CLICKHOUSE_OFFICIAL_BUILD) message (WARNING "Option GLIBC_COMPATIBILITY must be turned on for production builds.") endif () diff --git a/cmake/version.cmake b/cmake/version.cmake index 963f291c0f3..acaa772ff2f 100644 --- a/cmake/version.cmake +++ b/cmake/version.cmake @@ -18,6 +18,6 @@ set (VERSION_STRING_SHORT "${VERSION_MAJOR}.${VERSION_MINOR}") math (EXPR VERSION_INTEGER "${VERSION_PATCH} + ${VERSION_MINOR}*1000 + ${VERSION_MAJOR}*1000000") -if(YANDEX_OFFICIAL_BUILD) +if(CLICKHOUSE_OFFICIAL_BUILD) set(VERSION_OFFICIAL " (official build)") endif() diff --git a/docker/packager/packager b/docker/packager/packager index a5763273f5f..c05e39b1d56 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -244,7 +244,7 @@ def parse_env_variables( result.append(f"AUTHOR='{author}'") if official: - cmake_flags.append("-DYANDEX_OFFICIAL_BUILD=1") + cmake_flags.append("-DCLICKHOUSE_OFFICIAL_BUILD=1") result.append('CMAKE_FLAGS="' + " ".join(cmake_flags) + '"') diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index e18c07bf2c1..74711f476f8 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -13,7 +13,7 @@ script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" echo "$script_dir" repo_dir=ch BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-13_debug_none_bundled_unsplitted_disable_False_binary"} -BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} +BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} function clone { diff --git a/docker/test/keeper-jepsen/run.sh b/docker/test/keeper-jepsen/run.sh index d7534270e2c..4dec82234bc 100644 --- a/docker/test/keeper-jepsen/run.sh +++ b/docker/test/keeper-jepsen/run.sh @@ -2,7 +2,7 @@ set -euo pipefail -CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-13_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"} +CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-13_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"} CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""} @@ -10,7 +10,7 @@ if [ -z "$CLICKHOUSE_REPO_PATH" ]; then CLICKHOUSE_REPO_PATH=ch rm -rf ch ||: mkdir ch ||: - wget -nv -nd -c "https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/repo/clickhouse_no_subs.tar.gz" + wget -nv -nd -c "https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/repo/clickhouse_no_subs.tar.gz" tar -C ch --strip-components=1 -xf clickhouse_no_subs.tar.gz ls -lath ||: fi diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 16ac304d7fb..54f71ce05bb 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1294,15 +1294,15 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') select '' test_name, '$(sed -n 's/.*/\1/p' report.html)' test_status, 0 test_duration_ms, - 'https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#fail1' report_url + 'https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#fail1' report_url union all select test || ' #' || toString(query_index), 'slower' test_status, 0 test_duration_ms, - 'https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#changes-in-performance.' + 'https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#changes-in-performance.' || test || '.' || toString(query_index) report_url from queries where changed_fail != 0 and diff > 0 union all select test || ' #' || toString(query_index), 'unstable' test_status, 0 test_duration_ms, - 'https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#unstable-queries.' + 'https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#unstable-queries.' || test || '.' || toString(query_index) report_url from queries where unstable_fail != 0 ) diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index 8fa6eb5ec83..ae9e677713f 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -16,26 +16,17 @@ right_sha=$4 datasets=${CHPC_DATASETS-"hits1 hits10 hits100 values"} declare -A dataset_paths -if [[ $S3_URL == *"s3.amazonaws.com"* ]]; then - dataset_paths["hits10"]="https://clickhouse-private-datasets.s3.amazonaws.com/hits_10m_single/partitions/hits_10m_single.tar" - dataset_paths["hits100"]="https://clickhouse-private-datasets.s3.amazonaws.com/hits_100m_single/partitions/hits_100m_single.tar" - dataset_paths["hits1"]="https://clickhouse-datasets.s3.amazonaws.com/hits/partitions/hits_v1.tar" - dataset_paths["values"]="https://clickhouse-datasets.s3.amazonaws.com/values_with_expressions/partitions/test_values.tar" -else - dataset_paths["hits10"]="https://s3.mds.yandex.net/clickhouse-private-datasets/hits_10m_single/partitions/hits_10m_single.tar" - dataset_paths["hits100"]="https://s3.mds.yandex.net/clickhouse-private-datasets/hits_100m_single/partitions/hits_100m_single.tar" - dataset_paths["hits1"]="https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_v1.tar" - dataset_paths["values"]="https://clickhouse-datasets.s3.yandex.net/values_with_expressions/partitions/test_values.tar" -fi +dataset_paths["hits10"]="https://clickhouse-private-datasets.s3.amazonaws.com/hits_10m_single/partitions/hits_10m_single.tar" +dataset_paths["hits100"]="https://clickhouse-private-datasets.s3.amazonaws.com/hits_100m_single/partitions/hits_100m_single.tar" +dataset_paths["hits1"]="https://clickhouse-datasets.s3.amazonaws.com/hits/partitions/hits_v1.tar" +dataset_paths["values"]="https://clickhouse-datasets.s3.amazonaws.com/values_with_expressions/partitions/test_values.tar" function download { # Historically there were various paths for the performance test package. # Test all of them. - declare -a urls_to_try=("https://s3.amazonaws.com/clickhouse-builds/$left_pr/$left_sha/performance/performance.tgz" - "https://clickhouse-builds.s3.yandex.net/$left_pr/$left_sha/clickhouse_build_check/performance/performance.tgz" - ) + declare -a urls_to_try=("https://s3.amazonaws.com/clickhouse-builds/$left_pr/$left_sha/performance/performance.tgz") for path in "${urls_to_try[@]}" do diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 3d37a6c0e92..767807d008b 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -4,7 +4,7 @@ set -ex CHPC_CHECK_START_TIMESTAMP="$(date +%s)" export CHPC_CHECK_START_TIMESTAMP -S3_URL=${S3_URL:="https://clickhouse-builds.s3.yandex.net"} +S3_URL=${S3_URL:="https://clickhouse-builds.s3.amazonaws.com"} COMMON_BUILD_PREFIX="/clickhouse_build_check" if [[ $S3_URL == *"s3.amazonaws.com"* ]]; then @@ -64,9 +64,7 @@ function find_reference_sha # Historically there were various path for the performance test package, # test all of them. unset found - declare -a urls_to_try=("https://s3.amazonaws.com/clickhouse-builds/0/$REF_SHA/performance/performance.tgz" - "https://clickhouse-builds.s3.yandex.net/0/$REF_SHA/clickhouse_build_check/performance/performance.tgz" - ) + declare -a urls_to_try=("https://s3.amazonaws.com/clickhouse-builds/0/$REF_SHA/performance/performance.tgz") for path in "${urls_to_try[@]}" do if curl_with_retry "$path" diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 93e7cebb857..543cf113b2b 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -11,7 +11,7 @@ RUN apt-get update -y \ COPY s3downloader /s3downloader -ENV S3_URL="https://clickhouse-datasets.s3.yandex.net" +ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com" ENV DATASETS="hits visits" ENV EXPORT_S3_STORAGE_POLICIES=1 diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader index eb3b3cd9faf..b1302877d6a 100755 --- a/docker/test/stateful/s3downloader +++ b/docker/test/stateful/s3downloader @@ -10,7 +10,7 @@ import requests import tempfile -DEFAULT_URL = 'https://clickhouse-datasets.s3.yandex.net' +DEFAULT_URL = 'https://clickhouse-datasets.s3.amazonaws.com' AVAILABLE_DATASETS = { 'hits': 'hits_v1.tar', diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 495c12f4f83..ba6daffc014 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -29,7 +29,7 @@ COPY ./download_previous_release /download_previous_release COPY run.sh / ENV DATASETS="hits visits" -ENV S3_URL="https://clickhouse-datasets.s3.yandex.net" +ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com" ENV EXPORT_S3_STORAGE_POLICIES=1 CMD ["/bin/bash", "/run.sh"] From 1f028dd48e41509617c6a97a3e43d83d50bb7d02 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 28 Mar 2022 14:13:56 +0000 Subject: [PATCH 091/103] Fix flaky tests 02149_read_in_order_fixed_prefix and 02177_issue_31009 --- .../queries/0_stateless/02149_read_in_order_fixed_prefix.sql | 2 ++ tests/queries/0_stateless/02177_issue_31009.sql | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql index 7d0e9111d9c..8fb11ac383c 100644 --- a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql +++ b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql @@ -1,3 +1,5 @@ +SET max_threads=0; + DROP TABLE IF EXISTS t_read_in_order; CREATE TABLE t_read_in_order(date Date, i UInt64, v UInt64) diff --git a/tests/queries/0_stateless/02177_issue_31009.sql b/tests/queries/0_stateless/02177_issue_31009.sql index f4a65e3a3a0..ab4aec60ce4 100644 --- a/tests/queries/0_stateless/02177_issue_31009.sql +++ b/tests/queries/0_stateless/02177_issue_31009.sql @@ -1,5 +1,10 @@ -- Tags: long +SET max_threads=0; + +DROP TABLE IF EXISTS left; +DROP TABLE IF EXISTS right; + CREATE TABLE left ( key UInt32, value String ) ENGINE = MergeTree ORDER BY key; CREATE TABLE right ( key UInt32, value String ) ENGINE = MergeTree ORDER BY tuple(); From 933eaeae942a289fc33530c846551c656a6c9b4b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 28 Mar 2022 16:26:20 +0200 Subject: [PATCH 092/103] Fix black test --- tests/integration/test_allowed_url_from_config/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 3306d26da6c..13f3929902d 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -281,4 +281,3 @@ def test_HDFS(start_cluster): def test_schema_inference(start_cluster): error = node7.query_and_get_error("desc url('http://test.com`, 'TSVRaw'')") assert(error.find('ReadWriteBufferFromHTTPBase') == -1) - From 9c3e9a2c9b30ef8739616d54bb4ba759fc0f2bf0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 17:11:20 +0200 Subject: [PATCH 093/103] Update tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh Co-authored-by: Azat Khuzhin --- .../01108_restart_replicas_rename_deadlock_zookeeper.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh index 7b065e87f8a..a51e786b058 100755 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh @@ -42,7 +42,6 @@ function restart_replicas_loop() export -f rename_thread_1 export -f rename_thread_2 export -f restart_replicas_loop -export -f restart_replicas_loop TIMEOUT=10 From 93021e387e72251aa3a26d1428eb15dcf974805f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 28 Mar 2022 18:56:47 +0200 Subject: [PATCH 094/103] Implementation of makeDate and makeDate32 --- src/Functions/makeDate.cpp | 170 ++++++++++++++++++ src/Functions/registerFunctions.cpp | 2 + .../0_stateless/02242_make_date.reference | 55 ++++++ tests/queries/0_stateless/02242_make_date.sql | 69 +++++++ .../0_stateless/02243_make_date32.reference | 63 +++++++ .../queries/0_stateless/02243_make_date32.sql | 77 ++++++++ 6 files changed, 436 insertions(+) create mode 100644 src/Functions/makeDate.cpp create mode 100644 tests/queries/0_stateless/02242_make_date.reference create mode 100644 tests/queries/0_stateless/02242_make_date.sql create mode 100644 tests/queries/0_stateless/02243_make_date32.reference create mode 100644 tests/queries/0_stateless/02243_make_date32.sql diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp new file mode 100644 index 00000000000..049f8af39d1 --- /dev/null +++ b/src/Functions/makeDate.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +// A helper function to simplify comparisons of valid YYYY-MM-DD values for <,>,= +inline constexpr Int64 YearMonthDayToSingleInt(Int64 year, Int64 month, Int64 day) +{ + return year*512 + month*32 + day; +} + +// Common implementation for makeDate, makeDate32 +template +class FunctionMakeDate : public IFunction +{ +private: + static constexpr std::array argument_names = {"year", "month", "day"}; + +public: + static constexpr auto name = Traits::name; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + + size_t getNumberOfArguments() const override { return argument_names.size(); } + + bool isInjective(const ColumnsWithTypeAndName &) const override + { + return false; // {year,month,day} that are out of supported range are converted into a default value + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForNulls() const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != argument_names.size()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires 3 arguments, but {} given", getName(), arguments.size()); + + for (size_t i = 0; i < argument_names.size(); ++i) + { + DataTypePtr argument_type = arguments[i]; + if (!isNumber(argument_type) && !isStringOrFixedString(argument_type)) + throw Exception("Argument '" + std::string(argument_names[i]) + "' for function " + getName() + " must be number", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const DataTypePtr converted_argument_type = std::make_shared(); + Columns converted_arguments; + for (size_t i = 0; i < arguments.size(); ++i) + { + const auto & argument_type = arguments[i].type; + ColumnPtr argument_column = arguments[i].column->convertToFullColumnIfConst(); + + if (converted_argument_type->equals(*argument_type)) + converted_arguments.push_back(argument_column); + else if (isNumber(argument_type)) + converted_arguments.push_back(convert_to_number_func.executeImpl( + {ColumnWithTypeAndName(argument_column, argument_type, "")}, converted_argument_type, input_rows_count)); + else if (isStringOrFixedString(argument_type)) + converted_arguments.push_back(parse_number_func.executeImpl( + {ColumnWithTypeAndName(argument_column, argument_type, "")}, converted_argument_type, input_rows_count)); + else + throw Exception("Argument '" + std::string(argument_names[i]) + "' for function " + getName() + " must be number", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + auto res_column = Traits::ReturnColumnType::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); + const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); + const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); + + const auto & date_lut = DateLUT::instance(); + + for (size_t i = 0; i < input_rows_count; ++i) { + const auto year = year_data[i]; + const auto month = month_data[i]; + const auto day = day_data[i]; + + Int32 day_num = 0; + + if (year >= Traits::MIN_YEAR && + year <= Traits::MAX_YEAR && + month <= 12 && + day <= 31 && + YearMonthDayToSingleInt(year, month, day) <= Traits::MAX_DATE) + { + day_num = date_lut.makeDayNum(year, month, day); + } + + result_data[i] = day_num; + } + + return res_column; + } + +private: + FunctionToFloat64 convert_to_number_func; + FunctionToFloat64OrZero parse_number_func; + FunctionOr or_func; +}; + +// makeDate(year, month, day) +struct MakeDateTraits +{ + static constexpr auto name = "makeDate"; + using ReturnDataType = DataTypeDate; + using ReturnColumnType = ColumnUInt16; + + static constexpr auto MIN_YEAR = 1970; + static constexpr auto MAX_YEAR = 2149; + // This date has the maximum day number that fits in 16-bit uint + static constexpr auto MAX_DATE = YearMonthDayToSingleInt(MAX_YEAR, 6, 6); +}; + +// makeDate32(year, month, day) +struct MakeDate32Traits +{ + static constexpr auto name = "makeDate32"; + using ReturnDataType = DataTypeDate32; + using ReturnColumnType = ColumnInt32; + + static constexpr auto MIN_YEAR = 1925; + static constexpr auto MAX_YEAR = 2283; + static constexpr auto MAX_DATE = YearMonthDayToSingleInt(MAX_YEAR, 11, 11); +}; + +} + +void registerFunctionsMakeDate(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index 82f72d7a1fa..6b3c6e92945 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -8,6 +8,7 @@ namespace DB void registerFunctionsArithmetic(FunctionFactory &); void registerFunctionsArray(FunctionFactory &); void registerFunctionsTuple(FunctionFactory &); +void registerFunctionsMakeDate(FunctionFactory &); void registerFunctionsMap(FunctionFactory &); void registerFunctionsBitmap(FunctionFactory &); void registerFunctionsBinaryRepr(FunctionFactory &); @@ -73,6 +74,7 @@ void registerFunctions() registerFunctionsArithmetic(factory); registerFunctionsArray(factory); registerFunctionsTuple(factory); + registerFunctionsMakeDate(factory); registerFunctionsMap(factory); registerFunctionsBitmap(factory); registerFunctionsBinaryRepr(factory); diff --git a/tests/queries/0_stateless/02242_make_date.reference b/tests/queries/0_stateless/02242_make_date.reference new file mode 100644 index 00000000000..de46684cefe --- /dev/null +++ b/tests/queries/0_stateless/02242_make_date.reference @@ -0,0 +1,55 @@ +Date +Nullable(Date) +Nullable(Date) +Nullable(Date) +Nullable(Date) +1970-01-01 +2020-08-24 +1980-10-17 +1970-01-01 +1970-01-01 +1970-01-01 +1980-10-15 +1970-01-01 +1980-10-15 +1970-01-01 +1980-10-15 +1970-01-01 +1970-01-01 +1970-01-01 +1980-02-29 +1984-03-01 +1970-01-01 +2148-01-01 +2149-01-01 +2149-06-06 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1980-10-20 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1994-01-24 +1984-03-01 +1970-01-01 +1970-01-01 +1970-01-01 +\N +\N +\N +1984-02-03 +\N +\N +\N +1984-02-03 +1984-02-04 diff --git a/tests/queries/0_stateless/02242_make_date.sql b/tests/queries/0_stateless/02242_make_date.sql new file mode 100644 index 00000000000..83c500adede --- /dev/null +++ b/tests/queries/0_stateless/02242_make_date.sql @@ -0,0 +1,69 @@ +select toTypeName(makeDate(1991, 8, 24)); +select toTypeName(makeDate(cast(1991 as Nullable(UInt64)), 8, 24)); +select toTypeName(makeDate(1991, cast(8 as Nullable(UInt64)), 24)); +select toTypeName(makeDate(1991, 8, cast(24 as Nullable(UInt64)))); +select toTypeName(makeDate(1991, cast(8 as Nullable(UInt64)), cast(24 as Nullable(UInt64)))); + +select makeDate(1970, 01, 01); +select makeDate(2020, 08, 24); +select makeDate(1980, 10, 17); +select makeDate(-1980, 10, 17); +select makeDate(1980, -10, 17); +select makeDate(1980, 10, -17); +select makeDate(1980.0, '10', 30.0/2); +select makeDate(-1980.0, '10', 30.0/2); +select makeDate(cast(1980.1 as Decimal(20,5)), '10', 30.0/2); +select makeDate(cast('-1980.1' as Decimal(20,5)), '10', 30.0/2); +select makeDate(cast(1980.1 as Float32), '10', 30.0/2); +select makeDate(cast(-1980.1 as Float32), '10', 30.0/2); +--select makeDate(cast(1980 as Date), '10', 30.0/2); +--select makeDate(cast(-1980 as Date), '10', 30.0/2); +--select makeDate(cast(1980 as Date32), '10', 30.0/2); +--select makeDate(cast(-1980 as Date32), '10', 30.0/2); +--select makeDate(cast(1980 as DateTime), '10', 30.0/2); +--select makeDate(cast(-1980 as DateTime), '10', 30.0/2); +--select makeDate(cast(1980 as DateTime64), '10', 30.0/2); +--select makeDate(cast(-1980 as DateTime64), '10', 30.0/2); + +select makeDate(0.0, 1, 2); +select makeDate(1980, 15, 1); +select makeDate(1980, 2, 29); +select makeDate(1984, 2, 30); +select makeDate(19800, 12, 3); +select makeDate(2148,1,1); +select makeDate(2149,1,1); +select makeDate(2149,6,6); +select makeDate(2149,6,7); +select makeDate(2150,1,1); +select makeDate(1969,1,1); +select makeDate(1969,12,1); +select makeDate(1969,12,31); +select makeDate(1970,0,0); +select makeDate(1970,0,1); +select makeDate(1970,1,0); +select makeDate(1990,0,1); +select makeDate(1990,1,0); + +select makeDate('1980', '10', '20'); +select makeDate('-1980', 3, 17); + +select makeDate('aa', 3, 24); +select makeDate(1994, 'aa', 24); +select makeDate(1984, 3, 'aa'); + +select makeDate(True, 3, 24); +select makeDate(1994, True, 24); +select makeDate(1984, 3, True); +select makeDate(False, 3, 24); +select makeDate(1994, False, 24); +select makeDate(1984, 3, False); + +select makeDate(NULL, 3, 4); +select makeDate(1980, NULL, 4); +select makeDate(1980, 3, NULL); + +select makeDate(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select 1984 as year, 2 as month, 3 as day) order by year, month, day; + +select makeDate(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select NULL as year, 2 as month, 3 as day) order by year, month, day; + +select makeDate(year, month, day) from (select 1984 as year, 2 as month, 3 as day union all select 1984 as year, 2 as month, 4 as day) order by year, month, day; diff --git a/tests/queries/0_stateless/02243_make_date32.reference b/tests/queries/0_stateless/02243_make_date32.reference new file mode 100644 index 00000000000..545137b8d4e --- /dev/null +++ b/tests/queries/0_stateless/02243_make_date32.reference @@ -0,0 +1,63 @@ +Date32 +Nullable(Date32) +Nullable(Date32) +Nullable(Date32) +Nullable(Date32) +1970-01-01 +2020-08-24 +1980-10-17 +1970-01-01 +1970-01-01 +1970-01-01 +1980-10-15 +1970-01-01 +1980-10-15 +1970-01-01 +1980-10-15 +1970-01-01 +1970-01-01 +1970-01-01 +1980-02-29 +1984-03-01 +1970-01-01 +2148-01-01 +2149-01-01 +2149-06-06 +2149-06-07 +2150-01-01 +1969-01-01 +1969-12-01 +1969-12-31 +2282-01-01 +2283-01-01 +2283-11-11 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1980-10-20 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1994-01-24 +1984-03-01 +1970-01-01 +1970-01-01 +1970-01-01 +\N +\N +\N +1984-02-03 +\N +\N +\N +1984-02-03 +1984-02-04 diff --git a/tests/queries/0_stateless/02243_make_date32.sql b/tests/queries/0_stateless/02243_make_date32.sql new file mode 100644 index 00000000000..36cb73a85b4 --- /dev/null +++ b/tests/queries/0_stateless/02243_make_date32.sql @@ -0,0 +1,77 @@ +select toTypeName(makeDate32(1991, 8, 24)); +select toTypeName(makeDate32(cast(1991 as Nullable(UInt64)), 8, 24)); +select toTypeName(makeDate32(1991, cast(8 as Nullable(UInt64)), 24)); +select toTypeName(makeDate32(1991, 8, cast(24 as Nullable(UInt64)))); +select toTypeName(makeDate32(1991, cast(8 as Nullable(UInt64)), cast(24 as Nullable(UInt64)))); + +select makeDate32(1970, 01, 01); +select makeDate32(2020, 08, 24); +select makeDate32(1980, 10, 17); +select makeDate32(-1980, 10, 17); +select makeDate32(1980, -10, 17); +select makeDate32(1980, 10, -17); +select makeDate32(1980.0, '10', 30.0/2); +select makeDate32(-1980.0, '10', 30.0/2); +select makeDate32(cast(1980.1 as Decimal(20,5)), '10', 30.0/2); +select makeDate32(cast('-1980.1' as Decimal(20,5)), '10', 30.0/2); +select makeDate32(cast(1980.1 as Float32), '10', 30.0/2); +select makeDate32(cast(-1980.1 as Float32), '10', 30.0/2); +--select makeDate32(cast(1980 as Date), '10', 30.0/2); +--select makeDate32(cast(-1980 as Date), '10', 30.0/2); +--select makeDate32(cast(1980 as Date32), '10', 30.0/2); +--select makeDate32(cast(-1980 as Date32), '10', 30.0/2); +--select makeDate32(cast(1980 as DateTime), '10', 30.0/2); +--select makeDate32(cast(-1980 as DateTime), '10', 30.0/2); +--select makeDate32(cast(1980 as DateTime64), '10', 30.0/2); +--select makeDate32(cast(-1980 as DateTime64), '10', 30.0/2); + +select makeDate32(0.0, 1, 2); +select makeDate32(1980, 15, 1); +select makeDate32(1980, 2, 29); +select makeDate32(1984, 2, 30); +select makeDate32(19800, 12, 3); +select makeDate32(2148,1,1); +select makeDate32(2149,1,1); +select makeDate32(2149,6,6); +select makeDate32(2149,6,7); +select makeDate32(2150,1,1); +select makeDate32(1969,1,1); +select makeDate32(1969,12,1); +select makeDate32(1969,12,31); +select makeDate32(2282,1,1); +select makeDate32(2283,1,1); +select makeDate32(2283,11,11); +select makeDate32(2283,11,12); +select makeDate32(2284,1,1); +select makeDate32(1924,1,1); +select makeDate32(1924,12,1); +select makeDate32(1924,12,31); +select makeDate32(1970,0,0); +select makeDate32(1970,0,1); +select makeDate32(1970,1,0); +select makeDate32(1990,0,1); +select makeDate32(1990,1,0); + +select makeDate32('1980', '10', '20'); +select makeDate32('-1980', 3, 17); + +select makeDate32('aa', 3, 24); +select makeDate32(1994, 'aa', 24); +select makeDate32(1984, 3, 'aa'); + +select makeDate32(True, 3, 24); +select makeDate32(1994, True, 24); +select makeDate32(1984, 3, True); +select makeDate32(False, 3, 24); +select makeDate32(1994, False, 24); +select makeDate32(1984, 3, False); + +select makeDate32(NULL, 3, 4); +select makeDate32(1980, NULL, 4); +select makeDate32(1980, 3, NULL); + +select makeDate32(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select 1984 as year, 2 as month, 3 as day) order by year, month, day; + +select makeDate32(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select NULL as year, 2 as month, 3 as day) order by year, month, day; + +select makeDate32(year, month, day) from (select 1984 as year, 2 as month, 3 as day union all select 1984 as year, 2 as month, 4 as day) order by year, month, day; From ea7939103abc8ce25e0ed373de606a56f4252495 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 28 Mar 2022 18:43:45 +0200 Subject: [PATCH 095/103] Add debug information about event, rerun only failed jobs --- tests/ci/workflow_approve_rerun_lambda/app.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 50b9d9bfedc..b650d1651fe 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -379,12 +379,16 @@ def check_need_to_rerun(workflow_description): def rerun_workflow(workflow_description, token): print("Going to rerun workflow") - _exec_post_with_retry(workflow_description.rerun_url, token) + try: + _exec_post_with_retry(f"{workflow_description.rerun_url}-failed-jobs", token) + except Exception: + _exec_post_with_retry(workflow_description.rerun_url, token) def main(event): token = get_token_from_aws() event_data = json.loads(event["body"]) + print("The body received:", event_data) workflow_description = get_workflow_description_from_event(event_data) print("Got workflow description", workflow_description) From 9a650456eb6b132d0ee7ae4045d5e180f55fd4da Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 28 Mar 2022 20:42:51 +0300 Subject: [PATCH 096/103] Update 01037_polygon_dicts_correctness_fast.sh --- .../0_stateless/01037_polygon_dicts_correctness_fast.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index ea4389d610c..f6880ae5009 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -18,7 +18,7 @@ DROP TABLE IF EXISTS test_01037.points; CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; " -$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.points FORMAT TSV" --max_insert_block_size=100000 < "${CURDIR}/01037_point_data" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_point_data" rm "${CURDIR}"/01037_point_data @@ -34,7 +34,7 @@ CREATE TABLE test_01037.polygons_array ENGINE = Memory; " -$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.polygons_array FORMAT JSONEachRow" --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data" rm "${CURDIR}"/01037_polygon_data From bf4df5c6bb6d51cc581edac91fac625223c117ab Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 28 Mar 2022 20:40:21 +0300 Subject: [PATCH 097/103] Fix SIGSEGV for build under gcc-11 (due to auto deduction) During building with gcc-11 you will got SIGSEGV for building InterpretersMySQLDDLQuery (and some others), and it is due to endless recursion: (gdb) bt 5 0 0x00000000010978f2 in structural_comptypes (t1=0x7fde028c7dc8, t2=0x7fde028d1e70, strict=0) at ../../src/gcc/cp/typeck.c:1274 1 0x00000000011c3f9d in comp_template_parms (parms1=, parms2=) at ../../src/gcc/cp/pt.c:3369 2 0x0000000001097bd9 in structural_comptypes (t1=0x7fde028c7dc8, t2=0x7fde028d1e70, strict=) at ../../src/gcc/cp/typeck.c:1361 3 0x00000000011c3f9d in comp_template_parms (parms1=, parms2=) at ../../src/gcc/cp/pt.c:3369 4 0x0000000001097bd9 in structural_comptypes (t1=0x7fde028c7dc8, t2=0x7fde028d1e70, strict=) at ../../src/gcc/cp/typeck.c:1361 (gdb) bt -X 1397454 0x0000000001097bd9 in structural_comptypes (t1=0x7fde028d1540, t2=0x7fde028d27e0, strict=) at ../../src/gcc/cp/typeck.c:1361 1397455 0x0000000000f2d8b5 in cp_tree_equal (t1=, t2=) at ../../src/gcc/cp/tree.c:4144 1397456 0x00000000010909cb in template_args_equal (ot=0x7fde028cf578, nt=0x7fde028cfc58, partial_order=) at ../../src/gcc/cp/pt.c:9256 1397457 0x0000000001090422 in template_args_equal (partial_order=false, nt=0x7fde028cfc58, ot=0x7fde028cf578) at ../../src/gcc/cp/pt.c:9295 1397458 comp_template_args (oldargs=0x7fde028cf550, newargs=0x7fde028cfc30, oldarg_ptr=0x0, newarg_ptr=0x0, partial_order=false) at ../../src/gcc/cp/pt.c:9285 1397459 0x00000000010a08f4 in spec_hasher::equal (e1=0x7fde028c95d0, e2=0x7ffd1194e8c0) at ../../src/gcc/cp/pt.c:1726 1397460 0x0000000001085965 in hash_table::find_with_hash (this=0x7fde36b7f450, comparable=@0x7ffd1194e8b8: 0x7ffd1194e8c0, hash=) at ../../src/gcc/hash-table.h:936 1397461 0x0000000001079698 in lookup_template_class_1 (d1=, arglist=0x7fde028cfc30, in_decl=0x0, context=, entering_scope=, complain=3) at ../../src/gcc/cp/pt.c:9896 1397462 0x000000000109f8ef in lookup_template_class (complain=3, entering_scope=1, context=0x7fde27558e40, in_decl=0x0, arglist=0x7fde028cfc08, d1=0x7fde269bcd20) at ../../src/gcc/cp/pt.c:10251 1397463 tsubst_aggr_type (t=0x7fde269bcd20, args=, complain=3, in_decl=0x0, entering_scope=1) at ../../src/gcc/cp/pt.c:13646 1397464 0x000000000108f797 in tsubst (t=0x7fde269bcdc8, args=0x7fde028cf7a8, complain=3, in_decl=) at ../../src/gcc/cp/pt.c:16108 1397465 0x0000000000e61bf2 in rewrite_template_parm (level=1, complain=3, tsubst_args=0x7fde028cf7a8, index=5, olddecl=0x7fde269b5600) at ../../src/gcc/cp/pt.c:28556 1397466 rewrite_tparm_list(tree_node*, unsigned int, unsigned int, tree_node*, unsigned int, int) [clone .constprop.0] (oldelt=0x7fde269bd190, index=5, targs=0x7fde028cf7a8, targs_index=4, complain=3, level=1) at ../../src/gcc/cp/pt.c:28640 1397467 0x00000000009f3748 in build_deduction_guide (type=type@entry=0x7fde26e13dc8, ctor=0x7fde269ac300, outer_args=outer_args@entry=0x0, complain=complain@entry=3) at ../../src/gcc/cp/pt.c:28769 1397468 0x00000000009f444f in ctor_deduction_guides_for (complain=3, tmpl=) at ../../src/gcc/cp/cp-tree.h:842 1397469 deduction_guides_for (tmpl=, any_dguides_p=, complain=3) at ../../src/gcc/cp/pt.c:29282 1397470 0x00000000008507a8 in do_class_deduction (complain=3, flags=1, init=, tmpl=0x7fde26e0f980, ptype=0x7fde028c7b28) at ../../src/gcc/cp/pt.c:29402 1397471 do_auto_deduction (type=0x7fde028c7b28, init=, auto_node=, complain=3, context=, outer_targs=, flags=1) at ../../src/gcc/cp/pt.c:29572 1397472 0x00000000007c9569 in finish_compound_literal (type=, compound_literal=0x7fde028c95b8, complain=3, fcl_context=fcl_functional) at ../../src/gcc/cp/semantics.c:3060 1397473 0x0000000001123a79 in cp_parser_functional_cast (parser=0x7fde27558da8, type=0x7fde028c7b28) at ../../src/gcc/cp/parser.c:30670 1397474 0x0000000000fd7873 in cp_parser_postfix_expression (parser=0x7fde27558da8, address_p=, cast_p=, member_access_only_p=, decltype_p=false, pidk_return=0x0) at ../../src/gcc/cp/parser.c:7437 1397475 0x0000000000fd4ddf in cp_parser_binary_expression (parser=0x7fde27558da8, cast_p=, no_toplevel_fold_p=false, decltype_p=, prec=PREC_NOT_OPERATOR, pidk=) at ../../src/gcc/cp/parser.c:9842 1397476 0x0000000000fd4595 in cp_parser_assignment_expression (parser=0x7fde27558da8, pidk=, cast_p=, decltype_p=) at ../../src/gcc/cp/parser.c:10146 1397477 0x0000000000fd3b90 in cp_parser_constant_expression (parser=0x7fde27558da8, allow_non_constant_p=2, non_constant_p=0x7ffd1194f1d7, strict_p=) at ../../src/gcc/cp/parser.c:10449 1397478 0x0000000000fcfdd5 in cp_parser_initializer_clause (non_constant_p=, parser=0x7fde27558da8) at ../../src/gcc/cp/parser.c:24253 1397479 cp_parser_initializer (parser=0x7fde27558da8, is_direct_init=, non_constant_p=, subexpression_p=) at ../../src/gcc/cp/parser.c:24193 1397480 0x000000000062e5d8 in cp_parser_decomposition_declaration (init_loc=0x7ffd1194f1d8, maybe_range_for_decl=0x7ffd1194f498, decl_specifiers=0x7ffd1194f1f0, parser=0x7fde27558da8) at ../../src/gcc/cp/parser.c:14734 1397481 cp_parser_simple_declaration (parser=0x7fde27558da8, function_definition_allowed_p=, maybe_range_for_decl=0x7ffd1194f498) at ../../src/gcc/cp/parser.c:14393 1397482 0x000000000109b870 in cp_parser_init_statement (parser=0x7fde27558da8, decl=0x7ffd1194f498) at ../../src/gcc/cp/parser.c:13420 1397483 0x00000000010996f0 in cp_parser_for (unroll=0, ivdep=false, parser=0x7fde27558da8) at ../../src/gcc/cp/parser.c:12708 1397484 cp_parser_iteration_statement (parser=0x7fde27558da8, if_p=0x0, ivdep=, unroll=) at ../../src/gcc/cp/parser.c:13343 1397485 0x0000000000fe5c46 in cp_parser_statement (parser=0x7fde27558da8, in_statement_expr=0x0, in_compound=, if_p=0x0, chain=0x0, loc_after_labels=0x0) at ../../src/gcc/cp/parser.c:11718 1397486 0x0000000000fe15ac in cp_parser_statement_seq_opt (in_statement_expr=, parser=) at ../../src/gcc/cp/parser.c:12201 1397487 cp_parser_compound_statement (parser=0x7fde27558da8, in_statement_expr=0x0, bcs_flags=, function_body=) at ../../src/gcc/cp/parser.c:12150 Interesting frame is 1397471, from which we can extract location: (gdb) p line_table[0].info_ordinary.maps[line_table[0].info_ordinary.cache] $54 = { = { start_location = 1396581280 }, members of line_map_ordinary: reason = LC_RENAME, sysp = 0 '\000', m_column_and_range_bits = 8, m_range_bits = 0, to_file = 0x3eb4bb0 "/ch/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp", to_line = 46, included_from = 0 } By replicating SOURCE_LINE() macro from gcc-11 (libcpp/include/line-map.h): /* Converts a map and a location_t to source line. */ inline linenum_type SOURCE_LINE (const line_map_ordinary *ord_map, location_t loc) { return ((loc - ord_map->start_location) >> ord_map->m_column_and_range_bits) + ord_map->to_line; } We got line 154: (gdb) p ((input_location-1396581280) >> 8) + 46 $61 = 154 And this is: auto [column_name_and_type, declare_column_ast] = std::tuple{columns_name_and_type.begin(), columns_definition->children.begin()}; After rewriting it, everything works correctly. Also here is a reproducer for gcc-11 (and no failures for gcc-12, but gcc-12 has other issues, like [1] and one more for hash table): # cat /tmp/test.cpp #include auto multi() { return std::tuple{1, 1}; } double foo() { auto [a, b] = multi(); return a - b; } # g++-11 -std=gnu++20 -c -o /dev/null -isystem /ch/contrib/libcxx/include -nostdinc++ /tmp/test.cpp g++-11: internal compiler error: Segmentation fault signal terminated program cc1plus Please submit a full bug report, with preprocessed source if appropriate. See for instructions. # g++-12 -std=gnu++20 -c -o /dev/null -isystem /ch/contrib/libcxx/include -nostdinc++ /tmp/test.cpp [1]: https://reviews.llvm.org/D122598 Signed-off-by: Azat Khuzhin --- src/AggregateFunctions/AggregateFunctionSumMap.cpp | 2 +- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 12 ++++++------ src/Processors/Transforms/WindowTransform.cpp | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 32c51d5f746..0b3a2067703 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -67,7 +67,7 @@ auto parseArguments(const std::string & name, const DataTypes & arguments) values_types.push_back(array_type->getNestedType()); } - return std::tuple{std::move(keys_type), std::move(values_types), tuple_argument}; + return std::tuple{std::move(keys_type), std::move(values_types), tuple_argument}; } // This function instantiates a particular overload of the sumMap family of diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 515ef6c3058..4c0aa20d795 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -150,12 +150,12 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col ColumnsDescription columns_description; - for ( - auto [column_name_and_type, declare_column_ast] = std::tuple{columns_name_and_type.begin(), columns_definition->children.begin()}; - column_name_and_type != columns_name_and_type.end(); - column_name_and_type++, - declare_column_ast++ - ) + /// FIXME: we could write it like auto [a, b] = std::tuple(x, y), + /// but this produce endless recursion in gcc-11, and leads to SIGSEGV + /// (see git blame for details). + auto column_name_and_type = columns_name_and_type.begin(); + auto declare_column_ast = columns_definition->children.begin(); + for (; column_name_and_type != columns_name_and_type.end(); column_name_and_type++, declare_column_ast++) { const auto & declare_column = (*declare_column_ast)->as(); String comment; diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index b81ed099915..3af72aff2cd 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -488,7 +488,7 @@ auto WindowTransform::moveRowNumberNoCheck(const RowNumber & _x, int64_t offset) } } - return std::tuple{x, offset}; + return std::tuple{x, offset}; } auto WindowTransform::moveRowNumber(const RowNumber & _x, int64_t offset) const @@ -505,7 +505,7 @@ auto WindowTransform::moveRowNumber(const RowNumber & _x, int64_t offset) const assert(oo == 0); #endif - return std::tuple{x, o}; + return std::tuple{x, o}; } From f9c1a44cae036c00fec819872692bed21fb0649e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Mar 2022 21:32:53 +0200 Subject: [PATCH 098/103] Fix black --- .../test_azure_blob_storage_zero_copy_replication/test.py | 2 ++ tests/integration/test_backup_with_other_granularity/test.py | 1 + 2 files changed, 3 insertions(+) diff --git a/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py b/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py index 5ba40f60c53..136d2e2baf6 100644 --- a/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py @@ -14,6 +14,7 @@ CLUSTER_NAME = "test_cluster" drop_table_statement = f"DROP TABLE {TABLE_NAME} ON CLUSTER {CLUSTER_NAME} SYNC" + @pytest.fixture(scope="module") def cluster(): try: @@ -59,6 +60,7 @@ def create_table(node, table_name, replica, **additional_settings): node.query(create_table_statement) assert node.query(f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" + def get_large_objects_count(blob_container_client, large_size_threshold=100): return sum( blob["size"] > large_size_threshold diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index e6739782644..9cb998fb505 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -199,6 +199,7 @@ def test_backup_from_old_version_config(started_cluster): node3.query("DROP TABLE source_table") node3.query("DROP TABLE dest_table") + def test_backup_and_alter(started_cluster): node4.query( "CREATE DATABASE test ENGINE=Ordinary" From b49993f9932ce1bc57e874114332eec5f0a7e626 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 28 Mar 2022 18:04:12 +0200 Subject: [PATCH 099/103] Fixes according to the code review --- src/Functions/makeDate.cpp | 53 ++++++----------- .../0_stateless/02242_make_date.reference | 17 ++++-- tests/queries/0_stateless/02242_make_date.sql | 58 +++++++++++++------ .../0_stateless/02243_make_date32.reference | 9 +-- .../queries/0_stateless/02243_make_date32.sql | 50 ++++++++++------ 5 files changed, 107 insertions(+), 80 deletions(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 049f8af39d1..19fb14efc89 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -1,13 +1,12 @@ #include #include -#include -#include #include +#include #include #include +#include #include -#include #include #include @@ -26,7 +25,7 @@ namespace // A helper function to simplify comparisons of valid YYYY-MM-DD values for <,>,= inline constexpr Int64 YearMonthDayToSingleInt(Int64 year, Int64 month, Int64 day) { - return year*512 + month*32 + day; + return year * 512 + month * 32 + day; } // Common implementation for makeDate, makeDate32 @@ -67,9 +66,9 @@ public: for (size_t i = 0; i < argument_names.size(); ++i) { DataTypePtr argument_type = arguments[i]; - if (!isNumber(argument_type) && !isStringOrFixedString(argument_type)) - throw Exception("Argument '" + std::string(argument_names[i]) + "' for function " + getName() + " must be number", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isNumber(argument_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument '{}' for function {} must be number", std::string(argument_names[i]), getName()); } return std::make_shared(); @@ -77,36 +76,27 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const DataTypePtr converted_argument_type = std::make_shared(); + const DataTypePtr converted_argument_type = std::make_shared(); Columns converted_arguments; - for (size_t i = 0; i < arguments.size(); ++i) + converted_arguments.reserve(arguments.size()); + for (const auto & argument : arguments) { - const auto & argument_type = arguments[i].type; - ColumnPtr argument_column = arguments[i].column->convertToFullColumnIfConst(); - - if (converted_argument_type->equals(*argument_type)) - converted_arguments.push_back(argument_column); - else if (isNumber(argument_type)) - converted_arguments.push_back(convert_to_number_func.executeImpl( - {ColumnWithTypeAndName(argument_column, argument_type, "")}, converted_argument_type, input_rows_count)); - else if (isStringOrFixedString(argument_type)) - converted_arguments.push_back(parse_number_func.executeImpl( - {ColumnWithTypeAndName(argument_column, argument_type, "")}, converted_argument_type, input_rows_count)); - else - throw Exception("Argument '" + std::string(argument_names[i]) + "' for function " + getName() + " must be number", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ColumnPtr argument_column = castColumn(argument, converted_argument_type); + argument_column = argument_column->convertToFullColumnIfConst(); + converted_arguments.push_back(argument_column); } auto res_column = Traits::ReturnColumnType::create(input_rows_count); auto & result_data = res_column->getData(); - const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); - const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); - const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); + const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); + const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); + const auto & day_data = typeid_cast(*converted_arguments[2]).getData(); const auto & date_lut = DateLUT::instance(); - for (size_t i = 0; i < input_rows_count; ++i) { + for (size_t i = 0; i < input_rows_count; ++i) + { const auto year = year_data[i]; const auto month = month_data[i]; const auto day = day_data[i]; @@ -115,8 +105,8 @@ public: if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR && - month <= 12 && - day <= 31 && + month >= 1 && month <= 12 && + day >= 1 && day <= 31 && YearMonthDayToSingleInt(year, month, day) <= Traits::MAX_DATE) { day_num = date_lut.makeDayNum(year, month, day); @@ -127,11 +117,6 @@ public: return res_column; } - -private: - FunctionToFloat64 convert_to_number_func; - FunctionToFloat64OrZero parse_number_func; - FunctionOr or_func; }; // makeDate(year, month, day) diff --git a/tests/queries/0_stateless/02242_make_date.reference b/tests/queries/0_stateless/02242_make_date.reference index de46684cefe..0e8500baeb7 100644 --- a/tests/queries/0_stateless/02242_make_date.reference +++ b/tests/queries/0_stateless/02242_make_date.reference @@ -9,11 +9,11 @@ Nullable(Date) 1970-01-01 1970-01-01 1970-01-01 -1980-10-15 +1980-09-15 1970-01-01 -1980-10-15 +1980-09-17 1970-01-01 -1980-10-15 +1980-09-19 1970-01-01 1970-01-01 1970-01-01 @@ -33,12 +33,21 @@ Nullable(Date) 1970-01-01 1970-01-01 1970-01-01 -1980-10-20 1970-01-01 1970-01-01 1970-01-01 1970-01-01 1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +2009-01-04 +1970-01-01 1994-01-24 1984-03-01 1970-01-01 diff --git a/tests/queries/0_stateless/02242_make_date.sql b/tests/queries/0_stateless/02242_make_date.sql index 83c500adede..9e66daa2778 100644 --- a/tests/queries/0_stateless/02242_make_date.sql +++ b/tests/queries/0_stateless/02242_make_date.sql @@ -10,20 +10,21 @@ select makeDate(1980, 10, 17); select makeDate(-1980, 10, 17); select makeDate(1980, -10, 17); select makeDate(1980, 10, -17); -select makeDate(1980.0, '10', 30.0/2); -select makeDate(-1980.0, '10', 30.0/2); -select makeDate(cast(1980.1 as Decimal(20,5)), '10', 30.0/2); -select makeDate(cast('-1980.1' as Decimal(20,5)), '10', 30.0/2); -select makeDate(cast(1980.1 as Float32), '10', 30.0/2); -select makeDate(cast(-1980.1 as Float32), '10', 30.0/2); ---select makeDate(cast(1980 as Date), '10', 30.0/2); ---select makeDate(cast(-1980 as Date), '10', 30.0/2); ---select makeDate(cast(1980 as Date32), '10', 30.0/2); ---select makeDate(cast(-1980 as Date32), '10', 30.0/2); ---select makeDate(cast(1980 as DateTime), '10', 30.0/2); ---select makeDate(cast(-1980 as DateTime), '10', 30.0/2); ---select makeDate(cast(1980 as DateTime64), '10', 30.0/2); ---select makeDate(cast(-1980 as DateTime64), '10', 30.0/2); +select makeDate(1980.0, 9, 30.0/2); +select makeDate(-1980.0, 9, 32.0/2); +select makeDate(cast(1980.1 as Decimal(20,5)), 9, 17); +select makeDate(cast('-1980.1' as Decimal(20,5)), 9, 18); +select makeDate(cast(1980.1 as Float32), 9, 19); +select makeDate(cast(-1980.1 as Float32), 9, 20); + +select makeDate(cast(1980 as Date), 10, 30); -- { serverError 43 } +select makeDate(cast(-1980 as Date), 10, 30); -- { serverError 43 } +select makeDate(cast(1980 as Date32), 10, 30); -- { serverError 43 } +select makeDate(cast(-1980 as Date32), 10, 30); -- { serverError 43 } +select makeDate(cast(1980 as DateTime), 10, 30); -- { serverError 43 } +select makeDate(cast(-1980 as DateTime), 10, 30); -- { serverError 43 } +select makeDate(cast(1980 as DateTime64), 10, 30); -- { serverError 43 } +select makeDate(cast(-1980 as DateTime64), 10, 30); -- { serverError 43 } select makeDate(0.0, 1, 2); select makeDate(1980, 15, 1); @@ -38,18 +39,33 @@ select makeDate(2150,1,1); select makeDate(1969,1,1); select makeDate(1969,12,1); select makeDate(1969,12,31); +select makeDate(2282,1,1); +select makeDate(2283,1,1); +select makeDate(2283,11,11); +select makeDate(2283,11,12); +select makeDate(2284,1,1); +select makeDate(1924,1,1); +select makeDate(1924,12,1); +select makeDate(1924,12,31); select makeDate(1970,0,0); select makeDate(1970,0,1); select makeDate(1970,1,0); select makeDate(1990,0,1); select makeDate(1990,1,0); -select makeDate('1980', '10', '20'); -select makeDate('-1980', 3, 17); +select makeDate(0x7fff+2010,1,1); +select makeDate(0xffff+2010,1,2); +select makeDate(0x7fffffff+2010,1,3); +select makeDate(0xffffffff+2010,1,4); +select makeDate(0x7fffffffffffffff+2010,1,3); +select makeDate(0xffffffffffffffff+2010,1,4); -select makeDate('aa', 3, 24); -select makeDate(1994, 'aa', 24); -select makeDate(1984, 3, 'aa'); +select makeDate('1980', '10', '20'); -- { serverError 43 } +select makeDate('-1980', 3, 17); -- { serverError 43 } + +select makeDate('aa', 3, 24); -- { serverError 43 } +select makeDate(1994, 'aa', 24); -- { serverError 43 } +select makeDate(1984, 3, 'aa'); -- { serverError 43 } select makeDate(True, 3, 24); select makeDate(1994, True, 24); @@ -62,6 +78,10 @@ select makeDate(NULL, 3, 4); select makeDate(1980, NULL, 4); select makeDate(1980, 3, NULL); +select makeDate(1980); -- { serverError 42 } +select makeDate(1980, 1); -- { serverError 42 } +select makeDate(1980, 1, 1, 1); -- { serverError 42 } + select makeDate(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select 1984 as year, 2 as month, 3 as day) order by year, month, day; select makeDate(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select NULL as year, 2 as month, 3 as day) order by year, month, day; diff --git a/tests/queries/0_stateless/02243_make_date32.reference b/tests/queries/0_stateless/02243_make_date32.reference index 545137b8d4e..ac4b10d371a 100644 --- a/tests/queries/0_stateless/02243_make_date32.reference +++ b/tests/queries/0_stateless/02243_make_date32.reference @@ -9,11 +9,11 @@ Nullable(Date32) 1970-01-01 1970-01-01 1970-01-01 -1980-10-15 +1980-09-15 1970-01-01 -1980-10-15 +1980-09-17 1970-01-01 -1980-10-15 +1980-09-19 1970-01-01 1970-01-01 1970-01-01 @@ -41,12 +41,13 @@ Nullable(Date32) 1970-01-01 1970-01-01 1970-01-01 -1980-10-20 1970-01-01 1970-01-01 1970-01-01 1970-01-01 1970-01-01 +2009-01-04 +1970-01-01 1994-01-24 1984-03-01 1970-01-01 diff --git a/tests/queries/0_stateless/02243_make_date32.sql b/tests/queries/0_stateless/02243_make_date32.sql index 36cb73a85b4..c01855546c5 100644 --- a/tests/queries/0_stateless/02243_make_date32.sql +++ b/tests/queries/0_stateless/02243_make_date32.sql @@ -10,20 +10,21 @@ select makeDate32(1980, 10, 17); select makeDate32(-1980, 10, 17); select makeDate32(1980, -10, 17); select makeDate32(1980, 10, -17); -select makeDate32(1980.0, '10', 30.0/2); -select makeDate32(-1980.0, '10', 30.0/2); -select makeDate32(cast(1980.1 as Decimal(20,5)), '10', 30.0/2); -select makeDate32(cast('-1980.1' as Decimal(20,5)), '10', 30.0/2); -select makeDate32(cast(1980.1 as Float32), '10', 30.0/2); -select makeDate32(cast(-1980.1 as Float32), '10', 30.0/2); ---select makeDate32(cast(1980 as Date), '10', 30.0/2); ---select makeDate32(cast(-1980 as Date), '10', 30.0/2); ---select makeDate32(cast(1980 as Date32), '10', 30.0/2); ---select makeDate32(cast(-1980 as Date32), '10', 30.0/2); ---select makeDate32(cast(1980 as DateTime), '10', 30.0/2); ---select makeDate32(cast(-1980 as DateTime), '10', 30.0/2); ---select makeDate32(cast(1980 as DateTime64), '10', 30.0/2); ---select makeDate32(cast(-1980 as DateTime64), '10', 30.0/2); +select makeDate32(1980.0, 9, 30.0/2); +select makeDate32(-1980.0, 9, 32.0/2); +select makeDate32(cast(1980.1 as Decimal(20,5)), 9, 17); +select makeDate32(cast('-1980.1' as Decimal(20,5)), 9, 18); +select makeDate32(cast(1980.1 as Float32), 9, 19); +select makeDate32(cast(-1980.1 as Float32), 9, 20); + +select makeDate32(cast(1980 as Date), 10, 30); -- { serverError 43 } +select makeDate32(cast(-1980 as Date), 10, 30); -- { serverError 43 } +select makeDate32(cast(1980 as Date32), 10, 30); -- { serverError 43 } +select makeDate32(cast(-1980 as Date32), 10, 30); -- { serverError 43 } +select makeDate32(cast(1980 as DateTime), 10, 30); -- { serverError 43 } +select makeDate32(cast(-1980 as DateTime), 10, 30); -- { serverError 43 } +select makeDate32(cast(1980 as DateTime64), 10, 30); -- { serverError 43 } +select makeDate32(cast(-1980 as DateTime64), 10, 30); -- { serverError 43 } select makeDate32(0.0, 1, 2); select makeDate32(1980, 15, 1); @@ -52,12 +53,19 @@ select makeDate32(1970,1,0); select makeDate32(1990,0,1); select makeDate32(1990,1,0); -select makeDate32('1980', '10', '20'); -select makeDate32('-1980', 3, 17); +select makeDate32(0x7fff+2010,1,1); +select makeDate32(0xffff+2010,1,2); +select makeDate32(0x7fffffff+2010,1,3); +select makeDate32(0xffffffff+2010,1,4); +select makeDate32(0x7fffffffffffffff+2010,1,3); +select makeDate32(0xffffffffffffffff+2010,1,4); -select makeDate32('aa', 3, 24); -select makeDate32(1994, 'aa', 24); -select makeDate32(1984, 3, 'aa'); +select makeDate32('1980', '10', '20'); -- { serverError 43 } +select makeDate32('-1980', 3, 17); -- { serverError 43 } + +select makeDate32('aa', 3, 24); -- { serverError 43 } +select makeDate32(1994, 'aa', 24); -- { serverError 43 } +select makeDate32(1984, 3, 'aa'); -- { serverError 43 } select makeDate32(True, 3, 24); select makeDate32(1994, True, 24); @@ -70,6 +78,10 @@ select makeDate32(NULL, 3, 4); select makeDate32(1980, NULL, 4); select makeDate32(1980, 3, NULL); +select makeDate32(1980); -- { serverError 42 } +select makeDate32(1980, 1); -- { serverError 42 } +select makeDate32(1980, 1, 1, 1); -- { serverError 42 } + select makeDate32(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select 1984 as year, 2 as month, 3 as day) order by year, month, day; select makeDate32(year, month, day) from (select NULL as year, 2 as month, 3 as day union all select NULL as year, 2 as month, 3 as day) order by year, month, day; From 4c336c45fb3e53166b8dc9932a40c4d9ff1f3033 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 28 Mar 2022 23:15:31 +0200 Subject: [PATCH 100/103] Fix enable LLVM for JIT compilation in CMake --- contrib/llvm-cmake/CMakeLists.txt | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/contrib/llvm-cmake/CMakeLists.txt b/contrib/llvm-cmake/CMakeLists.txt index 6ff07f0e016..87c8a65510f 100644 --- a/contrib/llvm-cmake/CMakeLists.txt +++ b/contrib/llvm-cmake/CMakeLists.txt @@ -1,12 +1,9 @@ -# During cross-compilation in our CI we have to use llvm-tblgen and other building tools -# tools to be build for host architecture and everything else for target architecture (e.g. AArch64) -# Possible workaround is to use llvm-tblgen from some package... -# But lets just enable LLVM for native builds -if (CMAKE_CROSSCOMPILING OR SANITIZE STREQUAL "undefined") - set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF) +if (APPLE OR NOT ARCH_AMD64 OR SANITIZE STREQUAL "undefined") + set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF) else() - set (ENABLE_EMBEDDED_COMPILER_DEFAULT ON) + set (ENABLE_EMBEDDED_COMPILER_DEFAULT ON) endif() + option (ENABLE_EMBEDDED_COMPILER "Enable support for 'compile_expressions' option for query execution" ${ENABLE_EMBEDDED_COMPILER_DEFAULT}) if (NOT ENABLE_EMBEDDED_COMPILER) From 85282f1b4dde9e36cfb6f562b9e54e7fb89dfdca Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 29 Mar 2022 00:46:35 +0200 Subject: [PATCH 101/103] Made schema inference work with `s3Cluster` table function (#35544) --- docker/test/stateless/setup_minio.sh | 1 + .../ExternalDataSourceConfiguration.h | 6 + src/Storages/StorageS3Cluster.cpp | 21 ++- src/Storages/StorageS3Cluster.h | 1 + src/TableFunctions/TableFunctionS3.cpp | 107 ++++++------ src/TableFunctions/TableFunctionS3.h | 11 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 112 +++++++------ src/TableFunctions/TableFunctionS3Cluster.h | 17 +- tests/integration/test_storage_s3/test.py | 2 +- .../0_stateless/01801_s3_cluster.reference | 152 ++++++++++++++++++ .../queries/0_stateless/01801_s3_cluster.sql | 21 ++- .../02245_s3_schema_desc.reference | 24 +++ .../0_stateless/02245_s3_schema_desc.sql | 13 ++ 13 files changed, 371 insertions(+), 117 deletions(-) create mode 100644 tests/queries/0_stateless/02245_s3_schema_desc.reference create mode 100644 tests/queries/0_stateless/02245_s3_schema_desc.sql diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index df27b21b05b..aa2aedefad8 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -41,6 +41,7 @@ sleep 5 ./mc admin user add clickminio test testtest ./mc admin policy set clickminio readwrite user=test ./mc mb clickminio/test +./mc policy set public clickminio/test # Upload data to Minio. By default after unpacking all tests will in diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index cc3e136ba50..938922b2bb2 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -114,6 +114,12 @@ struct StorageS3Configuration : URLBasedDataSourceConfiguration String secret_access_key; }; + +struct StorageS3ClusterConfiguration : StorageS3Configuration +{ + String cluster_name; +}; + struct URLBasedDataSourceConfig { URLBasedDataSourceConfiguration configuration; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index b5549b32554..3c031d545fe 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -51,6 +51,7 @@ StorageS3Cluster::StorageS3Cluster( const StorageID & table_id_, String cluster_name_, const String & format_name_, + UInt64 max_single_read_retries_, UInt64 max_connections_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -63,11 +64,26 @@ StorageS3Cluster::StorageS3Cluster( , format_name(format_name_) , compression_method(compression_method_) { + context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI{filename}); StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + StorageS3::updateClientAndAuthSettings(context_, client_auth); + + if (columns_.empty()) + { + const bool is_key_with_globs = filename.find_first_of("*?{") != std::string::npos; + + /// `distributed_processing` is set to false, because this code is executed on the initiator, so there is no callback set + /// for asking for the next tasks. + /// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function + auto columns = StorageS3::getTableStructureFromDataImpl(format_name, client_auth, max_single_read_retries_, compression_method, + /*distributed_processing_*/false, is_key_with_globs, /*format_settings=*/std::nullopt, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - StorageS3::updateClientAndAuthSettings(context_, client_auth); } /// The code executes on initiator @@ -83,7 +99,6 @@ Pipe StorageS3Cluster::read( StorageS3::updateClientAndAuthSettings(context, client_auth); auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); - StorageS3::updateClientAndAuthSettings(context, client_auth); auto iterator = std::make_shared(*client_auth.client, client_auth.uri); auto callback = std::make_shared([iterator]() mutable -> String diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 6d64c56020f..242d374e59b 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -41,6 +41,7 @@ protected: const StorageID & table_id_, String cluster_name_, const String & format_name_, + UInt64 max_single_read_retries_, UInt64 max_connections_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index f844772983a..abd16f138b4 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -22,51 +22,29 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) + +/// This is needed to avoid copy-pase. Because s3Cluster arguments only differ in additional argument (first) - cluster name +void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & s3_configuration) { - /// Parse args - ASTs & args_func = ast_function->children; - - const auto message = fmt::format( - "The signature of table function {} could be the following:\n" \ - " - url\n" - " - url, format\n" \ - " - url, format, structure\n" \ - " - url, access_key_id, secret_access_key\n" \ - " - url, format, structure, compression_method\n" \ - " - url, access_key_id, secret_access_key, format\n" - " - url, access_key_id, secret_access_key, format, structure\n" \ - " - url, access_key_id, secret_access_key, format, structure, compression_method", - getName()); - - if (args_func.size() != 1) - throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTs & args = args_func.at(0)->children; - StorageS3Configuration configuration; - if (auto named_collection = getURLBasedDataSourceConfiguration(args, context)) { auto [common_configuration, storage_specific_args] = named_collection.value(); - configuration.set(common_configuration); + s3_configuration.set(common_configuration); for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "access_key_id") - configuration.access_key_id = arg_value->as()->value.safeGet(); + s3_configuration.access_key_id = arg_value->as()->value.safeGet(); else if (arg_name == "secret_access_key") - configuration.secret_access_key = arg_value->as()->value.safeGet(); + s3_configuration.secret_access_key = arg_value->as()->value.safeGet(); else - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Unknown key-value argument `{}` for StorageS3, expected: " - "url, [access_key_id, secret_access_key], name of used format, structure and [compression_method].", - arg_name); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); } } else { if (args.empty() || args.size() > 6) - throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); @@ -110,53 +88,76 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con } /// This argument is always the first - configuration.url = args[0]->as().value.safeGet(); + s3_configuration.url = args[0]->as().value.safeGet(); if (args_to_idx.contains("format")) - configuration.format = args[args_to_idx["format"]]->as().value.safeGet(); + s3_configuration.format = args[args_to_idx["format"]]->as().value.safeGet(); if (args_to_idx.contains("structure")) - configuration.structure = args[args_to_idx["structure"]]->as().value.safeGet(); + s3_configuration.structure = args[args_to_idx["structure"]]->as().value.safeGet(); if (args_to_idx.contains("compression_method")) - configuration.compression_method = args[args_to_idx["compression_method"]]->as().value.safeGet(); + s3_configuration.compression_method = args[args_to_idx["compression_method"]]->as().value.safeGet(); if (args_to_idx.contains("access_key_id")) - configuration.access_key_id = args[args_to_idx["access_key_id"]]->as().value.safeGet(); + s3_configuration.access_key_id = args[args_to_idx["access_key_id"]]->as().value.safeGet(); if (args_to_idx.contains("secret_access_key")) - configuration.secret_access_key = args[args_to_idx["secret_access_key"]]->as().value.safeGet(); + s3_configuration.secret_access_key = args[args_to_idx["secret_access_key"]]->as().value.safeGet(); } - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url, true); + if (s3_configuration.format == "auto") + s3_configuration.format = FormatFactory::instance().getFormatFromFileName(s3_configuration.url, true); +} - s3_configuration = std::move(configuration); +void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + /// Parse args + ASTs & args_func = ast_function->children; + + const auto message = fmt::format( + "The signature of table function {} could be the following:\n" \ + " - url\n" \ + " - url, format\n" \ + " - url, format, structure\n" \ + " - url, access_key_id, secret_access_key\n" \ + " - url, format, structure, compression_method\n" \ + " - url, access_key_id, secret_access_key, format\n" \ + " - url, access_key_id, secret_access_key, format, structure\n" \ + " - url, access_key_id, secret_access_key, format, structure, compression_method", + getName()); + + if (args_func.size() != 1) + throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + auto & args = args_func.at(0)->children; + + parseArgumentsImpl(message, args, context, configuration); } ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const { - if (s3_configuration->structure == "auto") + if (configuration.structure == "auto") { return StorageS3::getTableStructureFromData( - s3_configuration->format, - S3::URI(Poco::URI(s3_configuration->url)), - s3_configuration->access_key_id, - s3_configuration->secret_access_key, + configuration.format, + S3::URI(Poco::URI(configuration.url)), + configuration.access_key_id, + configuration.secret_access_key, context->getSettingsRef().s3_max_connections, context->getSettingsRef().s3_max_single_read_retries, - s3_configuration->compression_method, + configuration.compression_method, false, std::nullopt, context); } - return parseColumnsListFromString(s3_configuration->structure, context); + return parseColumnsListFromString(configuration.structure, context); } StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - Poco::URI uri (s3_configuration->url); + Poco::URI uri (configuration.url); S3::URI s3_uri (uri); UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; @@ -166,17 +167,17 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context UInt64 max_connections = context->getSettingsRef().s3_max_connections; ColumnsDescription columns; - if (s3_configuration->structure != "auto") - columns = parseColumnsListFromString(s3_configuration->structure, context); + if (configuration.structure != "auto") + columns = parseColumnsListFromString(configuration.structure, context); else if (!structure_hint.empty()) columns = structure_hint; StoragePtr storage = StorageS3::create( s3_uri, - s3_configuration->access_key_id, - s3_configuration->secret_access_key, + configuration.access_key_id, + configuration.secret_access_key, StorageID(getDatabaseName(), table_name), - s3_configuration->format, + configuration.format, max_single_read_retries, min_upload_part_size, upload_part_size_multiply_factor, @@ -189,7 +190,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context context, /// No format_settings for table function S3 std::nullopt, - s3_configuration->compression_method); + configuration.compression_method); storage->startup(); diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 06a327593b0..be84bc4d8ab 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -12,6 +12,7 @@ namespace DB { class Context; +class TableFunctionS3Cluster; /* s3(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary storage for a file in S3. */ @@ -23,13 +24,15 @@ public: { return name; } - bool hasStaticStructure() const override { return s3_configuration->structure != "auto"; } + bool hasStaticStructure() const override { return configuration.structure != "auto"; } - bool needStructureHint() const override { return s3_configuration->structure == "auto"; } + bool needStructureHint() const override { return configuration.structure == "auto"; } void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } protected: + friend class TableFunctionS3Cluster; + StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, @@ -41,7 +44,9 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - std::optional s3_configuration; + static void parseArgumentsImpl(const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & configuration); + + StorageS3Configuration configuration; ColumnsDescription structure_hint; }; diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index bc215b578b9..db57b337952 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -32,6 +32,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_GET; } @@ -45,55 +46,58 @@ void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, Context ASTs & args = args_func.at(0)->children; + for (auto & arg : args) + arg = evaluateConstantExpressionAsLiteral(arg, context); + const auto message = fmt::format( "The signature of table function {} could be the following:\n" \ + " - cluster, url\n" + " - cluster, url, format\n" \ " - cluster, url, format, structure\n" \ + " - cluster, url, access_key_id, secret_access_key\n" \ " - cluster, url, format, structure, compression_method\n" \ + " - cluster, url, access_key_id, secret_access_key, format\n" " - cluster, url, access_key_id, secret_access_key, format, structure\n" \ " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method", getName()); - if (args.size() < 4 || args.size() > 7) + if (args.size() < 2 || args.size() > 7) throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (auto & arg : args) - arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - /// This arguments are always the first - cluster_name = args[0]->as().value.safeGet(); - filename = args[1]->as().value.safeGet(); + configuration.cluster_name = args[0]->as().value.safeGet(); - /// Size -> argument indexes - static auto size_to_args = std::map> - { - {4, {{"format", 2}, {"structure", 3}}}, - {5, {{"format", 2}, {"structure", 3}, {"compression_method", 4}}}, - {6, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}}}, - {7, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}} - }; + if (!context->tryGetCluster(configuration.cluster_name)) + throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", configuration.cluster_name); - auto & args_to_idx = size_to_args[args.size()]; + /// Just cut the first arg (cluster_name) and try to parse s3 table function arguments as is + ASTs clipped_args; + clipped_args.reserve(args.size()); + std::copy(args.begin() + 1, args.end(), std::back_inserter(clipped_args)); - if (args_to_idx.contains("format")) - format = args[args_to_idx["format"]]->as().value.safeGet(); - - if (args_to_idx.contains("structure")) - structure = args[args_to_idx["structure"]]->as().value.safeGet(); - - if (args_to_idx.contains("compression_method")) - compression_method = args[args_to_idx["compression_method"]]->as().value.safeGet(); - - if (args_to_idx.contains("access_key_id")) - access_key_id = args[args_to_idx["access_key_id"]]->as().value.safeGet(); - - if (args_to_idx.contains("secret_access_key")) - secret_access_key = args[args_to_idx["secret_access_key"]]->as().value.safeGet(); + /// StorageS3ClusterConfiguration inherints from StorageS3Configuration, so it is safe to upcast it. + TableFunctionS3::parseArgumentsImpl(message, clipped_args, context, static_cast(configuration)); } ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr context) const { - return parseColumnsListFromString(structure, context); + if (configuration.structure == "auto") + { + return StorageS3::getTableStructureFromData( + configuration.format, + S3::URI(Poco::URI(configuration.url)), + configuration.access_key_id, + configuration.secret_access_key, + context->getSettingsRef().s3_max_connections, + context->getSettingsRef().s3_max_single_read_retries, + configuration.compression_method, + false, + std::nullopt, + context); + } + + return parseColumnsListFromString(configuration.structure, context); } StoragePtr TableFunctionS3Cluster::executeImpl( @@ -101,46 +105,60 @@ StoragePtr TableFunctionS3Cluster::executeImpl( const std::string & table_name, ColumnsDescription /*cached_columns*/) const { StoragePtr storage; + + UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; + UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; + UInt64 upload_part_size_multiply_factor = context->getSettingsRef().s3_upload_part_size_multiply_factor; + UInt64 upload_part_size_multiply_parts_count_threshold = context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold; + UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; + UInt64 max_connections = context->getSettingsRef().s3_max_connections; + + ColumnsDescription columns; + if (configuration.structure != "auto") + columns = parseColumnsListFromString(configuration.structure, context); + else if (!structure_hint.empty()) + columns = structure_hint; + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { /// On worker node this filename won't contains globs - Poco::URI uri (filename); + Poco::URI uri (configuration.url); S3::URI s3_uri (uri); - /// Actually this parameters are not used - UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; - UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; - UInt64 upload_part_size_multiply_factor = context->getSettingsRef().s3_upload_part_size_multiply_factor; - UInt64 upload_part_size_multiply_parts_count_threshold = context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold; - UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; - UInt64 max_connections = context->getSettingsRef().s3_max_connections; storage = StorageS3::create( s3_uri, - access_key_id, - secret_access_key, + configuration.access_key_id, + configuration.secret_access_key, StorageID(getDatabaseName(), table_name), - format, + configuration.format, max_single_read_retries, min_upload_part_size, upload_part_size_multiply_factor, upload_part_size_multiply_parts_count_threshold, max_single_part_upload_size, max_connections, - getActualTableStructure(context), + columns, ConstraintsDescription{}, String{}, context, // No format_settings for S3Cluster std::nullopt, - compression_method, + configuration.compression_method, /*distributed_processing=*/true); } else { storage = StorageS3Cluster::create( - filename, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name), - cluster_name, format, context->getSettingsRef().s3_max_connections, - getActualTableStructure(context), ConstraintsDescription{}, - context, compression_method); + configuration.url, + configuration.access_key_id, + configuration.secret_access_key, + StorageID(getDatabaseName(), table_name), + configuration.cluster_name, configuration.format, + max_single_read_retries, + max_connections, + columns, + ConstraintsDescription{}, + context, + configuration.compression_method); } storage->startup(); diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h index 35d18631ae1..81427b27aab 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ b/src/TableFunctions/TableFunctionS3Cluster.h @@ -5,6 +5,7 @@ #if USE_AWS_S3 #include +#include namespace DB @@ -28,7 +29,12 @@ public: { return name; } - bool hasStaticStructure() const override { return true; } + + bool hasStaticStructure() const override { return configuration.structure != "auto"; } + + bool needStructureHint() const override { return configuration.structure == "auto"; } + + void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } protected: StoragePtr executeImpl( @@ -42,13 +48,8 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr) const override; void parseArguments(const ASTPtr &, ContextPtr) override; - String cluster_name; - String filename; - String format; - String structure; - String access_key_id; - String secret_access_key; - String compression_method = "auto"; + StorageS3ClusterConfiguration configuration; + ColumnsDescription structure_hint; }; } diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index dd29d0a5d6a..644e425c294 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -162,7 +162,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test.csv" put_query = f"""insert into table function s3('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{filename}', - {maybe_auth}'CSV', '{table_format}', {compression}) values settings s3_truncate_on_insert=1 {values}""" + {maybe_auth}'CSV', '{table_format}', '{compression}') values settings s3_truncate_on_insert=1 {values}""" try: run_query(instance, put_query) diff --git a/tests/queries/0_stateless/01801_s3_cluster.reference b/tests/queries/0_stateless/01801_s3_cluster.reference index 5e36c5923ce..31c97f14fa3 100644 --- a/tests/queries/0_stateless/01801_s3_cluster.reference +++ b/tests/queries/0_stateless/01801_s3_cluster.reference @@ -2,6 +2,158 @@ 0 0 0 0 0 0 1 2 3 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +4 5 6 +7 8 9 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +4 5 6 +7 8 9 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +4 5 6 +7 8 9 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +4 5 6 +7 8 9 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 4 5 6 7 8 9 10 11 12 diff --git a/tests/queries/0_stateless/01801_s3_cluster.sql b/tests/queries/0_stateless/01801_s3_cluster.sql index e466b1e108e..68d90ea4be0 100644 --- a/tests/queries/0_stateless/01801_s3_cluster.sql +++ b/tests/queries/0_stateless/01801_s3_cluster.sql @@ -1,5 +1,22 @@ -- Tags: no-fasttest -- Tag no-fasttest: Depends on AWS -select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'a UInt64, b UInt64, c UInt64') ORDER BY a, b, c; -select * from s3Cluster('test_cluster_two_shards', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'a UInt64, b UInt64, c UInt64') ORDER BY a, b, c; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; + + +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; + diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.reference b/tests/queries/0_stateless/02245_s3_schema_desc.reference new file mode 100644 index 00000000000..a5b0f81a2c7 --- /dev/null +++ b/tests/queries/0_stateless/02245_s3_schema_desc.reference @@ -0,0 +1,24 @@ +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) +c1 UInt64 +c2 UInt64 +c3 UInt64 +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) +c1 UInt64 +c2 UInt64 +c3 UInt64 +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) +c1 UInt64 +c2 UInt64 +c3 UInt64 +c1 UInt64 +c2 UInt64 +c3 UInt64 diff --git a/tests/queries/0_stateless/02245_s3_schema_desc.sql b/tests/queries/0_stateless/02245_s3_schema_desc.sql new file mode 100644 index 00000000000..4ab870e1379 --- /dev/null +++ b/tests/queries/0_stateless/02245_s3_schema_desc.sql @@ -0,0 +1,13 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); + +SELECT * FROM s3(decodeURLComponent(NULL), [NULL]); --{serverError 170} From 3ab8fc1e7b15de54661649635b2cdd166c7c83de Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 28 Mar 2022 20:22:29 +0800 Subject: [PATCH 102/103] Fix projection part check Fix projection part check when projection part type is different from parent part type. --- src/Storages/MergeTree/checkDataPart.cpp | 2 +- .../01710_projection_part_check.reference | 3 +++ .../01710_projection_part_check.sql | 24 ++++++++++++------- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index e5c21ed8d3d..522dffe502d 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -136,7 +136,7 @@ IMergeTreeDataPart::Checksums checkDataPart( IMergeTreeDataPart::Checksums projection_checksums_data; const auto & projection_path = file_path; - if (part_type == MergeTreeDataPartType::COMPACT) + if (projection->getType() == MergeTreeDataPartType::COMPACT) { auto proj_path = file_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; auto file_buf = disk->readFile(proj_path); diff --git a/tests/queries/0_stateless/01710_projection_part_check.reference b/tests/queries/0_stateless/01710_projection_part_check.reference index 813e663bdfc..5839e92eeaf 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.reference +++ b/tests/queries/0_stateless/01710_projection_part_check.reference @@ -1,3 +1,6 @@ all_1_1_0 1 all_2_2_0 1 201805_1_1_0 1 +Wide +Compact +all_1_1_0 1 diff --git a/tests/queries/0_stateless/01710_projection_part_check.sql b/tests/queries/0_stateless/01710_projection_part_check.sql index aa087169ad1..5fc960ef426 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.sql +++ b/tests/queries/0_stateless/01710_projection_part_check.sql @@ -8,12 +8,20 @@ insert into tp select number, number from numbers(5); check table tp settings check_query_single_value_result=0; +drop table tp; + +create table tp (p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 ( select p, sum(k), sum(v1), sum(v2) group by p) ) engine = MergeTree partition by toYYYYMM(p) order by k settings min_bytes_for_wide_part = 0; + +insert into tp (p, k, v1, v2) values ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); + +check table tp settings check_query_single_value_result=0; + +drop table tp; + drop table if exists tp; - -CREATE TABLE tp (`p` Date, `k` UInt64, `v1` UInt64, `v2` Int64, PROJECTION p1 ( SELECT p, sum(k), sum(v1), sum(v2) GROUP BY p) ) ENGINE = MergeTree PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO tp (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); - -CHECK TABLE tp settings check_query_single_value_result=0; - -DROP TABLE if exists tp; +create table tp (x int, projection p (select sum(x))) engine = MergeTree order by x settings min_rows_for_wide_part = 2, min_bytes_for_wide_part = 0; +insert into tp values (1), (2), (3), (4); +select part_type from system.parts where database = currentDatabase() and table = 'tp'; +select part_type from system.projection_parts where database = currentDatabase() and table = 'tp'; +check table tp settings check_query_single_value_result=0; +drop table tp; From ddd5bebe555ce8feebcdd339e47fc45184c20dd1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 29 Mar 2022 11:53:51 +0800 Subject: [PATCH 103/103] fix integration test --- tests/integration/test_multiple_disks/test.py | 22 +++++-------------- 1 file changed, 5 insertions(+), 17 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 367948e8751..d7117e2546a 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -846,9 +846,10 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): @pytest.mark.parametrize( - "name,engine", + "name,engine,use_metadata_cache", [ - pytest.param("altering_mt", "MergeTree()", id="mt"), + pytest.param("altering_mt", "MergeTree()", "false", id="mt"), + pytest.param("altering_mt", "MergeTree()", "true", id="mt_use_metadata_cache"), # ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",), # SYSTEM STOP MERGES doesn't disable merges assignments ], @@ -1126,27 +1127,14 @@ def produce_alter_move(node, name): @pytest.mark.parametrize( - "name,engine,use_metadata_cache", + "name,engine", [ - pytest.param("concurrently_altering_mt", "MergeTree()", "false", id="mt"), - pytest.param( - "concurrently_altering_mt", - "MergeTree()", - "true", - id="mt_use_metadata_cache", - ), + pytest.param("concurrently_altering_mt", "MergeTree()", id="mt"), pytest.param( "concurrently_altering_replicated_mt", "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", - "false", id="replicated", ), - pytest.param( - "concurrently_altering_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", - "true", - id="replicated_use_metadata_cache", - ), ], ) def test_concurrent_alter_move(start_cluster, name, engine):