diff --git a/CMakeLists.txt b/CMakeLists.txt index e8b6e9217d2..57b5b7c2ef8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -328,23 +328,9 @@ if (ENABLE_BUILD_PROFILING) endif () endif () -if (${CMAKE_VERSION} VERSION_LESS "3.12.4") - # CMake < 3.12 doesn't support setting 20 as a C++ standard version. - # We will add C++ standard controlling flag in CMAKE_CXX_FLAGS manually for now. - - if (COMPILER_GCC OR COMPILER_CLANG) - # to make numeric_limits<__int128> works with GCC - set (_CXX_STANDARD "gnu++2a") - else () - set (_CXX_STANDARD "c++2a") - endif () - - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=${_CXX_STANDARD}") -else () - set (CMAKE_CXX_STANDARD 20) - set (CMAKE_CXX_EXTENSIONS ON) # Same as gnu++2a (ON) vs c++2a (OFF): https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html - set (CMAKE_CXX_STANDARD_REQUIRED ON) -endif () +set (CMAKE_CXX_STANDARD 20) +set (CMAKE_CXX_EXTENSIONS ON) # Same as gnu++2a (ON) vs c++2a (OFF): https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html +set (CMAKE_CXX_STANDARD_REQUIRED ON) set (CMAKE_C_STANDARD 11) set (CMAKE_C_EXTENSIONS ON) diff --git a/cmake/analysis.cmake b/cmake/analysis.cmake index d1b9c86f15f..b04cabdfa32 100644 --- a/cmake/analysis.cmake +++ b/cmake/analysis.cmake @@ -2,10 +2,6 @@ option (ENABLE_CLANG_TIDY "Use clang-tidy static analyzer" OFF) if (ENABLE_CLANG_TIDY) - if (${CMAKE_VERSION} VERSION_LESS "3.6.0") - message(FATAL_ERROR "clang-tidy requires CMake version at least 3.6.") - endif() - find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-13" "clang-tidy-12" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8") if (CLANG_TIDY_PATH) diff --git a/cmake/arch.cmake b/cmake/arch.cmake index 82c0d40994c..0b8880db81e 100644 --- a/cmake/arch.cmake +++ b/cmake/arch.cmake @@ -1,22 +1,15 @@ if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") + if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386") + message (FATAL_ERROR "32bit platforms are not supported") + endif () set (ARCH_AMD64 1) -endif () -if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)") +elseif (CMAKE_SYSTEM_PROCESSOR MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)") set (ARCH_AARCH64 1) -endif () -if (ARCH_AARCH64 OR CMAKE_SYSTEM_PROCESSOR MATCHES "arm") set (ARCH_ARM 1) -endif () -if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386") - set (ARCH_I386 1) -endif () -if ((ARCH_ARM AND NOT ARCH_AARCH64) OR ARCH_I386) - message (FATAL_ERROR "32bit platforms are not supported") -endif () -if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)") +elseif (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)") set (ARCH_PPC64LE 1) -endif () -if (CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64") +elseif (CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64") set (ARCH_RISCV64 1) +else () + message (FATAL_ERROR "Platform ${CMAKE_SYSTEM_PROCESSOR} is not supported") endif () - diff --git a/cmake/tools.cmake b/cmake/tools.cmake index d571a46ad26..350bc488837 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -13,7 +13,7 @@ execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version) if (COMPILER_GCC) # Require minimum version of gcc set (GCC_MINIMUM_VERSION 11) - if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${GCC_MINIMUM_VERSION} AND NOT CMAKE_VERSION VERSION_LESS 2.8.9) + if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${GCC_MINIMUM_VERSION}) message (FATAL_ERROR "GCC version must be at least ${GCC_MINIMUM_VERSION}. For example, if GCC ${GCC_MINIMUM_VERSION} is available under gcc-${GCC_MINIMUM_VERSION}, g++-${GCC_MINIMUM_VERSION} names, do the following: export CC=gcc-${GCC_MINIMUM_VERSION} CXX=g++-${GCC_MINIMUM_VERSION}; rm -rf CMakeCache.txt CMakeFiles; and re run cmake or ./release.") endif () diff --git a/docker/test/integration/runner/compose/docker_compose_keeper.yml b/docker/test/integration/runner/compose/docker_compose_keeper.yml index a729a9c7734..811bbdd800d 100644 --- a/docker/test/integration/runner/compose/docker_compose_keeper.yml +++ b/docker/test/integration/runner/compose/docker_compose_keeper.yml @@ -17,7 +17,7 @@ services: - type: ${keeper_fs:-tmpfs} source: ${keeper_db_dir1:-} target: /var/lib/clickhouse-keeper - entrypoint: "${keeper_cmd_prefix:-} --config=/etc/clickhouse-keeper/keeper_config1.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" + entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config1.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" cap_add: - SYS_PTRACE - NET_ADMIN @@ -47,7 +47,7 @@ services: - type: ${keeper_fs:-tmpfs} source: ${keeper_db_dir2:-} target: /var/lib/clickhouse-keeper - entrypoint: "${keeper_cmd_prefix:-} --config=/etc/clickhouse-keeper/keeper_config2.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" + entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config2.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" cap_add: - SYS_PTRACE - NET_ADMIN @@ -77,7 +77,7 @@ services: - type: ${keeper_fs:-tmpfs} source: ${keeper_db_dir3:-} target: /var/lib/clickhouse-keeper - entrypoint: "${keeper_cmd_prefix:-} --config=/etc/clickhouse-keeper/keeper_config3.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" + entrypoint: "${keeper_cmd_prefix:-clickhouse keeper} --config=/etc/clickhouse-keeper/keeper_config3.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log" cap_add: - SYS_PTRACE - NET_ADMIN diff --git a/docs/en/development/build.md b/docs/en/development/build.md index b128412a55e..f783a2ec62e 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -75,7 +75,7 @@ This will create the `programs/clickhouse` executable, which can be used with `c The build requires the following components: - Git (is used only to checkout the sources, it’s not needed for the build) -- CMake 3.10 or newer +- CMake 3.14 or newer - Ninja - C++ compiler: clang-13 or newer - Linker: lld diff --git a/docs/en/development/integrating_rust_libraries.md b/docs/en/development/integrating_rust_libraries.md index c2c5228f489..92d30d63986 100644 --- a/docs/en/development/integrating_rust_libraries.md +++ b/docs/en/development/integrating_rust_libraries.md @@ -1,4 +1,4 @@ -# Integrating Rust libraries into ClickHouse. +# Integrating Rust libraries Rust library integration will be described based on BLAKE3 hash-function integration. diff --git a/docs/en/operations/system-tables/processes.md b/docs/en/operations/system-tables/processes.md index f261ee9b696..a77fbf2a109 100644 --- a/docs/en/operations/system-tables/processes.md +++ b/docs/en/operations/system-tables/processes.md @@ -13,6 +13,8 @@ Columns: - `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. - `query` (String) – The query text. For `INSERT`, it does not include the data to insert. - `query_id` (String) – Query ID, if defined. +- `is_cancelled` (Int8) – Was query cancelled. +- `is_all_data_sent` (Int8) – Was all data sent to the client (in other words query had been finished on the server). ```sql :) SELECT * FROM system.processes LIMIT 10 FORMAT Vertical; @@ -43,6 +45,7 @@ http_user_agent: quota_key: elapsed: 0.000582537 is_cancelled: 0 +is_all_data_sent: 0 read_rows: 0 read_bytes: 0 total_rows_approx: 0 diff --git a/docs/tools/website.py b/docs/tools/website.py index 2c748d96414..e638af7b4b0 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -322,8 +322,9 @@ def process_benchmark_results(args): required_keys = { "dbms": ["result"], "hardware": ["result", "system", "system_full", "kind"], + "versions": ["version", "system"], } - for benchmark_kind in ["dbms", "hardware"]: + for benchmark_kind in ["dbms", "hardware", "versions"]: results = [] results_root = os.path.join(benchmark_root, benchmark_kind, "results") for result in sorted(os.listdir(results_root)): diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 2f9eec33022..e8d6c52d948 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -555,7 +555,7 @@ void LRUFileCache::remove(const Key & key) fs::remove(key_path); } -void LRUFileCache::tryRemoveAll() +void LRUFileCache::remove(bool force_remove_unreleasable) { /// Try remove all cached files by cache_base_path. /// Only releasable file segments are evicted. @@ -567,12 +567,13 @@ void LRUFileCache::tryRemoveAll() auto & [key, offset] = *it++; auto * cell = getCell(key, offset, cache_lock); - if (cell->releasable()) + if (cell->releasable() || force_remove_unreleasable) { auto file_segment = cell->file_segment; if (file_segment) { std::lock_guard segment_lock(file_segment->mutex); + file_segment->detached = true; remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); } } diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 983156959de..d18bdc74acf 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -42,7 +42,7 @@ public: virtual void remove(const Key & key) = 0; - virtual void tryRemoveAll() = 0; + virtual void remove(bool force_remove_unreleasable) = 0; static bool isReadOnly(); @@ -145,7 +145,7 @@ public: void remove(const Key & key) override; - void tryRemoveAll() override; + void remove(bool force_remove_unreleasable) override; std::vector tryGetCachePaths(const Key & key) override; diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index e94a54ec3f7..a14051fa9c8 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -455,6 +455,8 @@ void FileSegment::complete(State state) std::lock_guard cache_lock(cache->mutex); std::lock_guard segment_lock(mutex); + assertNotDetached(); + bool is_downloader = isDownloaderImpl(segment_lock); if (!is_downloader) { @@ -477,8 +479,6 @@ void FileSegment::complete(State state) download_state = state; - assertNotDetached(); - try { completeImpl(cache_lock, segment_lock); diff --git a/src/Common/examples/cow_compositions.cpp b/src/Common/examples/cow_compositions.cpp index f013e751063..05d6fe1d89e 100644 --- a/src/Common/examples/cow_compositions.cpp +++ b/src/Common/examples/cow_compositions.cpp @@ -52,7 +52,7 @@ private: { std::cerr << "Mutating\n"; auto res = shallowMutate(); - res->wrapped = IColumn::mutate(wrapped); + res->wrapped = IColumn::mutate(std::move(res->wrapped).detach()); return res; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 908e0184b8d..558b13927c1 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -54,9 +54,6 @@ static std::optional checkTupleNames(const Strings & names) if (name.empty()) return Exception("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS); - if (isNumericASCII(name[0])) - return Exception("Explicitly specified names of tuple elements cannot start with digit", ErrorCodes::BAD_ARGUMENTS); - if (!names_set.insert(name).second) return Exception("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN); } diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index bff33f9b061..440aa2ca463 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -20,6 +21,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int DECIMAL_OVERFLOW; } @@ -217,7 +219,9 @@ namespace { if (scale_multiplier < 1000) { - Int64 t_milliseconds = t * (static_cast(1000) / scale_multiplier); + Int64 t_milliseconds = 0; + if (common::mulOverflow(t, static_cast(1000) / scale_multiplier, t_milliseconds)) + throw DB::Exception("Numeric overflow", ErrorCodes::DECIMAL_OVERFLOW); if (likely(t >= 0)) return t_milliseconds / milliseconds * milliseconds; else @@ -252,7 +256,9 @@ namespace { if (scale_multiplier < 1000000) { - Int64 t_microseconds = t * (static_cast(1000000) / scale_multiplier); + Int64 t_microseconds = 0; + if (common::mulOverflow(t, static_cast(1000000) / scale_multiplier, t_microseconds)) + throw DB::Exception("Numeric overflow", ErrorCodes::DECIMAL_OVERFLOW); if (likely(t >= 0)) return t_microseconds / microseconds * microseconds; else @@ -287,7 +293,9 @@ namespace { if (scale_multiplier < 1000000000) { - Int64 t_nanoseconds = t * (static_cast(1000000000) / scale_multiplier); + Int64 t_nanoseconds = 0; + if (common::mulOverflow(t, (static_cast(1000000000) / scale_multiplier), t_nanoseconds)) + throw DB::Exception("Numeric overflow", ErrorCodes::DECIMAL_OVERFLOW); if (likely(t >= 0)) return t_nanoseconds / nanoseconds * nanoseconds; else diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 28a2082d233..289aba2a695 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -306,12 +306,12 @@ BlockIO InterpreterSystemQuery::execute() { auto caches = FileCacheFactory::instance().getAll(); for (const auto & [_, cache_data] : caches) - cache_data.cache->tryRemoveAll(); + cache_data.cache->remove(query.force_removal); } else { auto cache = FileCacheFactory::instance().get(query.filesystem_cache_path); - cache->tryRemoveAll(); + cache->remove(query.force_removal); } break; } diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 26146781327..0f728ec7a33 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -455,6 +455,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.client_info = client_info; res.elapsed_seconds = watch.elapsedSeconds(); res.is_cancelled = is_killed.load(std::memory_order_relaxed); + res.is_all_data_sent = is_all_data_sent.load(std::memory_order_relaxed); res.read_rows = progress_in.read_rows; res.read_bytes = progress_in.read_bytes; res.total_rows = progress_in.total_rows_to_read; diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index c5c83904b7c..a30c0f59feb 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -61,6 +61,7 @@ struct QueryStatusInfo Int64 peak_memory_usage; ClientInfo client_info; bool is_cancelled; + bool is_all_data_sent; /// Optional fields, filled by query std::vector thread_ids; @@ -101,6 +102,9 @@ protected: std::atomic is_killed { false }; + /// All data to the client already had been sent. Including EndOfStream. + std::atomic is_all_data_sent { false }; + void setUserProcessList(ProcessListForUser * user_process_list_); /// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently. const ProcessListForUser * getUserProcessList() const { return user_process_list; } @@ -194,6 +198,9 @@ public: bool isKilled() const { return is_killed; } + bool isAllDataSent() const { return is_all_data_sent; } + void setAllDataSent() { is_all_data_sent = true; } + /// Adds a pipeline to the QueryStatus void addPipelineExecutor(PipelineExecutor * e); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a4b0a69faaa..274d1639b4b 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -192,6 +192,13 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, << (settings.hilite ? hilite_keyword : "") << " SECOND" << (settings.hilite ? hilite_none : ""); } + else if (type == Type::DROP_FILESYSTEM_CACHE) + { + if (!filesystem_cache_path.empty()) + settings.ostr << (settings.hilite ? hilite_none : "") << filesystem_cache_path; + if (force_removal) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORCE"; + } } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 600525f9abe..213b741f63b 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -89,7 +89,10 @@ public: String volume; String disk; UInt64 seconds{}; + + /// Values for `drop filesystem cache` system query. String filesystem_cache_path; + bool force_removal = false; String getID(char) const override { return "SYSTEM query"; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 9c9989dc39f..0c36aeb3141 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -355,8 +355,9 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr settings; bool storage_like = false; + bool parsed_engine_keyword = s_engine.ignore(pos, expected); - if (s_engine.ignore(pos, expected)) + if (parsed_engine_keyword) { s_eq.ignore(pos, expected); @@ -422,7 +423,10 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - if (s_settings.ignore(pos, expected)) + /// Do not allow SETTINGS clause without ENGINE, + /// because we cannot distinguish engine settings from query settings in this case. + /// And because settings for each engine are different. + if (parsed_engine_keyword && s_settings.ignore(pos, expected)) { if (!settings_p.parse(pos, settings, expected)) return false; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 61e96b9c1de..8990becaedc 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -346,6 +346,16 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & res->seconds = seconds->as()->value.get(); break; } + case Type::DROP_FILESYSTEM_CACHE: + { + ParserLiteral path_parser; + ASTPtr ast; + if (path_parser.parse(pos, ast, expected)) + res->filesystem_cache_path = ast->as()->value.safeGet(); + if (ParserKeyword{"FORCE"}.ignore(pos, expected)) + res->force_removal = true; + break; + } default: { diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 859c9fd9e19..255beebbdf1 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -98,6 +98,8 @@ void SortingStep::updateInputStream(DataStream input_stream) void SortingStep::updateOutputStream(Block result_header) { output_stream = createOutputStream(input_streams.at(0), std::move(result_header), getDataStreamTraits()); + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; updateDistinctColumns(output_stream->header, output_stream->distinct_columns); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b6c4b068d32..911d0a4e76b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -1710,6 +1711,12 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace) void TCPHandler::sendEndOfStream() { state.sent_all_data = true; + /// The following queries does not have process_list_entry: + /// - internal + /// - SHOW PROCESSLIST + if (state.io.process_list_entry) + (*state.io.process_list_entry)->setAllDataSent(); + writeVarUInt(Protocol::Server::EndOfStream, *out); out->next(); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1b026279989..a1946ced259 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1414,6 +1414,9 @@ bool IMergeTreeDataPart::assertHasValidVersionMetadata() const if (part_is_probably_removed_from_disk) return true; + if (state == State::Temporary) + return true; + DiskPtr disk = volume->getDisk(); if (!disk->exists(getFullRelativePath())) return true; @@ -1772,6 +1775,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) void IMergeTreeDataPart::renameToDetached(const String & prefix) const { renameTo(getRelativePathForDetachedPart(prefix), true); + part_is_probably_removed_from_disk = true; } void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index e300ceeb4fc..dac304fcdff 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -44,12 +44,11 @@ bool injectRequiredColumnsRecursively( if (alter_conversions.isColumnRenamed(column_name_in_part)) column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); - auto column_in_part = NameAndTypePair( - column_name_in_part, column_in_storage->getSubcolumnName(), - column_in_storage->getTypeInStorage(), column_in_storage->type); + auto column_in_part = part->getColumns().tryGetByName(column_name_in_part); - /// column has files and hence does not require evaluation - if (part->hasColumnFiles(column_in_part)) + if (column_in_part + && (!column_in_storage->isSubcolumn() + || column_in_part->type->tryGetSubcolumnType(column_in_storage->getSubcolumnName()))) { /// ensure each column is added only once if (!required_columns.contains(column_name)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 89515b863ca..efaf605d7d3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -504,4 +504,15 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat return {new_part_name}; } +String ReplicatedMergeTreeLogEntryData::getDescriptionForLogs(MergeTreeDataFormatVersion format_version) const +{ + String description = fmt::format("{} with virtual parts [{}]", typeToString(), fmt::join(getVirtualPartNames(format_version), ", ")); + if (auto drop_range = getDropRange(format_version)) + { + description += " and drop range "; + description += *drop_range; + } + return description; +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 4d8f319c94a..19a463e4914 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -144,6 +144,8 @@ struct ReplicatedMergeTreeLogEntryData /// Returns fake part for drop range (for DROP_RANGE and REPLACE_RANGE) std::optional getDropRange(MergeTreeDataFormatVersion format_version) const; + String getDescriptionForLogs(MergeTreeDataFormatVersion format_version) const; + /// This entry is DROP PART, not DROP PARTITION. They both have same /// DROP_RANGE entry type, but differs in information about drop range. bool isDropPart(MergeTreeDataFormatVersion format_version) const; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 7dc064431d7..0e7c83742a4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -213,7 +213,7 @@ void ReplicatedMergeTreeQueue::insertUnlocked( { auto entry_virtual_parts = entry->getVirtualPartNames(format_version); - LOG_TEST(log, "Insert entry {} to queue with type {} with virtual parts [{}]", entry->znode_name, entry->typeToString(), fmt::join(entry_virtual_parts, ", ")); + LOG_TEST(log, "Insert entry {} to queue with type {}", entry->znode_name, entry->getDescriptionForLogs(format_version)); for (const String & virtual_part_name : entry_virtual_parts) { @@ -293,9 +293,9 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( { auto entry_virtual_parts = entry->getVirtualPartNames(format_version); - LOG_TEST(log, "Removing {} entry {} from queue with type {} with virtual parts [{}]", + LOG_TEST(log, "Removing {} entry {} from queue with type {}", is_successful ? "successful" : "unsuccessful", - entry->znode_name, entry->typeToString(), fmt::join(entry_virtual_parts, ", ")); + entry->znode_name, entry->getDescriptionForLogs(format_version)); /// Update insert times. if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART) { diff --git a/src/Storages/System/StorageSystemProcesses.cpp b/src/Storages/System/StorageSystemProcesses.cpp index efa01561ad4..08d3666216f 100644 --- a/src/Storages/System/StorageSystemProcesses.cpp +++ b/src/Storages/System/StorageSystemProcesses.cpp @@ -52,6 +52,7 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes() {"elapsed", std::make_shared()}, {"is_cancelled", std::make_shared()}, + {"is_all_data_sent", std::make_shared()}, {"read_rows", std::make_shared()}, {"read_bytes", std::make_shared()}, {"total_rows_approx", std::make_shared()}, @@ -120,6 +121,7 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr c res_columns[i++]->insert(process.elapsed_seconds); res_columns[i++]->insert(process.is_cancelled); + res_columns[i++]->insert(process.is_all_data_sent); res_columns[i++]->insert(process.read_rows); res_columns[i++]->insert(process.read_bytes); res_columns[i++]->insert(process.total_rows); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a1f1f467e59..878e6de264f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -225,6 +225,7 @@ def get_processlist_after_test(args): FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) WHERE query NOT LIKE '%system.processes%' AND + NOT is_all_data_sent AND Settings['log_comment'] = '{log_comment}' AND current_database = '{database}' """) @@ -234,6 +235,7 @@ def get_processlist_after_test(args): FROM system.processes WHERE query NOT LIKE '%system.processes%' AND + NOT is_all_data_sent AND Settings['log_comment'] = '{log_comment}' AND current_database = '{database}' """) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 3dd4811b1bf..45fad002c88 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -10,7 +10,18 @@ 0 22548578304 1 + ./s3_cache/ + + s3 + http://localhost:11111/test/00170_test/ + clickhouse + clickhouse + 1 + 0 + 22548578304 + 0 + @@ -20,6 +31,13 @@ + + +
+ s3_cache_2 +
+
+
diff --git a/tests/performance/function_calculation_after_sorting_and_limit.xml b/tests/performance/function_calculation_after_sorting_and_limit.xml index ddb8f860600..bd4b0e57aaa 100644 --- a/tests/performance/function_calculation_after_sorting_and_limit.xml +++ b/tests/performance/function_calculation_after_sorting_and_limit.xml @@ -1,4 +1,5 @@ SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number LIMIT 5 SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number + 1 LIMIT 5 + SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number + 1 LIMIT 99999995, 5 diff --git a/tests/queries/0_stateless/00061_merge_tree_alter.sql b/tests/queries/0_stateless/00061_merge_tree_alter.sql index 01c9c1ffb76..822386baa47 100644 --- a/tests/queries/0_stateless/00061_merge_tree_alter.sql +++ b/tests/queries/0_stateless/00061_merge_tree_alter.sql @@ -1,3 +1,5 @@ +-- Tags: no-backward-compatibility-check + DROP TABLE IF EXISTS alter_00061; CREATE TABLE alter_00061 (d Date, k UInt64, i32 Int32) ENGINE=MergeTree(d, k, 8192); diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index ab348fd31fb..9fef47c8d5d 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -17,9 +17,10 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE dst (n UInt64, type UInt8) ENGINE=Merge function thread_insert() { set -e - trap "exit 0" INT val=1 - while true; do + trap "STOP_THE_LOOP=1" INT + STOP_THE_LOOP=0 + while [[ $STOP_THE_LOOP != 1 ]]; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); @@ -91,8 +92,9 @@ function thread_partition_dst_to_src() function thread_select() { set -e - trap "exit 0" INT - while true; do + trap "STOP_THE_LOOP=1" INT + STOP_THE_LOOP=0 + while [[ $STOP_THE_LOOP != 1 ]]; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; -- no duplicates diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 5086515e9eb..261fa480491 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -50,8 +50,9 @@ function thread_insert_rollback() function thread_optimize() { set -e - trap "exit 0" INT - while true; do + trap "STOP_THE_LOOP=1" INT + STOP_THE_LOOP=0 + while [[ $STOP_THE_LOOP != 1 ]]; do optimize_query="OPTIMIZE TABLE src" partition_id=$(( RANDOM % 2 )) if (( RANDOM % 2 )); then @@ -102,8 +103,9 @@ function thread_select() function thread_select_insert() { set -e - trap "exit 0" INT - while true; do + trap "STOP_THE_LOOP=1" INT + STOP_THE_LOOP=0 + while [[ $STOP_THE_LOOP != 1 ]]; do $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; SELECT throwIf((SELECT count() FROM tmp) != 0) FORMAT Null; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 8872ab82c03..4bce09cf1d5 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -35,8 +35,9 @@ function thread_insert_rollback() function thread_select() { - trap "exit 0" INT - while true; do + trap "STOP_THE_LOOP=1" INT + STOP_THE_LOOP=0 + while [[ $STOP_THE_LOOP != 1 ]]; do # Result of `uniq | wc -l` must be 1 if the first and the last queries got the same result $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; diff --git a/tests/queries/0_stateless/01460_mark_inclusion_search_crash.sql b/tests/queries/0_stateless/01460_mark_inclusion_search_crash.sql index 148ecdab490..86c4d988c38 100644 --- a/tests/queries/0_stateless/01460_mark_inclusion_search_crash.sql +++ b/tests/queries/0_stateless/01460_mark_inclusion_search_crash.sql @@ -7,6 +7,6 @@ INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (2, 11, 4395), (3, 22, 3545), (4, SET max_block_size = 1; SET max_rows_to_read = 5; -SELECT toUInt32(x), y, z FROM pk WHERE (x >= toDateTime(100000)) AND (x <= toDateTime(3)); +SELECT toUInt32(x), y, z FROM pk WHERE (x >= toDateTime(100000)) AND (x <= toDateTime(90000)); DROP TABLE IF EXISTS pk; diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index 68875735110..678cbf7fb57 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -33,13 +33,12 @@ Expression (Projection) Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree (default.test_table) -Expression (Projection) +Expression ((Projection + Before ORDER BY [lifted up part])) Limit (preliminary LIMIT (without OFFSET)) - Expression (Before ORDER BY [lifted up part]) - Sorting - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromMergeTree (default.test_table) + Sorting + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromMergeTree (default.test_table) optimize_aggregation_in_order Expression ((Projection + Before ORDER BY)) Aggregating diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index bb9c614f728..f870a52284c 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -144,7 +144,7 @@ Filter 2 3 > function calculation should be done after sorting and limit (if possible) > Expression should be divided into two subexpressions and only one of them should be moved after Sorting -Expression (Before ORDER BY [lifted up part]) +Expression ((Projection + Before ORDER BY [lifted up part])) FUNCTION sipHash64 Sorting Expression (Before ORDER BY) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 0b7f004a2ce..3a3ce95460c 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -201,7 +201,7 @@ echo "> function calculation should be done after sorting and limit (if possible echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting" $CLICKHOUSE_CLIENT -q " explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" | - sed 's/^ *//g' | grep -o "^ *\(Expression (Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)" + sed 's/^ *//g' | grep -o "^ *\(Expression (.*Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)" echo "> this query should be executed without throwing an exception" $CLICKHOUSE_CLIENT -q " select throwIf(number = 5) from (select * from numbers(10)) order by number limit 1" diff --git a/tests/queries/0_stateless/01825_type_json_wide_parts_merge.reference b/tests/queries/0_stateless/01825_type_json_wide_parts_merge.reference new file mode 100644 index 00000000000..b41a6da16f5 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_wide_parts_merge.reference @@ -0,0 +1,3 @@ +{"data":{"k1":0,"k2":2}} +{"data":{"k1":1,"k2":0}} +Tuple(k1 Int8, k2 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_wide_parts_merge.sql b/tests/queries/0_stateless/01825_type_json_wide_parts_merge.sql new file mode 100644 index 00000000000..fd023e473a4 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_wide_parts_merge.sql @@ -0,0 +1,25 @@ +-- Tags: no-fasttest + +SET allow_experimental_object_type = 1; +SET output_format_json_named_tuples_as_objects = 1; + +DROP TABLE IF EXISTS t_json_wide_parts; + +CREATE TABLE t_json_wide_parts (data JSON) +ENGINE MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +SYSTEM STOP MERGES t_json_wide_parts; + +INSERT INTO t_json_wide_parts VALUES ('{"k1": 1}'); +INSERT INTO t_json_wide_parts VALUES ('{"k2": 2}'); + +SYSTEM START MERGES t_json_wide_parts; +OPTIMIZE TABLE t_json_wide_parts FINAL; + +SELECT data FROM t_json_wide_parts ORDER BY data.k1 FORMAT JSONEachRow; + +SELECT type FROM system.parts_columns +WHERE table = 't_json_wide_parts' AND database = currentDatabase() AND active; + +DROP TABLE t_json_wide_parts; diff --git a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference index 7326d960397..5290ffc5dbe 100644 --- a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference +++ b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference @@ -1 +1,2 @@ -Ok +SleepFunctionCalls: 4 (increment) +SleepFunctionMicroseconds: 400000 (increment) diff --git a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh index 1d70ba1df7c..26fcfd38c48 100755 --- a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh +++ b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh @@ -7,22 +7,16 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --multiquery --query " drop table if exists aliases_lazyness; create table aliases_lazyness (x UInt32, y ALIAS sleepEachRow(0.1)) Engine=MergeTree ORDER BY x; -insert into aliases_lazyness(x) select * from numbers(40); +insert into aliases_lazyness(x) select * from numbers(100); " # In very old ClickHouse versions alias column was calculated for every row. -# If it works this way, the query will take at least 0.1 * 40 = 4 seconds. -# If the issue does not exist, the query should take slightly more than 0.1 seconds. -# The exact time is not guaranteed, so we check in a loop that at least once -# the query will process in less than one second, that proves that the behaviour is not like it was long time ago. +# If it works this way, the query will take at least 0.1 * 100 = 10 seconds. +# If the issue does not exist, the query should call sleepEachRow() "only" 4 times: +# - from MergeTreeData::getQueryProcessingStageWithAggregateProjection() -> MergeTreeWhereOptimizer -> getBlockWithConstants() +# - from MergeTreeWhereOptimizer -> getBlockWithConstants() +# - ReadFromMergeTree::selectRangesToRead() -> getBlockWithConstants() +# - Pipeline +${CLICKHOUSE_CLIENT} --profile-events-delay-ms=-1 --print-profile-events --query "SELECT x, y FROM aliases_lazyness WHERE x = 1 FORMAT Null" |& grep -o -e "SleepFunctionMicroseconds.*" -e "SleepFunctionCalls.*" -i=0 retries=300 -while [[ $i -lt $retries ]]; do - timeout 1 ${CLICKHOUSE_CLIENT} --query "SELECT x, y FROM aliases_lazyness WHERE x = 1 FORMAT Null" && break - ((++i)) -done - -${CLICKHOUSE_CLIENT} --multiquery --query " -drop table aliases_lazyness; -SELECT 'Ok'; -" +${CLICKHOUSE_CLIENT} --query "drop table aliases_lazyness" diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 246b8ef6d3b..a67cf5770bf 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -1,66 +1,1113 @@ -CREATE TABLE system.aggregate_function_combinators\n(\n `name` String,\n `is_internal` UInt8\n)\nENGINE = SystemAggregateFunctionCombinators()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.asynchronous_inserts\n(\n `query` String,\n `database` String,\n `table` String,\n `format` String,\n `first_update` DateTime64(6),\n `last_update` DateTime64(6),\n `total_bytes` UInt64,\n `entries.query_id` Array(String),\n `entries.bytes` Array(UInt64),\n `entries.finished` Array(UInt8),\n `entries.exception` Array(String)\n)\nENGINE = SystemAsynchronousInserts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.asynchronous_metrics\n(\n `metric` String,\n `value` Float64\n)\nENGINE = SystemAsynchronousMetrics()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.build_options\n(\n `name` String,\n `value` String\n)\nENGINE = SystemBuildOptions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.clusters\n(\n `cluster` String,\n `shard_num` UInt32,\n `shard_weight` UInt32,\n `replica_num` UInt32,\n `host_name` String,\n `host_address` String,\n `port` UInt16,\n `is_local` UInt8,\n `user` String,\n `default_database` String,\n `errors_count` UInt32,\n `slowdowns_count` UInt32,\n `estimated_recovery_time` UInt32\n)\nENGINE = SystemClusters()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.collations\n(\n `name` String,\n `language` Nullable(String)\n)\nENGINE = SystemTableCollations()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.columns\n(\n `database` String,\n `table` String,\n `name` String,\n `type` String,\n `position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `comment` String,\n `is_in_partition_key` UInt8,\n `is_in_sorting_key` UInt8,\n `is_in_primary_key` UInt8,\n `is_in_sampling_key` UInt8,\n `compression_codec` String,\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64)\n)\nENGINE = SystemColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.contributors\n(\n `name` String\n)\nENGINE = SystemContributors()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.current_roles\n(\n `role_name` String,\n `with_admin_option` UInt8,\n `is_default` UInt8\n)\nENGINE = SystemCurrentRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.data_skipping_indices\n(\n `database` String,\n `table` String,\n `name` String,\n `type` String,\n `expr` String,\n `granularity` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks` UInt64\n)\nENGINE = SystemDataSkippingIndices()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.data_type_families\n(\n `name` String,\n `case_insensitive` UInt8,\n `alias_to` String\n)\nENGINE = SystemTableDataTypeFamilies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.databases\n(\n `name` String,\n `engine` String,\n `data_path` String,\n `metadata_path` String,\n `uuid` UUID,\n `comment` String,\n `database` String\n)\nENGINE = SystemDatabases()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.detached_parts\n(\n `database` String,\n `table` String,\n `partition_id` Nullable(String),\n `name` String,\n `disk` String,\n `reason` Nullable(String),\n `min_block_number` Nullable(Int64),\n `max_block_number` Nullable(Int64),\n `level` Nullable(UInt32)\n)\nENGINE = SystemDetachedParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.dictionaries\n(\n `database` String,\n `name` String,\n `uuid` UUID,\n `status` Enum8(\'NOT_LOADED\' = 0, \'LOADED\' = 1, \'FAILED\' = 2, \'LOADING\' = 3, \'FAILED_AND_RELOADING\' = 4, \'LOADED_AND_RELOADING\' = 5, \'NOT_EXIST\' = 6),\n `origin` String,\n `type` String,\n `key.names` Array(String),\n `key.types` Array(String),\n `attribute.names` Array(String),\n `attribute.types` Array(String),\n `bytes_allocated` UInt64,\n `query_count` UInt64,\n `hit_rate` Float64,\n `found_rate` Float64,\n `element_count` UInt64,\n `load_factor` Float64,\n `source` String,\n `lifetime_min` UInt64,\n `lifetime_max` UInt64,\n `loading_start_time` DateTime,\n `last_successful_update_time` DateTime,\n `loading_duration` Float32,\n `last_exception` String,\n `comment` String\n)\nENGINE = SystemDictionaries()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.disks\n(\n `name` String,\n `path` String,\n `free_space` UInt64,\n `total_space` UInt64,\n `keep_free_space` UInt64,\n `type` String,\n `cache_path` String\n)\nENGINE = SystemDisks()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.distributed_ddl_queue\n(\n `entry` String,\n `entry_version` Nullable(UInt8),\n `initiator_host` Nullable(String),\n `initiator_port` Nullable(UInt16),\n `cluster` String,\n `query` String,\n `settings` Map(String, String),\n `query_create_time` DateTime,\n `host` Nullable(String),\n `port` Nullable(UInt16),\n `status` Nullable(Enum8(\'Inactive\' = 0, \'Active\' = 1, \'Finished\' = 2, \'Removing\' = 3, \'Unknown\' = 4)),\n `exception_code` Nullable(UInt16),\n `exception_text` Nullable(String),\n `query_finish_time` Nullable(DateTime),\n `query_duration_ms` Nullable(UInt64)\n)\nENGINE = SystemDDLWorkerQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.distribution_queue\n(\n `database` String,\n `table` String,\n `data_path` String,\n `is_blocked` UInt8,\n `error_count` UInt64,\n `data_files` UInt64,\n `data_compressed_bytes` UInt64,\n `broken_data_files` UInt64,\n `broken_data_compressed_bytes` UInt64,\n `last_exception` String\n)\nENGINE = SystemDistributionQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.enabled_roles\n(\n `role_name` String,\n `with_admin_option` UInt8,\n `is_current` UInt8,\n `is_default` UInt8\n)\nENGINE = SystemEnabledRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.errors\n(\n `name` String,\n `code` Int32,\n `value` UInt64,\n `last_error_time` DateTime,\n `last_error_message` String,\n `last_error_trace` Array(UInt64),\n `remote` UInt8\n)\nENGINE = SystemErrors()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String\n)\nENGINE = SystemEvents()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'KILL TRANSACTION\' = 62, \'MOVE PARTITION BETWEEN SHARDS\' = 63, \'CREATE USER\' = 64, \'ALTER USER\' = 65, \'DROP USER\' = 66, \'CREATE ROLE\' = 67, \'ALTER ROLE\' = 68, \'DROP ROLE\' = 69, \'ROLE ADMIN\' = 70, \'CREATE ROW POLICY\' = 71, \'ALTER ROW POLICY\' = 72, \'DROP ROW POLICY\' = 73, \'CREATE QUOTA\' = 74, \'ALTER QUOTA\' = 75, \'DROP QUOTA\' = 76, \'CREATE SETTINGS PROFILE\' = 77, \'ALTER SETTINGS PROFILE\' = 78, \'DROP SETTINGS PROFILE\' = 79, \'SHOW USERS\' = 80, \'SHOW ROLES\' = 81, \'SHOW ROW POLICIES\' = 82, \'SHOW QUOTAS\' = 83, \'SHOW SETTINGS PROFILES\' = 84, \'SHOW ACCESS\' = 85, \'ACCESS MANAGEMENT\' = 86, \'SYSTEM SHUTDOWN\' = 87, \'SYSTEM DROP DNS CACHE\' = 88, \'SYSTEM DROP MARK CACHE\' = 89, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 90, \'SYSTEM DROP MMAP CACHE\' = 91, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 92, \'SYSTEM DROP CACHE\' = 93, \'SYSTEM RELOAD CONFIG\' = 94, \'SYSTEM RELOAD SYMBOLS\' = 95, \'SYSTEM RELOAD DICTIONARY\' = 96, \'SYSTEM RELOAD MODEL\' = 97, \'SYSTEM RELOAD FUNCTION\' = 98, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 99, \'SYSTEM RELOAD\' = 100, \'SYSTEM RESTART DISK\' = 101, \'SYSTEM MERGES\' = 102, \'SYSTEM TTL MERGES\' = 103, \'SYSTEM FETCHES\' = 104, \'SYSTEM MOVES\' = 105, \'SYSTEM DISTRIBUTED SENDS\' = 106, \'SYSTEM REPLICATED SENDS\' = 107, \'SYSTEM SENDS\' = 108, \'SYSTEM REPLICATION QUEUES\' = 109, \'SYSTEM DROP REPLICA\' = 110, \'SYSTEM SYNC REPLICA\' = 111, \'SYSTEM RESTART REPLICA\' = 112, \'SYSTEM RESTORE REPLICA\' = 113, \'SYSTEM FLUSH DISTRIBUTED\' = 114, \'SYSTEM FLUSH LOGS\' = 115, \'SYSTEM FLUSH\' = 116, \'SYSTEM THREAD FUZZER\' = 117, \'SYSTEM\' = 118, \'dictGet\' = 119, \'addressToLine\' = 120, \'addressToLineWithInlines\' = 121, \'addressToSymbol\' = 122, \'demangle\' = 123, \'INTROSPECTION\' = 124, \'FILE\' = 125, \'URL\' = 126, \'REMOTE\' = 127, \'MONGO\' = 128, \'MYSQL\' = 129, \'POSTGRES\' = 130, \'SQLITE\' = 131, \'ODBC\' = 132, \'JDBC\' = 133, \'HDFS\' = 134, \'S3\' = 135, \'HIVE\' = 136, \'SOURCES\' = 137, \'ALL\' = 138, \'NONE\' = 139),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `rule_type` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.merges\n(\n `database` String,\n `table` String,\n `elapsed` Float64,\n `progress` Float64,\n `num_parts` UInt64,\n `source_part_names` Array(String),\n `result_part_name` String,\n `source_part_paths` Array(String),\n `result_part_path` String,\n `partition_id` String,\n `is_mutation` UInt8,\n `total_size_bytes_compressed` UInt64,\n `total_size_marks` UInt64,\n `bytes_read_uncompressed` UInt64,\n `rows_read` UInt64,\n `bytes_written_uncompressed` UInt64,\n `rows_written` UInt64,\n `columns_written` UInt64,\n `memory_usage` UInt64,\n `thread_id` UInt64,\n `merge_type` String,\n `merge_algorithm` String\n)\nENGINE = SystemMerges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.metrics\n(\n `metric` String,\n `value` Int64,\n `description` String\n)\nENGINE = SystemMetrics()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.models\n(\n `name` String,\n `status` Enum8(\'NOT_LOADED\' = 0, \'LOADED\' = 1, \'FAILED\' = 2, \'LOADING\' = 3, \'FAILED_AND_RELOADING\' = 4, \'LOADED_AND_RELOADING\' = 5, \'NOT_EXIST\' = 6),\n `origin` String,\n `type` String,\n `loading_start_time` DateTime,\n `loading_duration` Float32,\n `last_exception` String\n)\nENGINE = SystemModels()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.mutations\n(\n `database` String,\n `table` String,\n `mutation_id` String,\n `command` String,\n `create_time` DateTime,\n `block_numbers.partition_id` Array(String),\n `block_numbers.number` Array(Int64),\n `parts_to_do_names` Array(String),\n `parts_to_do` Int64,\n `is_done` UInt8,\n `latest_failed_part` String,\n `latest_fail_time` DateTime,\n `latest_fail_reason` String\n)\nENGINE = SystemMutations()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.numbers\n(\n `number` UInt64\n)\nENGINE = SystemNumbers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.numbers_mt\n(\n `number` UInt64\n)\nENGINE = SystemNumbers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `visible` UInt8,\n `creation_tid` Tuple(UInt64, UInt64, UUID),\n `removal_tid` Tuple(UInt64, UInt64, UUID),\n `creation_csn` UInt64,\n `removal_csn` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `serialization_kind` String,\n `subcolumns.names` Array(String),\n `subcolumns.types` Array(String),\n `subcolumns.serializations` Array(String),\n `subcolumns.bytes_on_disk` Array(UInt64),\n `subcolumns.data_compressed_bytes` Array(UInt64),\n `subcolumns.data_uncompressed_bytes` Array(UInt64),\n `subcolumns.marks_bytes` Array(UInt64),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.privileges\n(\n `privilege` Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'KILL TRANSACTION\' = 62, \'MOVE PARTITION BETWEEN SHARDS\' = 63, \'CREATE USER\' = 64, \'ALTER USER\' = 65, \'DROP USER\' = 66, \'CREATE ROLE\' = 67, \'ALTER ROLE\' = 68, \'DROP ROLE\' = 69, \'ROLE ADMIN\' = 70, \'CREATE ROW POLICY\' = 71, \'ALTER ROW POLICY\' = 72, \'DROP ROW POLICY\' = 73, \'CREATE QUOTA\' = 74, \'ALTER QUOTA\' = 75, \'DROP QUOTA\' = 76, \'CREATE SETTINGS PROFILE\' = 77, \'ALTER SETTINGS PROFILE\' = 78, \'DROP SETTINGS PROFILE\' = 79, \'SHOW USERS\' = 80, \'SHOW ROLES\' = 81, \'SHOW ROW POLICIES\' = 82, \'SHOW QUOTAS\' = 83, \'SHOW SETTINGS PROFILES\' = 84, \'SHOW ACCESS\' = 85, \'ACCESS MANAGEMENT\' = 86, \'SYSTEM SHUTDOWN\' = 87, \'SYSTEM DROP DNS CACHE\' = 88, \'SYSTEM DROP MARK CACHE\' = 89, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 90, \'SYSTEM DROP MMAP CACHE\' = 91, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 92, \'SYSTEM DROP CACHE\' = 93, \'SYSTEM RELOAD CONFIG\' = 94, \'SYSTEM RELOAD SYMBOLS\' = 95, \'SYSTEM RELOAD DICTIONARY\' = 96, \'SYSTEM RELOAD MODEL\' = 97, \'SYSTEM RELOAD FUNCTION\' = 98, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 99, \'SYSTEM RELOAD\' = 100, \'SYSTEM RESTART DISK\' = 101, \'SYSTEM MERGES\' = 102, \'SYSTEM TTL MERGES\' = 103, \'SYSTEM FETCHES\' = 104, \'SYSTEM MOVES\' = 105, \'SYSTEM DISTRIBUTED SENDS\' = 106, \'SYSTEM REPLICATED SENDS\' = 107, \'SYSTEM SENDS\' = 108, \'SYSTEM REPLICATION QUEUES\' = 109, \'SYSTEM DROP REPLICA\' = 110, \'SYSTEM SYNC REPLICA\' = 111, \'SYSTEM RESTART REPLICA\' = 112, \'SYSTEM RESTORE REPLICA\' = 113, \'SYSTEM FLUSH DISTRIBUTED\' = 114, \'SYSTEM FLUSH LOGS\' = 115, \'SYSTEM FLUSH\' = 116, \'SYSTEM THREAD FUZZER\' = 117, \'SYSTEM\' = 118, \'dictGet\' = 119, \'addressToLine\' = 120, \'addressToLineWithInlines\' = 121, \'addressToSymbol\' = 122, \'demangle\' = 123, \'INTROSPECTION\' = 124, \'FILE\' = 125, \'URL\' = 126, \'REMOTE\' = 127, \'MONGO\' = 128, \'MYSQL\' = 129, \'POSTGRES\' = 130, \'SQLITE\' = 131, \'ODBC\' = 132, \'JDBC\' = 133, \'HDFS\' = 134, \'S3\' = 135, \'HIVE\' = 136, \'SOURCES\' = 137, \'ALL\' = 138, \'NONE\' = 139),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum16(\'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'KILL TRANSACTION\' = 62, \'MOVE PARTITION BETWEEN SHARDS\' = 63, \'CREATE USER\' = 64, \'ALTER USER\' = 65, \'DROP USER\' = 66, \'CREATE ROLE\' = 67, \'ALTER ROLE\' = 68, \'DROP ROLE\' = 69, \'ROLE ADMIN\' = 70, \'CREATE ROW POLICY\' = 71, \'ALTER ROW POLICY\' = 72, \'DROP ROW POLICY\' = 73, \'CREATE QUOTA\' = 74, \'ALTER QUOTA\' = 75, \'DROP QUOTA\' = 76, \'CREATE SETTINGS PROFILE\' = 77, \'ALTER SETTINGS PROFILE\' = 78, \'DROP SETTINGS PROFILE\' = 79, \'SHOW USERS\' = 80, \'SHOW ROLES\' = 81, \'SHOW ROW POLICIES\' = 82, \'SHOW QUOTAS\' = 83, \'SHOW SETTINGS PROFILES\' = 84, \'SHOW ACCESS\' = 85, \'ACCESS MANAGEMENT\' = 86, \'SYSTEM SHUTDOWN\' = 87, \'SYSTEM DROP DNS CACHE\' = 88, \'SYSTEM DROP MARK CACHE\' = 89, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 90, \'SYSTEM DROP MMAP CACHE\' = 91, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 92, \'SYSTEM DROP CACHE\' = 93, \'SYSTEM RELOAD CONFIG\' = 94, \'SYSTEM RELOAD SYMBOLS\' = 95, \'SYSTEM RELOAD DICTIONARY\' = 96, \'SYSTEM RELOAD MODEL\' = 97, \'SYSTEM RELOAD FUNCTION\' = 98, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 99, \'SYSTEM RELOAD\' = 100, \'SYSTEM RESTART DISK\' = 101, \'SYSTEM MERGES\' = 102, \'SYSTEM TTL MERGES\' = 103, \'SYSTEM FETCHES\' = 104, \'SYSTEM MOVES\' = 105, \'SYSTEM DISTRIBUTED SENDS\' = 106, \'SYSTEM REPLICATED SENDS\' = 107, \'SYSTEM SENDS\' = 108, \'SYSTEM REPLICATION QUEUES\' = 109, \'SYSTEM DROP REPLICA\' = 110, \'SYSTEM SYNC REPLICA\' = 111, \'SYSTEM RESTART REPLICA\' = 112, \'SYSTEM RESTORE REPLICA\' = 113, \'SYSTEM FLUSH DISTRIBUTED\' = 114, \'SYSTEM FLUSH LOGS\' = 115, \'SYSTEM FLUSH\' = 116, \'SYSTEM THREAD FUZZER\' = 117, \'SYSTEM\' = 118, \'dictGet\' = 119, \'addressToLine\' = 120, \'addressToLineWithInlines\' = 121, \'addressToSymbol\' = 122, \'demangle\' = 123, \'INTROSPECTION\' = 124, \'FILE\' = 125, \'URL\' = 126, \'REMOTE\' = 127, \'MONGO\' = 128, \'MYSQL\' = 129, \'POSTGRES\' = 130, \'SQLITE\' = 131, \'ODBC\' = 132, \'JDBC\' = 133, \'HDFS\' = 134, \'S3\' = 135, \'HIVE\' = 136, \'SOURCES\' = 137, \'ALL\' = 138, \'NONE\' = 139))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `distributed_depth` UInt64,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.projection_parts\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.projection_parts_columns\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.quota_limits\n(\n `quota_name` String,\n `duration` UInt32,\n `is_randomized_interval` UInt8,\n `max_queries` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `max_execution_time` Nullable(Float64),\n `max_written_bytes` Nullable(UInt64)\n)\nENGINE = SystemQuotaLimits()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.quota_usage\n(\n `quota_name` String,\n `quota_key` String,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64),\n `written_bytes` Nullable(UInt64),\n `max_written_bytes` Nullable(UInt64)\n)\nENGINE = SystemQuotaUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.quotas\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `keys` Array(Enum8(\'user_name\' = 1, \'ip_address\' = 2, \'forwarded_ip_address\' = 3, \'client_key\' = 4)),\n `durations` Array(UInt32),\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemQuotas()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.quotas_usage\n(\n `quota_name` String,\n `quota_key` String,\n `is_current` UInt8,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64),\n `written_bytes` Nullable(UInt64),\n `max_written_bytes` Nullable(UInt64)\n)\nENGINE = SystemQuotasUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.replicas\n(\n `database` String,\n `table` String,\n `engine` String,\n `is_leader` UInt8,\n `can_become_leader` UInt8,\n `is_readonly` UInt8,\n `is_session_expired` UInt8,\n `future_parts` UInt32,\n `parts_to_check` UInt32,\n `zookeeper_path` String,\n `replica_name` String,\n `replica_path` String,\n `columns_version` Int32,\n `queue_size` UInt32,\n `inserts_in_queue` UInt32,\n `merges_in_queue` UInt32,\n `part_mutations_in_queue` UInt32,\n `queue_oldest_time` DateTime,\n `inserts_oldest_time` DateTime,\n `merges_oldest_time` DateTime,\n `part_mutations_oldest_time` DateTime,\n `oldest_part_to_get` String,\n `oldest_part_to_merge_to` String,\n `oldest_part_to_mutate_to` String,\n `log_max_index` UInt64,\n `log_pointer` UInt64,\n `last_queue_update` DateTime,\n `absolute_delay` UInt64,\n `total_replicas` UInt8,\n `active_replicas` UInt8,\n `last_queue_update_exception` String,\n `zookeeper_exception` String,\n `replica_is_active` Map(String, UInt8)\n)\nENGINE = SystemReplicas()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.replicated_fetches\n(\n `database` String,\n `table` String,\n `elapsed` Float64,\n `progress` Float64,\n `result_part_name` String,\n `result_part_path` String,\n `partition_id` String,\n `total_size_bytes_compressed` UInt64,\n `bytes_read_compressed` UInt64,\n `source_replica_path` String,\n `source_replica_hostname` String,\n `source_replica_port` UInt16,\n `interserver_scheme` String,\n `URI` String,\n `to_detached` UInt8,\n `thread_id` UInt64\n)\nENGINE = SystemReplicatedFetches()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.replicated_merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemReplicatedMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.replication_queue\n(\n `database` String,\n `table` String,\n `replica_name` String,\n `position` UInt32,\n `node_name` String,\n `type` String,\n `create_time` DateTime,\n `required_quorum` UInt32,\n `source_replica` String,\n `new_part_name` String,\n `parts_to_merge` Array(String),\n `is_detach` UInt8,\n `is_currently_executing` UInt8,\n `num_tries` UInt32,\n `last_exception` String,\n `last_attempt_time` DateTime,\n `num_postponed` UInt32,\n `postpone_reason` String,\n `last_postpone_time` DateTime,\n `merge_type` String\n)\nENGINE = SystemReplicationQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.role_grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `granted_role_name` String,\n `granted_role_is_default` UInt8,\n `with_admin_option` UInt8\n)\nENGINE = SystemRoleGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.roles\n(\n `name` String,\n `id` UUID,\n `storage` String\n)\nENGINE = SystemRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.row_policies\n(\n `name` String,\n `short_name` String,\n `database` String,\n `table` String,\n `id` UUID,\n `storage` String,\n `select_filter` Nullable(String),\n `is_restrictive` UInt8,\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemRowPolicies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `min` Nullable(String),\n `max` Nullable(String),\n `readonly` UInt8,\n `type` String\n)\nENGINE = SystemSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.settings_profile_elements\n(\n `profile_name` Nullable(String),\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `index` UInt64,\n `setting_name` Nullable(String),\n `value` Nullable(String),\n `min` Nullable(String),\n `max` Nullable(String),\n `readonly` Nullable(UInt8),\n `inherit_profile` Nullable(String)\n)\nENGINE = SystemSettingsProfileElements()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.settings_profiles\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `num_elements` UInt64,\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemSettingsProfiles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.stack_trace\n(\n `thread_name` String,\n `thread_id` UInt64,\n `query_id` String,\n `trace` Array(UInt64)\n)\nENGINE = SystemStackTrace()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.storage_policies\n(\n `policy_name` String,\n `volume_name` String,\n `volume_priority` UInt64,\n `disks` Array(String),\n `volume_type` String,\n `max_data_part_size` UInt64,\n `move_factor` Float32,\n `prefer_not_to_merge` UInt8\n)\nENGINE = SystemStoragePolicies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.table_engines\n(\n `name` String,\n `supports_settings` UInt8,\n `supports_skipping_indices` UInt8,\n `supports_projections` UInt8,\n `supports_sort_order` UInt8,\n `supports_ttl` UInt8,\n `supports_replication` UInt8,\n `supports_deduplication` UInt8,\n `supports_parallel_insert` UInt8\n)\nENGINE = SystemTableEngines()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.table_functions\n(\n `name` String\n)\nENGINE = SystemTableFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.tables\n(\n `database` String,\n `name` String,\n `uuid` UUID,\n `engine` String,\n `is_temporary` UInt8,\n `data_paths` Array(String),\n `metadata_path` String,\n `metadata_modification_time` DateTime,\n `dependencies_database` Array(String),\n `dependencies_table` Array(String),\n `create_table_query` String,\n `engine_full` String,\n `as_select` String,\n `partition_key` String,\n `sorting_key` String,\n `primary_key` String,\n `sampling_key` String,\n `storage_policy` String,\n `total_rows` Nullable(UInt64),\n `total_bytes` Nullable(UInt64),\n `lifetime_rows` Nullable(UInt64),\n `lifetime_bytes` Nullable(UInt64),\n `comment` String,\n `has_own_data` UInt8,\n `loading_dependencies_database` Array(String),\n `loading_dependencies_table` Array(String),\n `loading_dependent_database` Array(String),\n `loading_dependent_table` Array(String),\n `table` String\n)\nENGINE = SystemTables()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.time_zones\n(\n `time_zone` String\n)\nENGINE = SystemTimeZones()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.user_directories\n(\n `name` String,\n `type` String,\n `params` String,\n `precedence` UInt64\n)\nENGINE = SystemUserDirectories()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.users\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `auth_type` Enum8(\'no_password\' = 0, \'plaintext_password\' = 1, \'sha256_password\' = 2, \'double_sha1_password\' = 3, \'ldap\' = 4, \'kerberos\' = 5, \'ssl_certificate\' = 6),\n `auth_params` String,\n `host_ip` Array(String),\n `host_names` Array(String),\n `host_names_regexp` Array(String),\n `host_names_like` Array(String),\n `default_roles_all` UInt8,\n `default_roles_list` Array(String),\n `default_roles_except` Array(String),\n `grantees_any` UInt8,\n `grantees_list` Array(String),\n `grantees_except` Array(String),\n `default_database` String\n)\nENGINE = SystemUsers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.warnings\n(\n `message` String\n)\nENGINE = SystemWarnings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.zeros\n(\n `zero` UInt8\n)\nENGINE = SystemZeros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.zeros_mt\n(\n `zero` UInt8\n)\nENGINE = SystemZeros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.aggregate_function_combinators +( + `name` String, + `is_internal` UInt8 +) +ENGINE = SystemAggregateFunctionCombinators() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.asynchronous_inserts +( + `query` String, + `database` String, + `table` String, + `format` String, + `first_update` DateTime64(6), + `last_update` DateTime64(6), + `total_bytes` UInt64, + `entries.query_id` Array(String), + `entries.bytes` Array(UInt64), + `entries.finished` Array(UInt8), + `entries.exception` Array(String) +) +ENGINE = SystemAsynchronousInserts() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.asynchronous_metrics +( + `metric` String, + `value` Float64 +) +ENGINE = SystemAsynchronousMetrics() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.build_options +( + `name` String, + `value` String +) +ENGINE = SystemBuildOptions() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.clusters +( + `cluster` String, + `shard_num` UInt32, + `shard_weight` UInt32, + `replica_num` UInt32, + `host_name` String, + `host_address` String, + `port` UInt16, + `is_local` UInt8, + `user` String, + `default_database` String, + `errors_count` UInt32, + `slowdowns_count` UInt32, + `estimated_recovery_time` UInt32 +) +ENGINE = SystemClusters() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.collations +( + `name` String, + `language` Nullable(String) +) +ENGINE = SystemTableCollations() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.columns +( + `database` String, + `table` String, + `name` String, + `type` String, + `position` UInt64, + `default_kind` String, + `default_expression` String, + `data_compressed_bytes` UInt64, + `data_uncompressed_bytes` UInt64, + `marks_bytes` UInt64, + `comment` String, + `is_in_partition_key` UInt8, + `is_in_sorting_key` UInt8, + `is_in_primary_key` UInt8, + `is_in_sampling_key` UInt8, + `compression_codec` String, + `character_octet_length` Nullable(UInt64), + `numeric_precision` Nullable(UInt64), + `numeric_precision_radix` Nullable(UInt64), + `numeric_scale` Nullable(UInt64), + `datetime_precision` Nullable(UInt64) +) +ENGINE = SystemColumns() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.contributors +( + `name` String +) +ENGINE = SystemContributors() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.current_roles +( + `role_name` String, + `with_admin_option` UInt8, + `is_default` UInt8 +) +ENGINE = SystemCurrentRoles() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.data_skipping_indices +( + `database` String, + `table` String, + `name` String, + `type` String, + `expr` String, + `granularity` UInt64, + `data_compressed_bytes` UInt64, + `data_uncompressed_bytes` UInt64, + `marks` UInt64 +) +ENGINE = SystemDataSkippingIndices() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.data_type_families +( + `name` String, + `case_insensitive` UInt8, + `alias_to` String +) +ENGINE = SystemTableDataTypeFamilies() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.databases +( + `name` String, + `engine` String, + `data_path` String, + `metadata_path` String, + `uuid` UUID, + `comment` String, + `database` String +) +ENGINE = SystemDatabases() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.detached_parts +( + `database` String, + `table` String, + `partition_id` Nullable(String), + `name` String, + `disk` String, + `reason` Nullable(String), + `min_block_number` Nullable(Int64), + `max_block_number` Nullable(Int64), + `level` Nullable(UInt32) +) +ENGINE = SystemDetachedParts() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.dictionaries +( + `database` String, + `name` String, + `uuid` UUID, + `status` Enum8('NOT_LOADED' = 0, 'LOADED' = 1, 'FAILED' = 2, 'LOADING' = 3, 'FAILED_AND_RELOADING' = 4, 'LOADED_AND_RELOADING' = 5, 'NOT_EXIST' = 6), + `origin` String, + `type` String, + `key.names` Array(String), + `key.types` Array(String), + `attribute.names` Array(String), + `attribute.types` Array(String), + `bytes_allocated` UInt64, + `query_count` UInt64, + `hit_rate` Float64, + `found_rate` Float64, + `element_count` UInt64, + `load_factor` Float64, + `source` String, + `lifetime_min` UInt64, + `lifetime_max` UInt64, + `loading_start_time` DateTime, + `last_successful_update_time` DateTime, + `loading_duration` Float32, + `last_exception` String, + `comment` String +) +ENGINE = SystemDictionaries() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.disks +( + `name` String, + `path` String, + `free_space` UInt64, + `total_space` UInt64, + `keep_free_space` UInt64, + `type` String, + `cache_path` String +) +ENGINE = SystemDisks() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.distributed_ddl_queue +( + `entry` String, + `entry_version` Nullable(UInt8), + `initiator_host` Nullable(String), + `initiator_port` Nullable(UInt16), + `cluster` String, + `query` String, + `settings` Map(String, String), + `query_create_time` DateTime, + `host` Nullable(String), + `port` Nullable(UInt16), + `status` Nullable(Enum8('Inactive' = 0, 'Active' = 1, 'Finished' = 2, 'Removing' = 3, 'Unknown' = 4)), + `exception_code` Nullable(UInt16), + `exception_text` Nullable(String), + `query_finish_time` Nullable(DateTime), + `query_duration_ms` Nullable(UInt64) +) +ENGINE = SystemDDLWorkerQueue() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.distribution_queue +( + `database` String, + `table` String, + `data_path` String, + `is_blocked` UInt8, + `error_count` UInt64, + `data_files` UInt64, + `data_compressed_bytes` UInt64, + `broken_data_files` UInt64, + `broken_data_compressed_bytes` UInt64, + `last_exception` String +) +ENGINE = SystemDistributionQueue() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.enabled_roles +( + `role_name` String, + `with_admin_option` UInt8, + `is_current` UInt8, + `is_default` UInt8 +) +ENGINE = SystemEnabledRoles() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.errors +( + `name` String, + `code` Int32, + `value` UInt64, + `last_error_time` DateTime, + `last_error_message` String, + `last_error_trace` Array(UInt64), + `remote` UInt8 +) +ENGINE = SystemErrors() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.events +( + `event` String, + `value` UInt64, + `description` String +) +ENGINE = SystemEvents() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.formats +( + `name` String, + `is_input` UInt8, + `is_output` UInt8 +) +ENGINE = SystemFormats() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.functions +( + `name` String, + `is_aggregate` UInt8, + `case_insensitive` UInt8, + `alias_to` String, + `create_query` String, + `origin` Enum8('System' = 0, 'SQLUserDefined' = 1, 'ExecutableUserDefined' = 2) +) +ENGINE = SystemFunctions() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.grants +( + `user_name` Nullable(String), + `role_name` Nullable(String), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SELECT' = 5, 'INSERT' = 6, 'ALTER UPDATE' = 7, 'ALTER DELETE' = 8, 'ALTER ADD COLUMN' = 9, 'ALTER MODIFY COLUMN' = 10, 'ALTER DROP COLUMN' = 11, 'ALTER COMMENT COLUMN' = 12, 'ALTER CLEAR COLUMN' = 13, 'ALTER RENAME COLUMN' = 14, 'ALTER MATERIALIZE COLUMN' = 15, 'ALTER COLUMN' = 16, 'ALTER MODIFY COMMENT' = 17, 'ALTER ORDER BY' = 18, 'ALTER SAMPLE BY' = 19, 'ALTER ADD INDEX' = 20, 'ALTER DROP INDEX' = 21, 'ALTER MATERIALIZE INDEX' = 22, 'ALTER CLEAR INDEX' = 23, 'ALTER INDEX' = 24, 'ALTER ADD PROJECTION' = 25, 'ALTER DROP PROJECTION' = 26, 'ALTER MATERIALIZE PROJECTION' = 27, 'ALTER CLEAR PROJECTION' = 28, 'ALTER PROJECTION' = 29, 'ALTER ADD CONSTRAINT' = 30, 'ALTER DROP CONSTRAINT' = 31, 'ALTER CONSTRAINT' = 32, 'ALTER TTL' = 33, 'ALTER MATERIALIZE TTL' = 34, 'ALTER SETTINGS' = 35, 'ALTER MOVE PARTITION' = 36, 'ALTER FETCH PARTITION' = 37, 'ALTER FREEZE PARTITION' = 38, 'ALTER DATABASE SETTINGS' = 39, 'ALTER TABLE' = 40, 'ALTER DATABASE' = 41, 'ALTER VIEW REFRESH' = 42, 'ALTER VIEW MODIFY QUERY' = 43, 'ALTER VIEW' = 44, 'ALTER' = 45, 'CREATE DATABASE' = 46, 'CREATE TABLE' = 47, 'CREATE VIEW' = 48, 'CREATE DICTIONARY' = 49, 'CREATE TEMPORARY TABLE' = 50, 'CREATE FUNCTION' = 51, 'CREATE' = 52, 'DROP DATABASE' = 53, 'DROP TABLE' = 54, 'DROP VIEW' = 55, 'DROP DICTIONARY' = 56, 'DROP FUNCTION' = 57, 'DROP' = 58, 'TRUNCATE' = 59, 'OPTIMIZE' = 60, 'KILL QUERY' = 61, 'KILL TRANSACTION' = 62, 'MOVE PARTITION BETWEEN SHARDS' = 63, 'CREATE USER' = 64, 'ALTER USER' = 65, 'DROP USER' = 66, 'CREATE ROLE' = 67, 'ALTER ROLE' = 68, 'DROP ROLE' = 69, 'ROLE ADMIN' = 70, 'CREATE ROW POLICY' = 71, 'ALTER ROW POLICY' = 72, 'DROP ROW POLICY' = 73, 'CREATE QUOTA' = 74, 'ALTER QUOTA' = 75, 'DROP QUOTA' = 76, 'CREATE SETTINGS PROFILE' = 77, 'ALTER SETTINGS PROFILE' = 78, 'DROP SETTINGS PROFILE' = 79, 'SHOW USERS' = 80, 'SHOW ROLES' = 81, 'SHOW ROW POLICIES' = 82, 'SHOW QUOTAS' = 83, 'SHOW SETTINGS PROFILES' = 84, 'SHOW ACCESS' = 85, 'ACCESS MANAGEMENT' = 86, 'SYSTEM SHUTDOWN' = 87, 'SYSTEM DROP DNS CACHE' = 88, 'SYSTEM DROP MARK CACHE' = 89, 'SYSTEM DROP UNCOMPRESSED CACHE' = 90, 'SYSTEM DROP MMAP CACHE' = 91, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 92, 'SYSTEM DROP CACHE' = 93, 'SYSTEM RELOAD CONFIG' = 94, 'SYSTEM RELOAD SYMBOLS' = 95, 'SYSTEM RELOAD DICTIONARY' = 96, 'SYSTEM RELOAD MODEL' = 97, 'SYSTEM RELOAD FUNCTION' = 98, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 99, 'SYSTEM RELOAD' = 100, 'SYSTEM RESTART DISK' = 101, 'SYSTEM MERGES' = 102, 'SYSTEM TTL MERGES' = 103, 'SYSTEM FETCHES' = 104, 'SYSTEM MOVES' = 105, 'SYSTEM DISTRIBUTED SENDS' = 106, 'SYSTEM REPLICATED SENDS' = 107, 'SYSTEM SENDS' = 108, 'SYSTEM REPLICATION QUEUES' = 109, 'SYSTEM DROP REPLICA' = 110, 'SYSTEM SYNC REPLICA' = 111, 'SYSTEM RESTART REPLICA' = 112, 'SYSTEM RESTORE REPLICA' = 113, 'SYSTEM FLUSH DISTRIBUTED' = 114, 'SYSTEM FLUSH LOGS' = 115, 'SYSTEM FLUSH' = 116, 'SYSTEM THREAD FUZZER' = 117, 'SYSTEM' = 118, 'dictGet' = 119, 'addressToLine' = 120, 'addressToLineWithInlines' = 121, 'addressToSymbol' = 122, 'demangle' = 123, 'INTROSPECTION' = 124, 'FILE' = 125, 'URL' = 126, 'REMOTE' = 127, 'MONGO' = 128, 'MYSQL' = 129, 'POSTGRES' = 130, 'SQLITE' = 131, 'ODBC' = 132, 'JDBC' = 133, 'HDFS' = 134, 'S3' = 135, 'HIVE' = 136, 'SOURCES' = 137, 'ALL' = 138, 'NONE' = 139), + `database` Nullable(String), + `table` Nullable(String), + `column` Nullable(String), + `is_partial_revoke` UInt8, + `grant_option` UInt8 +) +ENGINE = SystemGrants() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.graphite_retentions +( + `config_name` String, + `rule_type` String, + `regexp` String, + `function` String, + `age` UInt64, + `precision` UInt64, + `priority` UInt16, + `is_default` UInt8, + `Tables.database` Array(String), + `Tables.table` Array(String) +) +ENGINE = SystemGraphite() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.licenses +( + `library_name` String, + `license_type` String, + `license_path` String, + `license_text` String +) +ENGINE = SystemLicenses() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.macros +( + `macro` String, + `substitution` String +) +ENGINE = SystemMacros() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.merge_tree_settings +( + `name` String, + `value` String, + `changed` UInt8, + `description` String, + `type` String +) +ENGINE = SystemMergeTreeSettings() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.merges +( + `database` String, + `table` String, + `elapsed` Float64, + `progress` Float64, + `num_parts` UInt64, + `source_part_names` Array(String), + `result_part_name` String, + `source_part_paths` Array(String), + `result_part_path` String, + `partition_id` String, + `is_mutation` UInt8, + `total_size_bytes_compressed` UInt64, + `total_size_marks` UInt64, + `bytes_read_uncompressed` UInt64, + `rows_read` UInt64, + `bytes_written_uncompressed` UInt64, + `rows_written` UInt64, + `columns_written` UInt64, + `memory_usage` UInt64, + `thread_id` UInt64, + `merge_type` String, + `merge_algorithm` String +) +ENGINE = SystemMerges() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.metrics +( + `metric` String, + `value` Int64, + `description` String +) +ENGINE = SystemMetrics() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.models +( + `name` String, + `status` Enum8('NOT_LOADED' = 0, 'LOADED' = 1, 'FAILED' = 2, 'LOADING' = 3, 'FAILED_AND_RELOADING' = 4, 'LOADED_AND_RELOADING' = 5, 'NOT_EXIST' = 6), + `origin` String, + `type` String, + `loading_start_time` DateTime, + `loading_duration` Float32, + `last_exception` String +) +ENGINE = SystemModels() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.mutations +( + `database` String, + `table` String, + `mutation_id` String, + `command` String, + `create_time` DateTime, + `block_numbers.partition_id` Array(String), + `block_numbers.number` Array(Int64), + `parts_to_do_names` Array(String), + `parts_to_do` Int64, + `is_done` UInt8, + `latest_failed_part` String, + `latest_fail_time` DateTime, + `latest_fail_reason` String +) +ENGINE = SystemMutations() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.numbers +( + `number` UInt64 +) +ENGINE = SystemNumbers() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.numbers_mt +( + `number` UInt64 +) +ENGINE = SystemNumbers() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.one +( + `dummy` UInt8 +) +ENGINE = SystemOne() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.part_moves_between_shards +( + `database` String, + `table` String, + `task_name` String, + `task_uuid` UUID, + `create_time` DateTime, + `part_name` String, + `part_uuid` UUID, + `to_shard` String, + `dst_part_name` String, + `update_time` DateTime, + `state` String, + `rollback` UInt8, + `num_tries` UInt32, + `last_exception` String +) +ENGINE = SystemShardMoves() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.parts +( + `partition` String, + `name` String, + `uuid` UUID, + `part_type` String, + `active` UInt8, + `marks` UInt64, + `rows` UInt64, + `bytes_on_disk` UInt64, + `data_compressed_bytes` UInt64, + `data_uncompressed_bytes` UInt64, + `marks_bytes` UInt64, + `secondary_indices_compressed_bytes` UInt64, + `secondary_indices_uncompressed_bytes` UInt64, + `secondary_indices_marks_bytes` UInt64, + `modification_time` DateTime, + `remove_time` DateTime, + `refcount` UInt32, + `min_date` Date, + `max_date` Date, + `min_time` DateTime, + `max_time` DateTime, + `partition_id` String, + `min_block_number` Int64, + `max_block_number` Int64, + `level` UInt32, + `data_version` UInt64, + `primary_key_bytes_in_memory` UInt64, + `primary_key_bytes_in_memory_allocated` UInt64, + `is_frozen` UInt8, + `database` String, + `table` String, + `engine` String, + `disk_name` String, + `path` String, + `hash_of_all_files` String, + `hash_of_uncompressed_files` String, + `uncompressed_hash_of_compressed_files` String, + `delete_ttl_info_min` DateTime, + `delete_ttl_info_max` DateTime, + `move_ttl_info.expression` Array(String), + `move_ttl_info.min` Array(DateTime), + `move_ttl_info.max` Array(DateTime), + `default_compression_codec` String, + `recompression_ttl_info.expression` Array(String), + `recompression_ttl_info.min` Array(DateTime), + `recompression_ttl_info.max` Array(DateTime), + `group_by_ttl_info.expression` Array(String), + `group_by_ttl_info.min` Array(DateTime), + `group_by_ttl_info.max` Array(DateTime), + `rows_where_ttl_info.expression` Array(String), + `rows_where_ttl_info.min` Array(DateTime), + `rows_where_ttl_info.max` Array(DateTime), + `projections` Array(String), + `visible` UInt8, + `creation_tid` Tuple(UInt64, UInt64, UUID), + `removal_tid` Tuple(UInt64, UInt64, UUID), + `creation_csn` UInt64, + `removal_csn` UInt64, + `bytes` UInt64, + `marks_size` UInt64 +) +ENGINE = SystemParts() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.parts_columns +( + `partition` String, + `name` String, + `uuid` UUID, + `part_type` String, + `active` UInt8, + `marks` UInt64, + `rows` UInt64, + `bytes_on_disk` UInt64, + `data_compressed_bytes` UInt64, + `data_uncompressed_bytes` UInt64, + `marks_bytes` UInt64, + `modification_time` DateTime, + `remove_time` DateTime, + `refcount` UInt32, + `min_date` Date, + `max_date` Date, + `min_time` DateTime, + `max_time` DateTime, + `partition_id` String, + `min_block_number` Int64, + `max_block_number` Int64, + `level` UInt32, + `data_version` UInt64, + `primary_key_bytes_in_memory` UInt64, + `primary_key_bytes_in_memory_allocated` UInt64, + `database` String, + `table` String, + `engine` String, + `disk_name` String, + `path` String, + `column` String, + `type` String, + `column_position` UInt64, + `default_kind` String, + `default_expression` String, + `column_bytes_on_disk` UInt64, + `column_data_compressed_bytes` UInt64, + `column_data_uncompressed_bytes` UInt64, + `column_marks_bytes` UInt64, + `serialization_kind` String, + `subcolumns.names` Array(String), + `subcolumns.types` Array(String), + `subcolumns.serializations` Array(String), + `subcolumns.bytes_on_disk` Array(UInt64), + `subcolumns.data_compressed_bytes` Array(UInt64), + `subcolumns.data_uncompressed_bytes` Array(UInt64), + `subcolumns.marks_bytes` Array(UInt64), + `bytes` UInt64, + `marks_size` UInt64 +) +ENGINE = SystemPartsColumns() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.privileges +( + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SELECT' = 5, 'INSERT' = 6, 'ALTER UPDATE' = 7, 'ALTER DELETE' = 8, 'ALTER ADD COLUMN' = 9, 'ALTER MODIFY COLUMN' = 10, 'ALTER DROP COLUMN' = 11, 'ALTER COMMENT COLUMN' = 12, 'ALTER CLEAR COLUMN' = 13, 'ALTER RENAME COLUMN' = 14, 'ALTER MATERIALIZE COLUMN' = 15, 'ALTER COLUMN' = 16, 'ALTER MODIFY COMMENT' = 17, 'ALTER ORDER BY' = 18, 'ALTER SAMPLE BY' = 19, 'ALTER ADD INDEX' = 20, 'ALTER DROP INDEX' = 21, 'ALTER MATERIALIZE INDEX' = 22, 'ALTER CLEAR INDEX' = 23, 'ALTER INDEX' = 24, 'ALTER ADD PROJECTION' = 25, 'ALTER DROP PROJECTION' = 26, 'ALTER MATERIALIZE PROJECTION' = 27, 'ALTER CLEAR PROJECTION' = 28, 'ALTER PROJECTION' = 29, 'ALTER ADD CONSTRAINT' = 30, 'ALTER DROP CONSTRAINT' = 31, 'ALTER CONSTRAINT' = 32, 'ALTER TTL' = 33, 'ALTER MATERIALIZE TTL' = 34, 'ALTER SETTINGS' = 35, 'ALTER MOVE PARTITION' = 36, 'ALTER FETCH PARTITION' = 37, 'ALTER FREEZE PARTITION' = 38, 'ALTER DATABASE SETTINGS' = 39, 'ALTER TABLE' = 40, 'ALTER DATABASE' = 41, 'ALTER VIEW REFRESH' = 42, 'ALTER VIEW MODIFY QUERY' = 43, 'ALTER VIEW' = 44, 'ALTER' = 45, 'CREATE DATABASE' = 46, 'CREATE TABLE' = 47, 'CREATE VIEW' = 48, 'CREATE DICTIONARY' = 49, 'CREATE TEMPORARY TABLE' = 50, 'CREATE FUNCTION' = 51, 'CREATE' = 52, 'DROP DATABASE' = 53, 'DROP TABLE' = 54, 'DROP VIEW' = 55, 'DROP DICTIONARY' = 56, 'DROP FUNCTION' = 57, 'DROP' = 58, 'TRUNCATE' = 59, 'OPTIMIZE' = 60, 'KILL QUERY' = 61, 'KILL TRANSACTION' = 62, 'MOVE PARTITION BETWEEN SHARDS' = 63, 'CREATE USER' = 64, 'ALTER USER' = 65, 'DROP USER' = 66, 'CREATE ROLE' = 67, 'ALTER ROLE' = 68, 'DROP ROLE' = 69, 'ROLE ADMIN' = 70, 'CREATE ROW POLICY' = 71, 'ALTER ROW POLICY' = 72, 'DROP ROW POLICY' = 73, 'CREATE QUOTA' = 74, 'ALTER QUOTA' = 75, 'DROP QUOTA' = 76, 'CREATE SETTINGS PROFILE' = 77, 'ALTER SETTINGS PROFILE' = 78, 'DROP SETTINGS PROFILE' = 79, 'SHOW USERS' = 80, 'SHOW ROLES' = 81, 'SHOW ROW POLICIES' = 82, 'SHOW QUOTAS' = 83, 'SHOW SETTINGS PROFILES' = 84, 'SHOW ACCESS' = 85, 'ACCESS MANAGEMENT' = 86, 'SYSTEM SHUTDOWN' = 87, 'SYSTEM DROP DNS CACHE' = 88, 'SYSTEM DROP MARK CACHE' = 89, 'SYSTEM DROP UNCOMPRESSED CACHE' = 90, 'SYSTEM DROP MMAP CACHE' = 91, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 92, 'SYSTEM DROP CACHE' = 93, 'SYSTEM RELOAD CONFIG' = 94, 'SYSTEM RELOAD SYMBOLS' = 95, 'SYSTEM RELOAD DICTIONARY' = 96, 'SYSTEM RELOAD MODEL' = 97, 'SYSTEM RELOAD FUNCTION' = 98, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 99, 'SYSTEM RELOAD' = 100, 'SYSTEM RESTART DISK' = 101, 'SYSTEM MERGES' = 102, 'SYSTEM TTL MERGES' = 103, 'SYSTEM FETCHES' = 104, 'SYSTEM MOVES' = 105, 'SYSTEM DISTRIBUTED SENDS' = 106, 'SYSTEM REPLICATED SENDS' = 107, 'SYSTEM SENDS' = 108, 'SYSTEM REPLICATION QUEUES' = 109, 'SYSTEM DROP REPLICA' = 110, 'SYSTEM SYNC REPLICA' = 111, 'SYSTEM RESTART REPLICA' = 112, 'SYSTEM RESTORE REPLICA' = 113, 'SYSTEM FLUSH DISTRIBUTED' = 114, 'SYSTEM FLUSH LOGS' = 115, 'SYSTEM FLUSH' = 116, 'SYSTEM THREAD FUZZER' = 117, 'SYSTEM' = 118, 'dictGet' = 119, 'addressToLine' = 120, 'addressToLineWithInlines' = 121, 'addressToSymbol' = 122, 'demangle' = 123, 'INTROSPECTION' = 124, 'FILE' = 125, 'URL' = 126, 'REMOTE' = 127, 'MONGO' = 128, 'MYSQL' = 129, 'POSTGRES' = 130, 'SQLITE' = 131, 'ODBC' = 132, 'JDBC' = 133, 'HDFS' = 134, 'S3' = 135, 'HIVE' = 136, 'SOURCES' = 137, 'ALL' = 138, 'NONE' = 139), + `aliases` Array(String), + `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SELECT' = 5, 'INSERT' = 6, 'ALTER UPDATE' = 7, 'ALTER DELETE' = 8, 'ALTER ADD COLUMN' = 9, 'ALTER MODIFY COLUMN' = 10, 'ALTER DROP COLUMN' = 11, 'ALTER COMMENT COLUMN' = 12, 'ALTER CLEAR COLUMN' = 13, 'ALTER RENAME COLUMN' = 14, 'ALTER MATERIALIZE COLUMN' = 15, 'ALTER COLUMN' = 16, 'ALTER MODIFY COMMENT' = 17, 'ALTER ORDER BY' = 18, 'ALTER SAMPLE BY' = 19, 'ALTER ADD INDEX' = 20, 'ALTER DROP INDEX' = 21, 'ALTER MATERIALIZE INDEX' = 22, 'ALTER CLEAR INDEX' = 23, 'ALTER INDEX' = 24, 'ALTER ADD PROJECTION' = 25, 'ALTER DROP PROJECTION' = 26, 'ALTER MATERIALIZE PROJECTION' = 27, 'ALTER CLEAR PROJECTION' = 28, 'ALTER PROJECTION' = 29, 'ALTER ADD CONSTRAINT' = 30, 'ALTER DROP CONSTRAINT' = 31, 'ALTER CONSTRAINT' = 32, 'ALTER TTL' = 33, 'ALTER MATERIALIZE TTL' = 34, 'ALTER SETTINGS' = 35, 'ALTER MOVE PARTITION' = 36, 'ALTER FETCH PARTITION' = 37, 'ALTER FREEZE PARTITION' = 38, 'ALTER DATABASE SETTINGS' = 39, 'ALTER TABLE' = 40, 'ALTER DATABASE' = 41, 'ALTER VIEW REFRESH' = 42, 'ALTER VIEW MODIFY QUERY' = 43, 'ALTER VIEW' = 44, 'ALTER' = 45, 'CREATE DATABASE' = 46, 'CREATE TABLE' = 47, 'CREATE VIEW' = 48, 'CREATE DICTIONARY' = 49, 'CREATE TEMPORARY TABLE' = 50, 'CREATE FUNCTION' = 51, 'CREATE' = 52, 'DROP DATABASE' = 53, 'DROP TABLE' = 54, 'DROP VIEW' = 55, 'DROP DICTIONARY' = 56, 'DROP FUNCTION' = 57, 'DROP' = 58, 'TRUNCATE' = 59, 'OPTIMIZE' = 60, 'KILL QUERY' = 61, 'KILL TRANSACTION' = 62, 'MOVE PARTITION BETWEEN SHARDS' = 63, 'CREATE USER' = 64, 'ALTER USER' = 65, 'DROP USER' = 66, 'CREATE ROLE' = 67, 'ALTER ROLE' = 68, 'DROP ROLE' = 69, 'ROLE ADMIN' = 70, 'CREATE ROW POLICY' = 71, 'ALTER ROW POLICY' = 72, 'DROP ROW POLICY' = 73, 'CREATE QUOTA' = 74, 'ALTER QUOTA' = 75, 'DROP QUOTA' = 76, 'CREATE SETTINGS PROFILE' = 77, 'ALTER SETTINGS PROFILE' = 78, 'DROP SETTINGS PROFILE' = 79, 'SHOW USERS' = 80, 'SHOW ROLES' = 81, 'SHOW ROW POLICIES' = 82, 'SHOW QUOTAS' = 83, 'SHOW SETTINGS PROFILES' = 84, 'SHOW ACCESS' = 85, 'ACCESS MANAGEMENT' = 86, 'SYSTEM SHUTDOWN' = 87, 'SYSTEM DROP DNS CACHE' = 88, 'SYSTEM DROP MARK CACHE' = 89, 'SYSTEM DROP UNCOMPRESSED CACHE' = 90, 'SYSTEM DROP MMAP CACHE' = 91, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 92, 'SYSTEM DROP CACHE' = 93, 'SYSTEM RELOAD CONFIG' = 94, 'SYSTEM RELOAD SYMBOLS' = 95, 'SYSTEM RELOAD DICTIONARY' = 96, 'SYSTEM RELOAD MODEL' = 97, 'SYSTEM RELOAD FUNCTION' = 98, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 99, 'SYSTEM RELOAD' = 100, 'SYSTEM RESTART DISK' = 101, 'SYSTEM MERGES' = 102, 'SYSTEM TTL MERGES' = 103, 'SYSTEM FETCHES' = 104, 'SYSTEM MOVES' = 105, 'SYSTEM DISTRIBUTED SENDS' = 106, 'SYSTEM REPLICATED SENDS' = 107, 'SYSTEM SENDS' = 108, 'SYSTEM REPLICATION QUEUES' = 109, 'SYSTEM DROP REPLICA' = 110, 'SYSTEM SYNC REPLICA' = 111, 'SYSTEM RESTART REPLICA' = 112, 'SYSTEM RESTORE REPLICA' = 113, 'SYSTEM FLUSH DISTRIBUTED' = 114, 'SYSTEM FLUSH LOGS' = 115, 'SYSTEM FLUSH' = 116, 'SYSTEM THREAD FUZZER' = 117, 'SYSTEM' = 118, 'dictGet' = 119, 'addressToLine' = 120, 'addressToLineWithInlines' = 121, 'addressToSymbol' = 122, 'demangle' = 123, 'INTROSPECTION' = 124, 'FILE' = 125, 'URL' = 126, 'REMOTE' = 127, 'MONGO' = 128, 'MYSQL' = 129, 'POSTGRES' = 130, 'SQLITE' = 131, 'ODBC' = 132, 'JDBC' = 133, 'HDFS' = 134, 'S3' = 135, 'HIVE' = 136, 'SOURCES' = 137, 'ALL' = 138, 'NONE' = 139)) +) +ENGINE = SystemPrivileges() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.processes +( + `is_initial_query` UInt8, + `user` String, + `query_id` String, + `address` IPv6, + `port` UInt16, + `initial_user` String, + `initial_query_id` String, + `initial_address` IPv6, + `initial_port` UInt16, + `interface` UInt8, + `os_user` String, + `client_hostname` String, + `client_name` String, + `client_revision` UInt64, + `client_version_major` UInt64, + `client_version_minor` UInt64, + `client_version_patch` UInt64, + `http_method` UInt8, + `http_user_agent` String, + `http_referer` String, + `forwarded_for` String, + `quota_key` String, + `distributed_depth` UInt64, + `elapsed` Float64, + `is_cancelled` UInt8, + `is_all_data_sent` UInt8, + `read_rows` UInt64, + `read_bytes` UInt64, + `total_rows_approx` UInt64, + `written_rows` UInt64, + `written_bytes` UInt64, + `memory_usage` Int64, + `peak_memory_usage` Int64, + `query` String, + `thread_ids` Array(UInt64), + `ProfileEvents` Map(String, UInt64), + `Settings` Map(String, String), + `current_database` String, + `ProfileEvents.Names` Array(String), + `ProfileEvents.Values` Array(UInt64), + `Settings.Names` Array(String), + `Settings.Values` Array(String) +) +ENGINE = SystemProcesses() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.projection_parts +( + `partition` String, + `name` String, + `part_type` String, + `parent_name` String, + `parent_uuid` UUID, + `parent_part_type` String, + `active` UInt8, + `marks` UInt64, + `rows` UInt64, + `bytes_on_disk` UInt64, + `data_compressed_bytes` UInt64, + `data_uncompressed_bytes` UInt64, + `marks_bytes` UInt64, + `parent_marks` UInt64, + `parent_rows` UInt64, + `parent_bytes_on_disk` UInt64, + `parent_data_compressed_bytes` UInt64, + `parent_data_uncompressed_bytes` UInt64, + `parent_marks_bytes` UInt64, + `modification_time` DateTime, + `remove_time` DateTime, + `refcount` UInt32, + `min_date` Date, + `max_date` Date, + `min_time` DateTime, + `max_time` DateTime, + `partition_id` String, + `min_block_number` Int64, + `max_block_number` Int64, + `level` UInt32, + `data_version` UInt64, + `primary_key_bytes_in_memory` UInt64, + `primary_key_bytes_in_memory_allocated` UInt64, + `is_frozen` UInt8, + `database` String, + `table` String, + `engine` String, + `disk_name` String, + `path` String, + `hash_of_all_files` String, + `hash_of_uncompressed_files` String, + `uncompressed_hash_of_compressed_files` String, + `delete_ttl_info_min` DateTime, + `delete_ttl_info_max` DateTime, + `move_ttl_info.expression` Array(String), + `move_ttl_info.min` Array(DateTime), + `move_ttl_info.max` Array(DateTime), + `default_compression_codec` String, + `recompression_ttl_info.expression` Array(String), + `recompression_ttl_info.min` Array(DateTime), + `recompression_ttl_info.max` Array(DateTime), + `group_by_ttl_info.expression` Array(String), + `group_by_ttl_info.min` Array(DateTime), + `group_by_ttl_info.max` Array(DateTime), + `rows_where_ttl_info.expression` Array(String), + `rows_where_ttl_info.min` Array(DateTime), + `rows_where_ttl_info.max` Array(DateTime), + `bytes` UInt64, + `marks_size` UInt64 +) +ENGINE = SystemProjectionParts() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.projection_parts_columns +( + `partition` String, + `name` String, + `part_type` String, + `parent_name` String, + `parent_uuid` UUID, + `parent_part_type` String, + `active` UInt8, + `marks` UInt64, + `rows` UInt64, + `bytes_on_disk` UInt64, + `data_compressed_bytes` UInt64, + `data_uncompressed_bytes` UInt64, + `marks_bytes` UInt64, + `parent_marks` UInt64, + `parent_rows` UInt64, + `parent_bytes_on_disk` UInt64, + `parent_data_compressed_bytes` UInt64, + `parent_data_uncompressed_bytes` UInt64, + `parent_marks_bytes` UInt64, + `modification_time` DateTime, + `remove_time` DateTime, + `refcount` UInt32, + `min_date` Date, + `max_date` Date, + `min_time` DateTime, + `max_time` DateTime, + `partition_id` String, + `min_block_number` Int64, + `max_block_number` Int64, + `level` UInt32, + `data_version` UInt64, + `primary_key_bytes_in_memory` UInt64, + `primary_key_bytes_in_memory_allocated` UInt64, + `database` String, + `table` String, + `engine` String, + `disk_name` String, + `path` String, + `column` String, + `type` String, + `column_position` UInt64, + `default_kind` String, + `default_expression` String, + `column_bytes_on_disk` UInt64, + `column_data_compressed_bytes` UInt64, + `column_data_uncompressed_bytes` UInt64, + `column_marks_bytes` UInt64, + `bytes` UInt64, + `marks_size` UInt64 +) +ENGINE = SystemProjectionPartsColumns() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.quota_limits +( + `quota_name` String, + `duration` UInt32, + `is_randomized_interval` UInt8, + `max_queries` Nullable(UInt64), + `max_query_selects` Nullable(UInt64), + `max_query_inserts` Nullable(UInt64), + `max_errors` Nullable(UInt64), + `max_result_rows` Nullable(UInt64), + `max_result_bytes` Nullable(UInt64), + `max_read_rows` Nullable(UInt64), + `max_read_bytes` Nullable(UInt64), + `max_execution_time` Nullable(Float64), + `max_written_bytes` Nullable(UInt64) +) +ENGINE = SystemQuotaLimits() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.quota_usage +( + `quota_name` String, + `quota_key` String, + `start_time` Nullable(DateTime), + `end_time` Nullable(DateTime), + `duration` Nullable(UInt32), + `queries` Nullable(UInt64), + `max_queries` Nullable(UInt64), + `query_selects` Nullable(UInt64), + `max_query_selects` Nullable(UInt64), + `query_inserts` Nullable(UInt64), + `max_query_inserts` Nullable(UInt64), + `errors` Nullable(UInt64), + `max_errors` Nullable(UInt64), + `result_rows` Nullable(UInt64), + `max_result_rows` Nullable(UInt64), + `result_bytes` Nullable(UInt64), + `max_result_bytes` Nullable(UInt64), + `read_rows` Nullable(UInt64), + `max_read_rows` Nullable(UInt64), + `read_bytes` Nullable(UInt64), + `max_read_bytes` Nullable(UInt64), + `execution_time` Nullable(Float64), + `max_execution_time` Nullable(Float64), + `written_bytes` Nullable(UInt64), + `max_written_bytes` Nullable(UInt64) +) +ENGINE = SystemQuotaUsage() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.quotas +( + `name` String, + `id` UUID, + `storage` String, + `keys` Array(Enum8('user_name' = 1, 'ip_address' = 2, 'forwarded_ip_address' = 3, 'client_key' = 4)), + `durations` Array(UInt32), + `apply_to_all` UInt8, + `apply_to_list` Array(String), + `apply_to_except` Array(String) +) +ENGINE = SystemQuotas() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.quotas_usage +( + `quota_name` String, + `quota_key` String, + `is_current` UInt8, + `start_time` Nullable(DateTime), + `end_time` Nullable(DateTime), + `duration` Nullable(UInt32), + `queries` Nullable(UInt64), + `max_queries` Nullable(UInt64), + `query_selects` Nullable(UInt64), + `max_query_selects` Nullable(UInt64), + `query_inserts` Nullable(UInt64), + `max_query_inserts` Nullable(UInt64), + `errors` Nullable(UInt64), + `max_errors` Nullable(UInt64), + `result_rows` Nullable(UInt64), + `max_result_rows` Nullable(UInt64), + `result_bytes` Nullable(UInt64), + `max_result_bytes` Nullable(UInt64), + `read_rows` Nullable(UInt64), + `max_read_rows` Nullable(UInt64), + `read_bytes` Nullable(UInt64), + `max_read_bytes` Nullable(UInt64), + `execution_time` Nullable(Float64), + `max_execution_time` Nullable(Float64), + `written_bytes` Nullable(UInt64), + `max_written_bytes` Nullable(UInt64) +) +ENGINE = SystemQuotasUsage() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.replicas +( + `database` String, + `table` String, + `engine` String, + `is_leader` UInt8, + `can_become_leader` UInt8, + `is_readonly` UInt8, + `is_session_expired` UInt8, + `future_parts` UInt32, + `parts_to_check` UInt32, + `zookeeper_path` String, + `replica_name` String, + `replica_path` String, + `columns_version` Int32, + `queue_size` UInt32, + `inserts_in_queue` UInt32, + `merges_in_queue` UInt32, + `part_mutations_in_queue` UInt32, + `queue_oldest_time` DateTime, + `inserts_oldest_time` DateTime, + `merges_oldest_time` DateTime, + `part_mutations_oldest_time` DateTime, + `oldest_part_to_get` String, + `oldest_part_to_merge_to` String, + `oldest_part_to_mutate_to` String, + `log_max_index` UInt64, + `log_pointer` UInt64, + `last_queue_update` DateTime, + `absolute_delay` UInt64, + `total_replicas` UInt8, + `active_replicas` UInt8, + `last_queue_update_exception` String, + `zookeeper_exception` String, + `replica_is_active` Map(String, UInt8) +) +ENGINE = SystemReplicas() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.replicated_fetches +( + `database` String, + `table` String, + `elapsed` Float64, + `progress` Float64, + `result_part_name` String, + `result_part_path` String, + `partition_id` String, + `total_size_bytes_compressed` UInt64, + `bytes_read_compressed` UInt64, + `source_replica_path` String, + `source_replica_hostname` String, + `source_replica_port` UInt16, + `interserver_scheme` String, + `URI` String, + `to_detached` UInt8, + `thread_id` UInt64 +) +ENGINE = SystemReplicatedFetches() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.replicated_merge_tree_settings +( + `name` String, + `value` String, + `changed` UInt8, + `description` String, + `type` String +) +ENGINE = SystemReplicatedMergeTreeSettings() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.replication_queue +( + `database` String, + `table` String, + `replica_name` String, + `position` UInt32, + `node_name` String, + `type` String, + `create_time` DateTime, + `required_quorum` UInt32, + `source_replica` String, + `new_part_name` String, + `parts_to_merge` Array(String), + `is_detach` UInt8, + `is_currently_executing` UInt8, + `num_tries` UInt32, + `last_exception` String, + `last_attempt_time` DateTime, + `num_postponed` UInt32, + `postpone_reason` String, + `last_postpone_time` DateTime, + `merge_type` String +) +ENGINE = SystemReplicationQueue() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.role_grants +( + `user_name` Nullable(String), + `role_name` Nullable(String), + `granted_role_name` String, + `granted_role_is_default` UInt8, + `with_admin_option` UInt8 +) +ENGINE = SystemRoleGrants() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.roles +( + `name` String, + `id` UUID, + `storage` String +) +ENGINE = SystemRoles() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.row_policies +( + `name` String, + `short_name` String, + `database` String, + `table` String, + `id` UUID, + `storage` String, + `select_filter` Nullable(String), + `is_restrictive` UInt8, + `apply_to_all` UInt8, + `apply_to_list` Array(String), + `apply_to_except` Array(String) +) +ENGINE = SystemRowPolicies() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.settings +( + `name` String, + `value` String, + `changed` UInt8, + `description` String, + `min` Nullable(String), + `max` Nullable(String), + `readonly` UInt8, + `type` String +) +ENGINE = SystemSettings() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.settings_profile_elements +( + `profile_name` Nullable(String), + `user_name` Nullable(String), + `role_name` Nullable(String), + `index` UInt64, + `setting_name` Nullable(String), + `value` Nullable(String), + `min` Nullable(String), + `max` Nullable(String), + `readonly` Nullable(UInt8), + `inherit_profile` Nullable(String) +) +ENGINE = SystemSettingsProfileElements() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.settings_profiles +( + `name` String, + `id` UUID, + `storage` String, + `num_elements` UInt64, + `apply_to_all` UInt8, + `apply_to_list` Array(String), + `apply_to_except` Array(String) +) +ENGINE = SystemSettingsProfiles() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.stack_trace +( + `thread_name` String, + `thread_id` UInt64, + `query_id` String, + `trace` Array(UInt64) +) +ENGINE = SystemStackTrace() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.storage_policies +( + `policy_name` String, + `volume_name` String, + `volume_priority` UInt64, + `disks` Array(String), + `volume_type` String, + `max_data_part_size` UInt64, + `move_factor` Float32, + `prefer_not_to_merge` UInt8 +) +ENGINE = SystemStoragePolicies() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.table_engines +( + `name` String, + `supports_settings` UInt8, + `supports_skipping_indices` UInt8, + `supports_projections` UInt8, + `supports_sort_order` UInt8, + `supports_ttl` UInt8, + `supports_replication` UInt8, + `supports_deduplication` UInt8, + `supports_parallel_insert` UInt8 +) +ENGINE = SystemTableEngines() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.table_functions +( + `name` String +) +ENGINE = SystemTableFunctions() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.tables +( + `database` String, + `name` String, + `uuid` UUID, + `engine` String, + `is_temporary` UInt8, + `data_paths` Array(String), + `metadata_path` String, + `metadata_modification_time` DateTime, + `dependencies_database` Array(String), + `dependencies_table` Array(String), + `create_table_query` String, + `engine_full` String, + `as_select` String, + `partition_key` String, + `sorting_key` String, + `primary_key` String, + `sampling_key` String, + `storage_policy` String, + `total_rows` Nullable(UInt64), + `total_bytes` Nullable(UInt64), + `lifetime_rows` Nullable(UInt64), + `lifetime_bytes` Nullable(UInt64), + `comment` String, + `has_own_data` UInt8, + `loading_dependencies_database` Array(String), + `loading_dependencies_table` Array(String), + `loading_dependent_database` Array(String), + `loading_dependent_table` Array(String), + `table` String +) +ENGINE = SystemTables() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.time_zones +( + `time_zone` String +) +ENGINE = SystemTimeZones() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.user_directories +( + `name` String, + `type` String, + `params` String, + `precedence` UInt64 +) +ENGINE = SystemUserDirectories() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.users +( + `name` String, + `id` UUID, + `storage` String, + `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6), + `auth_params` String, + `host_ip` Array(String), + `host_names` Array(String), + `host_names_regexp` Array(String), + `host_names_like` Array(String), + `default_roles_all` UInt8, + `default_roles_list` Array(String), + `default_roles_except` Array(String), + `grantees_any` UInt8, + `grantees_list` Array(String), + `grantees_except` Array(String), + `default_database` String +) +ENGINE = SystemUsers() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.warnings +( + `message` String +) +ENGINE = SystemWarnings() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.zeros +( + `zero` UInt8 +) +ENGINE = SystemZeros() +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.zeros_mt +( + `zero` UInt8 +) +ENGINE = SystemZeros() +COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02117_show_create_table_system.sql b/tests/queries/0_stateless/02117_show_create_table_system.sql index 16861cc3f8e..9a5726a0780 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.sql +++ b/tests/queries/0_stateless/02117_show_create_table_system.sql @@ -16,69 +16,69 @@ tables_with_database_column=( */ use system; -show create table aggregate_function_combinators; -show create table asynchronous_inserts; -show create table asynchronous_metrics; -show create table build_options; -show create table clusters; -show create table collations; -show create table columns; -show create table contributors; -show create table current_roles; -show create table data_skipping_indices; -show create table data_type_families; -show create table databases; -show create table detached_parts; -show create table dictionaries; -show create table disks; -show create table distributed_ddl_queue; -show create table distribution_queue; -show create table enabled_roles; -show create table errors; -show create table events; -show create table formats; -show create table functions; -show create table grants; -show create table graphite_retentions; -show create table licenses; -show create table macros; -show create table merge_tree_settings; -show create table merges; -show create table metrics; -show create table models; -show create table mutations; -show create table numbers; -show create table numbers_mt; -show create table one; -show create table part_moves_between_shards; -show create table parts; -show create table parts_columns; -show create table privileges; -show create table processes; -show create table projection_parts; -show create table projection_parts_columns; -show create table quota_limits; -show create table quota_usage; -show create table quotas; -show create table quotas_usage; -show create table replicas; -show create table replicated_fetches; -show create table replicated_merge_tree_settings; -show create table replication_queue; -show create table role_grants; -show create table roles; -show create table row_policies; -show create table settings; -show create table settings_profile_elements; -show create table settings_profiles; -show create table stack_trace; -show create table storage_policies; -show create table table_engines; -show create table table_functions; -show create table tables; -show create table time_zones; -show create table user_directories; -show create table users; -show create table warnings; -show create table zeros; -show create table zeros_mt; +show create table aggregate_function_combinators format TSVRaw; +show create table asynchronous_inserts format TSVRaw; +show create table asynchronous_metrics format TSVRaw; +show create table build_options format TSVRaw; +show create table clusters format TSVRaw; +show create table collations format TSVRaw; +show create table columns format TSVRaw; +show create table contributors format TSVRaw; +show create table current_roles format TSVRaw; +show create table data_skipping_indices format TSVRaw; +show create table data_type_families format TSVRaw; +show create table databases format TSVRaw; +show create table detached_parts format TSVRaw; +show create table dictionaries format TSVRaw; +show create table disks format TSVRaw; +show create table distributed_ddl_queue format TSVRaw; +show create table distribution_queue format TSVRaw; +show create table enabled_roles format TSVRaw; +show create table errors format TSVRaw; +show create table events format TSVRaw; +show create table formats format TSVRaw; +show create table functions format TSVRaw; +show create table grants format TSVRaw; +show create table graphite_retentions format TSVRaw; +show create table licenses format TSVRaw; +show create table macros format TSVRaw; +show create table merge_tree_settings format TSVRaw; +show create table merges format TSVRaw; +show create table metrics format TSVRaw; +show create table models format TSVRaw; +show create table mutations format TSVRaw; +show create table numbers format TSVRaw; +show create table numbers_mt format TSVRaw; +show create table one format TSVRaw; +show create table part_moves_between_shards format TSVRaw; +show create table parts format TSVRaw; +show create table parts_columns format TSVRaw; +show create table privileges format TSVRaw; +show create table processes format TSVRaw; +show create table projection_parts format TSVRaw; +show create table projection_parts_columns format TSVRaw; +show create table quota_limits format TSVRaw; +show create table quota_usage format TSVRaw; +show create table quotas format TSVRaw; +show create table quotas_usage format TSVRaw; +show create table replicas format TSVRaw; +show create table replicated_fetches format TSVRaw; +show create table replicated_merge_tree_settings format TSVRaw; +show create table replication_queue format TSVRaw; +show create table role_grants format TSVRaw; +show create table roles format TSVRaw; +show create table row_policies format TSVRaw; +show create table settings format TSVRaw; +show create table settings_profile_elements format TSVRaw; +show create table settings_profiles format TSVRaw; +show create table stack_trace format TSVRaw; +show create table storage_policies format TSVRaw; +show create table table_engines format TSVRaw; +show create table table_functions format TSVRaw; +show create table tables format TSVRaw; +show create table time_zones format TSVRaw; +show create table user_directories format TSVRaw; +show create table users format TSVRaw; +show create table warnings format TSVRaw; +show create table zeros format TSVRaw; +show create table zeros_mt format TSVRaw; diff --git a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference index 67a043d6646..9e24b7c6ea6 100644 --- a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference +++ b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference @@ -7,15 +7,13 @@ ExpressionTransform (Limit) Limit - (Expression) - ExpressionTransform - (Sorting) - MergingSortedTransform 2 → 1 - (Expression) - ExpressionTransform × 2 - (SettingQuotaAndLimits) - (ReadFromMergeTree) - MergeTreeInOrder × 2 0 → 1 + (Sorting) + MergingSortedTransform 2 → 1 + (Expression) + ExpressionTransform × 2 + (SettingQuotaAndLimits) + (ReadFromMergeTree) + MergeTreeInOrder × 2 0 → 1 2020-10-01 9 2020-10-01 9 2020-10-01 9 @@ -25,18 +23,16 @@ ExpressionTransform ExpressionTransform (Limit) Limit - (Expression) - ExpressionTransform - (Sorting) - MergingSortedTransform 2 → 1 - (Expression) - ExpressionTransform × 2 - (SettingQuotaAndLimits) - (ReadFromMergeTree) - ReverseTransform - MergeTreeReverse 0 → 1 - ReverseTransform - MergeTreeReverse 0 → 1 + (Sorting) + MergingSortedTransform 2 → 1 + (Expression) + ExpressionTransform × 2 + (SettingQuotaAndLimits) + (ReadFromMergeTree) + ReverseTransform + MergeTreeReverse 0 → 1 + ReverseTransform + MergeTreeReverse 0 → 1 2020-10-01 9 2020-10-01 9 2020-10-01 9 @@ -46,17 +42,15 @@ ExpressionTransform ExpressionTransform (Limit) Limit - (Expression) - ExpressionTransform - (Sorting) - FinishSortingTransform - PartialSortingTransform - MergingSortedTransform 2 → 1 - (Expression) - ExpressionTransform × 2 - (SettingQuotaAndLimits) - (ReadFromMergeTree) - MergeTreeInOrder × 2 0 → 1 + (Sorting) + FinishSortingTransform + PartialSortingTransform + MergingSortedTransform 2 → 1 + (Expression) + ExpressionTransform × 2 + (SettingQuotaAndLimits) + (ReadFromMergeTree) + MergeTreeInOrder × 2 0 → 1 2020-10-11 0 2020-10-11 0 2020-10-11 0 diff --git a/tests/queries/0_stateless/02180_group_by_lowcardinality.sql b/tests/queries/0_stateless/02180_group_by_lowcardinality.sql index 463753a624e..8f9e8891890 100644 --- a/tests/queries/0_stateless/02180_group_by_lowcardinality.sql +++ b/tests/queries/0_stateless/02180_group_by_lowcardinality.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + create table if not exists t_group_by_lowcardinality(p_date Date, val LowCardinality(Nullable(String))) engine=MergeTree() partition by p_date order by tuple(); diff --git a/tests/queries/0_stateless/02184_default_table_engine.reference b/tests/queries/0_stateless/02184_default_table_engine.reference index 200578f3da9..bc217f3b23a 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.reference +++ b/tests/queries/0_stateless/02184_default_table_engine.reference @@ -25,3 +25,5 @@ CREATE TABLE default.mem\n(\n `n` UInt8\n)\nENGINE = Memory CREATE TABLE default.val\n(\n `n` Int32\n) AS values(\'n int\', 1, 2) CREATE TABLE default.val2\n(\n `n` Int32\n) AS values(\'n int\', 1, 2) CREATE TABLE default.log\n(\n `n` Int32\n)\nENGINE = Log +CREATE TABLE default.kek\n(\n `n` Int32\n)\nENGINE = Memory +CREATE TABLE default.lol\n(\n `n` Int32\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS min_bytes_for_wide_part = 123, index_granularity = 8192 diff --git a/tests/queries/0_stateless/02184_default_table_engine.sql b/tests/queries/0_stateless/02184_default_table_engine.sql index d129ccc801e..0aac32c9869 100644 --- a/tests/queries/0_stateless/02184_default_table_engine.sql +++ b/tests/queries/0_stateless/02184_default_table_engine.sql @@ -66,7 +66,7 @@ DROP TABLE t2; CREATE DATABASE test_02184 ORDER BY kek; -- {serverError 80} -CREATE DATABASE test_02184 SETTINGS x=1; -- {serverError 80} +CREATE DATABASE test_02184 SETTINGS x=1; -- {serverError 115} CREATE TABLE table_02184 (x UInt8, y int, PRIMARY KEY (x)) ENGINE=MergeTree PRIMARY KEY y; -- {clientError 36} SET default_table_engine = 'MergeTree'; CREATE TABLE table_02184 (x UInt8, y int, PRIMARY KEY (x)) PRIMARY KEY y; -- {clientError 36} @@ -118,3 +118,13 @@ SHOW CREATE TABLE log; DROP TABLE val; DROP TABLE val2; DROP TABLE log; + +DROP TABLE IF EXISTS kek; +DROP TABLE IF EXISTS lol; +SET default_table_engine = 'Memory'; +CREATE TABLE kek (n int) SETTINGS log_queries=1; +CREATE TABLE lol (n int) ENGINE=MergeTree ORDER BY n SETTINGS min_bytes_for_wide_part=123 SETTINGS log_queries=1; +SHOW CREATE TABLE kek; +SHOW CREATE TABLE lol; +DROP TABLE kek; +DROP TABLE lol; diff --git a/tests/queries/0_stateless/02269_to_start_of_interval_overflow.reference b/tests/queries/0_stateless/02269_to_start_of_interval_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02269_to_start_of_interval_overflow.sql b/tests/queries/0_stateless/02269_to_start_of_interval_overflow.sql new file mode 100644 index 00000000000..84204834614 --- /dev/null +++ b/tests/queries/0_stateless/02269_to_start_of_interval_overflow.sql @@ -0,0 +1,6 @@ +select toStartOfInterval(toDateTime64('\0930-12-12 12:12:12.1234567', 3), toIntervalNanosecond(1024)); -- {serverError 407} + +SELECT + toDateTime64(-9223372036854775808, 1048575, toIntervalNanosecond(9223372036854775806), NULL), + toStartOfInterval(toDateTime64(toIntervalNanosecond(toIntervalNanosecond(257), toDateTime64(toStartOfInterval(toDateTime64(NULL)))), '', 100), toIntervalNanosecond(toStartOfInterval(toDateTime64(toIntervalNanosecond(NULL), NULL)), -1)), + toStartOfInterval(toDateTime64('\0930-12-12 12:12:12.1234567', 3), toIntervalNanosecond(1024)); -- {serverError 407} diff --git a/tests/queries/0_stateless/02285_hex_bin_support_more_types.reference b/tests/queries/0_stateless/02285_hex_bin_support_more_types.reference new file mode 100644 index 00000000000..53d97c3b7b2 --- /dev/null +++ b/tests/queries/0_stateless/02285_hex_bin_support_more_types.reference @@ -0,0 +1,30 @@ +174876E800 +174876E800 +64 +9C +64 +FF9C +64 +FFFFFF9C +64 +FFFFFFFFFFFFFF9C +174876E800 +174876E800 +174876E800 +FFFFFFFFFFFFFFFFFFFFFFE8B7891800 +FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE8B7891800 +0001011101001000011101101110100000000000 +0001011101001000011101101110100000000000 +01100100 +10011100 +01100100 +1111111110011100 +01100100 +11111111111111111111111110011100 +01100100 +1111111111111111111111111111111111111111111111111111111110011100 +0001011101001000011101101110100000000000 +0001011101001000011101101110100000000000 +0001011101001000011101101110100000000000 +11111111111111111111111111111111111111111111111111111111111111111111111111111111111111111110100010110111100010010001100000000000 +1111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111110100010110111100010010001100000000000 diff --git a/tests/queries/0_stateless/02285_hex_support_more_types.sql b/tests/queries/0_stateless/02285_hex_bin_support_more_types.sql similarity index 50% rename from tests/queries/0_stateless/02285_hex_support_more_types.sql rename to tests/queries/0_stateless/02285_hex_bin_support_more_types.sql index f07009c11cd..0efbd8dd34e 100644 --- a/tests/queries/0_stateless/02285_hex_support_more_types.sql +++ b/tests/queries/0_stateless/02285_hex_bin_support_more_types.sql @@ -28,3 +28,33 @@ SELECT hex(toInt128(-100000000000)) AS res; SELECT hex(toInt256(-100000000000)) AS res; +SELECT bin(toUInt128(100000000000)) AS res; + +SELECT bin(toUInt256(100000000000)) AS res; + +SELECT bin(toInt8(100)) AS res; + +SELECT bin(toInt8(-100)) AS res; + +SELECT bin(toInt16(100)) AS res; + +SELECT bin(toInt16(-100)) AS res; + +SELECT bin(toInt32(100)) AS res; + +SELECT bin(toInt32(-100)) AS res; + +SELECT bin(toInt64(100)) AS res; + +SELECT bin(toInt64(-100)) AS res; + +SELECT bin(toInt128(100000000000)) AS res; + +SELECT bin(toInt128(100000000000)) AS res; + +SELECT bin(toInt256(100000000000)) AS res; + +SELECT bin(toInt128(-100000000000)) AS res; + +SELECT bin(toInt256(-100000000000)) AS res; + diff --git a/tests/queries/0_stateless/02285_hex_support_more_types.reference b/tests/queries/0_stateless/02285_hex_support_more_types.reference deleted file mode 100644 index b016b9e3dec..00000000000 --- a/tests/queries/0_stateless/02285_hex_support_more_types.reference +++ /dev/null @@ -1,15 +0,0 @@ -174876E800 -174876E800 -64 -9C -64 -FF9C -64 -FFFFFF9C -64 -FFFFFFFFFFFFFF9C -174876E800 -174876E800 -174876E800 -FFFFFFFFFFFFFFFFFFFFFFE8B7891800 -FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE8B7891800 diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.reference b/tests/queries/0_stateless/02286_drop_filesystem_cache.reference new file mode 100644 index 00000000000..0dabc778274 --- /dev/null +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.reference @@ -0,0 +1,30 @@ +-- { echo } + +SET enable_filesystem_cache_on_write_operations=0; +DROP TABLE IF EXISTS test; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760; +SYSTEM DROP FILESYSTEM CACHE; +SELECT count() FROM system.filesystem_cache; +0 +INSERT INTO test SELECT number, toString(number) FROM numbers(100); +SELECT * FROM test FORMAT Null; +SELECT count() FROM system.filesystem_cache; +2 +SYSTEM DROP FILESYSTEM CACHE FORCE; +SELECT count() FROM system.filesystem_cache; +0 +SELECT * FROM test FORMAT Null; +SELECT count() FROM system.filesystem_cache; +1 +SYSTEM DROP FILESYSTEM CACHE './data'; -- { serverError 36 } +SELECT count() FROM system.filesystem_cache; +1 +DROP TABLE IF EXISTS test2; +CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_2', min_bytes_for_wide_part = 10485760; +INSERT INTO test2 SELECT number, toString(number) FROM numbers(100); +SELECT * FROM test2 FORMAT Null; +SELECT count() FROM system.filesystem_cache; +3 +SYSTEM DROP FILESYSTEM CACHE './s3_cache/'; +SELECT count() FROM system.filesystem_cache; +2 diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.sql b/tests/queries/0_stateless/02286_drop_filesystem_cache.sql new file mode 100644 index 00000000000..4c99c248dbc --- /dev/null +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.sql @@ -0,0 +1,34 @@ +-- Tags: no-parallel, no-fasttest, no-s3-storage + +-- { echo } + +SET enable_filesystem_cache_on_write_operations=0; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760; + +SYSTEM DROP FILESYSTEM CACHE; + +SELECT count() FROM system.filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100); + +SELECT * FROM test FORMAT Null; +SELECT count() FROM system.filesystem_cache; + +SYSTEM DROP FILESYSTEM CACHE FORCE; +SELECT count() FROM system.filesystem_cache; + +SELECT * FROM test FORMAT Null; +SELECT count() FROM system.filesystem_cache; + +SYSTEM DROP FILESYSTEM CACHE './data'; -- { serverError 36 } +SELECT count() FROM system.filesystem_cache; + +DROP TABLE IF EXISTS test2; +CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_2', min_bytes_for_wide_part = 10485760; +INSERT INTO test2 SELECT number, toString(number) FROM numbers(100); +SELECT * FROM test2 FORMAT Null; +SELECT count() FROM system.filesystem_cache; + +SYSTEM DROP FILESYSTEM CACHE './s3_cache/'; +SELECT count() FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference new file mode 100644 index 00000000000..5f330409b2a --- /dev/null +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference @@ -0,0 +1,10 @@ +CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(`1` Tuple(`2` Int32, `3` Int32), `4` Int32)\n)\nENGINE = Memory +{"t":{"1":{"2":2,"3":3},"4":4}} +2 3 4 +2 3 4 +2 3 4 +2 3 4 +Tuple(`1` Tuple(`2` Int8, `3` Int8), `4` Int8) +{"t":{"1":{"2":2,"3":3},"4":4}} +2 3 4 +(('value')) diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql b/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql new file mode 100644 index 00000000000..b5b0508deaa --- /dev/null +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql @@ -0,0 +1,45 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_tuple_numeric; + +CREATE TABLE t_tuple_numeric (t Tuple(`1` Tuple(`2` Int, `3` Int), `4` Int)) ENGINE = Memory; +SHOW CREATE TABLE t_tuple_numeric; + +INSERT INTO t_tuple_numeric VALUES (((2, 3), 4)); + +SET output_format_json_named_tuples_as_objects = 1; + +SELECT * FROM t_tuple_numeric FORMAT JSONEachRow; +SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric; +SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric; + +SELECT t.1.3 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX} +SELECT t.4 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX} +SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric; -- {serverError UNKNOWN_IDENTIFIER} + +DROP TABLE t_tuple_numeric; + +CREATE TABLE t_tuple_numeric (t Tuple(Tuple(Int, Int), Int)) ENGINE = Memory; + +INSERT INTO t_tuple_numeric VALUES (((2, 3), 4)); + +SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric; +SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric; + +DROP TABLE t_tuple_numeric; + +SET allow_experimental_object_type = 1; +CREATE TABLE t_tuple_numeric (t JSON) ENGINE = Memory; +INSERT INTO t_tuple_numeric FORMAT JSONEachRow {"t":{"1":{"2":2,"3":3},"4":4}} + +SELECT toTypeName(t) FROM t_tuple_numeric LIMIT 1; + +SELECT * FROM t_tuple_numeric FORMAT JSONEachRow; +SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric; + +DROP TABLE t_tuple_numeric; + +WITH + '{"1":{"key":"value"}}' AS data, + JSONExtract(data, 'Tuple("1" Tuple(key String))') AS parsed_json +SELECT parsed_json AS ssid diff --git a/tests/queries/1_stateful/00170_s3_cache.sql b/tests/queries/1_stateful/00170_s3_cache.sql index d4837a1a800..68e1c9f6c60 100644 --- a/tests/queries/1_stateful/00170_s3_cache.sql +++ b/tests/queries/1_stateful/00170_s3_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-settings -- { echo } diff --git a/utils/clickhouse-diagnostics/requirements.txt b/utils/clickhouse-diagnostics/requirements.txt index 1d2b6ef3916..d29acab2161 100644 --- a/utils/clickhouse-diagnostics/requirements.txt +++ b/utils/clickhouse-diagnostics/requirements.txt @@ -4,3 +4,4 @@ requests sqlparse tenacity xmltodict +pandas diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 9d1f8487169..ccc1d05c90c 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,4 +1,6 @@ +v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.4.20-lts 2022-04-26 v22.3.3.44-lts 2022-04-06 v22.3.2.2-lts 2022-03-17 v22.2.3.5-stable 2022-02-25 diff --git a/website/benchmark/versions/index.html b/website/benchmark/versions/index.html new file mode 100644 index 00000000000..da0702a04de --- /dev/null +++ b/website/benchmark/versions/index.html @@ -0,0 +1,52 @@ +{% extends 'templates/base.html' %} + +{% set title = 'Performance comparison of different ClickHouse versions' %} +{% set extra_js = [ + 'queries.js?' + rev_short, + 'results.js?' + rev_short, + '../benchmark.js?' + rev_short] +%} +{% set url = 'https://clickhouse.com/benchmark/versions/' %} +{% set no_footer = True %} + +{% block content %} +
+ +
+
+ + ClickHouse + +

{{ title }}

+
+
+ +
+
+
+ +
+
+

Relative query processing time (lower is better)

+
+
+
+ +
+
+

Full results

+
+
+
+ +
+
+

Comments

+

Hardware used to run tests: x86_64 AWS m5.8xlarge Ubuntu 20.04.

+

Old versions had no support of this features/syntax: toISOWeek, SUBSTRING, EXTRACT, WITH. Queries were changed. All versions runs use same queries.

+

Star Schema Benchmark Patrick O'Neil, Elizabeth (Betty) O'Neil and Xuedong Chen. "The Star Schema Benchmark," Online Publication of Database Generation program., January 2007. http://www.cs.umb.edu/~poneil/StarSchemaB.pdf

+
+

+
+
+{% endblock %} diff --git a/website/benchmark/versions/queries.js b/website/benchmark/versions/queries.js new file mode 100644 index 00000000000..f2de5a3c95a --- /dev/null +++ b/website/benchmark/versions/queries.js @@ -0,0 +1,291 @@ +var current_data_size = 0; + +var current_systems = []; + +var queries = + [ + { + "query": "SELECT machine_name, MIN(cpu) AS cpu_min, MAX(cpu) AS cpu_max, AVG(cpu) AS cpu_avg, MIN(net_in) AS net_in_min, MAX(net_in) AS net_in_max, AVG(net_in) AS net_in_avg, MIN(net_out) AS net_out_min, MAX(net_out) AS net_out_max, AVG(net_out) AS net_out_avg FROM ( SELECT machine_name, ifNull(cpu_user, 0.0) AS cpu, ifNull(bytes_in, 0.0) AS net_in, ifNull(bytes_out, 0.0) AS net_out FROM mgbench.logs1 WHERE machine_name IN ('anansi','aragog','urd') AND log_time >= toDateTime('2017-01-11 00:00:00')) AS r GROUP BY machine_name;", + "comment": "Q1.1: What is the CPU/network utilization for each web server since midnight?", + }, + { + "query": "SELECT machine_name, log_time FROM mgbench.logs1 WHERE (machine_name LIKE 'cslab%' OR machine_name LIKE 'mslab%') AND load_one IS NULL AND log_time >= toDateTime('2017-01-10 00:00:00') ORDER BY machine_name, log_time;", + "comment": "Q1.2: Which computer lab machines have been offline in the past day?", + }, + { + "query": "SELECT dt, hr, AVG(load_fifteen) AS load_fifteen_avg, AVG(load_five) AS load_five_avg, AVG(load_one) AS load_one_avg, AVG(mem_free) AS mem_free_avg, AVG(swap_free) AS swap_free_avg FROM ( SELECT CAST(log_time AS DATE) AS dt, toHour(log_time) AS hr, load_fifteen, load_five, load_one, mem_free, swap_free FROM mgbench.logs1 WHERE machine_name = 'babbage' AND load_fifteen IS NOT NULL AND load_five IS NOT NULL AND load_one IS NOT NULL AND mem_free IS NOT NULL AND swap_free IS NOT NULL AND log_time >= toDateTime('2017-01-01 00:00:00')) AS r GROUP BY dt, hr ORDER BY dt, hr;", + "comment": "Q1.3: What are the hourly average metrics during the past 10 days for a specific workstation?", + }, + { + "query": "SELECT machine_name, COUNT(*) AS spikes FROM mgbench.logs1 WHERE machine_group = 'Servers' AND cpu_wio > 0.99 AND log_time >= toDateTime('2016-12-01 00:00:00') AND log_time < toDateTime('2017-01-01 00:00:00') GROUP BY machine_name ORDER BY spikes DESC LIMIT 10;", + "comment": "Q1.4: Over 1 month, how often was each server blocked on disk I/O?", + }, + { + "query": "SELECT machine_name, dt, MIN(mem_free) AS mem_free_min FROM ( SELECT machine_name, CAST(log_time AS DATE) AS dt, mem_free FROM mgbench.logs1 WHERE machine_group = 'DMZ' AND mem_free IS NOT NULL ) AS r GROUP BY machine_name, dt HAVING MIN(mem_free) < 10000 ORDER BY machine_name, dt;", + "comment": "Q1.5: Which externally reachable VMs have run low on memory?", + }, + { + "query": "SELECT dt, hr, SUM(net_in) AS net_in_sum, SUM(net_out) AS net_out_sum, SUM(net_in) + SUM(net_out) AS both_sum FROM ( SELECT CAST(log_time AS DATE) AS dt, toHour(log_time) AS hr, ifNull(bytes_in, 0.0) / 1000000000.0 AS net_in, ifNull(bytes_out, 0.0) / 1000000000.0 AS net_out FROM mgbench.logs1 WHERE machine_name IN ('allsorts','andes','bigred','blackjack','bonbon','cadbury','chiclets','cotton','crows','dove','fireball','hearts','huey','lindt','milkduds','milkyway','mnm','necco','nerds','orbit','peeps','poprocks','razzles','runts','smarties','smuggler','spree','stride','tootsie','trident','wrigley','york') ) AS r GROUP BY dt, hr ORDER BY both_sum DESC LIMIT 10;", + "comment": "Q1.6: What is the total hourly network traffic across all file servers?", + }, + { + "query": "SELECT * FROM mgbench.logs2 WHERE status_code >= 500 AND log_time >= toDateTime('2012-12-18 00:00:00') ORDER BY log_time;", + "comment": "Q2.1: Which requests have caused server errors within the past 2 weeks?", + }, + { + "query": "SELECT * FROM mgbench.logs2 WHERE status_code >= 200 AND status_code < 300 AND request LIKE '%/etc/passwd%' AND log_time >= toDateTime('2012-05-06 00:00:00') AND log_time < toDateTime('2012-05-20 00:00:00');", + "comment": "Q2.3: What was the average path depth for top-level requests in the past month?", + }, + { + "query": "SELECT top_level, AVG(length(request) - length(replaceOne(request, '/',''))) AS depth_avg FROM ( SELECT substring(request, 1, len) AS top_level, request FROM ( SELECT position('/', substring(request, 2)) AS len, request FROM mgbench.logs2 WHERE status_code >= 200 AND status_code < 300 AND log_time >= toDateTime('2012-12-01 00:00:00')) AS r WHERE len > 0 ) AS s WHERE top_level IN ('/about','/courses','/degrees','/events','/grad','/industry','/news','/people','/publications','/research','/teaching','/ugrad') GROUP BY top_level ORDER BY top_level;", + "comment": "Q2.2: During a specific 2-week period, was the user password file leaked?", + }, + { + "query": "SELECT client_ip, COUNT(*) AS num_requests FROM mgbench.logs2 WHERE log_time >= toDateTime('2012-10-01 00:00:00') GROUP BY client_ip HAVING COUNT(*) >= 100000 ORDER BY num_requests DESC;", + "comment": "Q2.4: During the last 3 months, which clients have made an excessive number of requests?", + }, + { + "query": "SELECT dt, COUNT(DISTINCT client_ip) FROM ( SELECT CAST(log_time AS DATE) AS dt, client_ip FROM mgbench.logs2) AS r GROUP BY dt ORDER BY dt;", + "comment": "Q2.5: What are the daily unique visitors?", + }, + { + "query": "SELECT AVG(transfer) / 125000000.0 AS transfer_avg, MAX(transfer) / 125000000.0 AS transfer_max FROM ( SELECT log_time, SUM(object_size) AS transfer FROM mgbench.logs2 GROUP BY log_time) AS r;", + "comment": "Q2.6: What are the average and maximum data transfer rates (Gbps)?", + }, + { + "query": "SELECT * FROM mgbench.logs3 WHERE event_type = 'temperature' AND event_value <= 32.0 AND log_time >= '2019-11-29 17:00:00';", + "comment": "Q3.1: Did the indoor temperature reach freezing over the weekend?", + }, + { + "query": "SELECT device_name, device_floor, COUNT(*) AS ct FROM mgbench.logs3 WHERE event_type = 'door_open' AND log_time >= '2019-06-01 00:00:00' GROUP BY device_name, device_floor ORDER BY ct DESC;", + "comment": "Q3.4: Over the past 6 months, how frequently were each door opened?", + }, + { + "query": "SELECT yr, mo, SUM(coffee_hourly_avg) AS coffee_monthly_sum, AVG(coffee_hourly_avg) AS coffee_monthly_avg, SUM(printer_hourly_avg) AS printer_monthly_sum, AVG(printer_hourly_avg) AS printer_monthly_avg, SUM(projector_hourly_avg) AS projector_monthly_sum, AVG(projector_hourly_avg) AS projector_monthly_avg, SUM(vending_hourly_avg) AS vending_monthly_sum, AVG(vending_hourly_avg) AS vending_monthly_avg FROM ( SELECT dt, yr, mo, hr, AVG(coffee) AS coffee_hourly_avg, AVG(printer) AS printer_hourly_avg, AVG(projector) AS projector_hourly_avg, AVG(vending) AS vending_hourly_avg FROM ( SELECT CAST(log_time AS DATE) AS dt, toYear(log_time) AS yr, EXTRACT(MONTH FROM log_time) AS mo, toHour(log_time) AS hr, CASE WHEN device_name LIKE 'coffee%' THEN event_value END AS coffee, CASE WHEN device_name LIKE 'printer%' THEN event_value END AS printer, CASE WHEN device_name LIKE 'projector%' THEN event_value END AS projector, CASE WHEN device_name LIKE 'vending%' THEN event_value END AS vending FROM mgbench.logs3 WHERE device_type = 'meter' ) AS r GROUP BY dt, yr, mo, hr ) AS s GROUP BY yr, mo ORDER BY yr, mo;", + "comment": " -- Q3.6: For each device category, what are the monthly power consumption metrics?", + }, + { + "query": "SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE F_YEAR = 1993 AND LO_DISCOUNT BETWEEN 1 AND 3 AND LO_QUANTITY < 25;", + "comment": " -- Q1.1", + }, + { + "query": "SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE toYYYYMM(LO_ORDERDATE) = 199401 AND LO_DISCOUNT BETWEEN 4 AND 6 AND LO_QUANTITY BETWEEN 26 AND 35;", + "comment": " -- Q1.2", + }, + { + "query": "SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE toRelativeWeekNum(LO_ORDERDATE) - toRelativeWeekNum(toDate('1994-01-01')) = 6 AND F_YEAR = 1994 AND LO_DISCOUNT BETWEEN 5 AND 7 AND LO_QUANTITY BETWEEN 26 AND 35;", + "comment": " -- Q1.3", + }, + { + "query": "SELECT sum(LO_REVENUE), F_YEAR AS year, P_BRAND FROM lineorder_flat WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' GROUP BY year, P_BRAND ORDER BY year, P_BRAND;", + "comment": " -- Q2.1", + }, + { + "query": "SELECT sum(LO_REVENUE), F_YEAR AS year, P_BRAND FROM lineorder_flat WHERE P_BRAND >= 'MFGR#2221' AND P_BRAND <= 'MFGR#2228' AND S_REGION = 'ASIA' GROUP BY year, P_BRAND ORDER BY year, P_BRAND;", + "comment": " -- Q2.2", + }, + { + "query": "SELECT sum(LO_REVENUE), F_YEAR AS year, P_BRAND FROM lineorder_flat WHERE P_BRAND = 'MFGR#2239' AND S_REGION = 'EUROPE' GROUP BY year, P_BRAND ORDER BY year, P_BRAND;", + "comment": " -- Q2.3", + }, + { + "query": "SELECT C_NATION, S_NATION, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_REGION = 'ASIA' AND S_REGION = 'ASIA' AND year >= 1992 AND year <= 1997 GROUP BY C_NATION, S_NATION, year ORDER BY year ASC, revenue DESC;", + "comment": " -- Q3.1", + }, + { + "query": "SELECT C_CITY, S_CITY, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_NATION = 'UNITED STATES' AND S_NATION = 'UNITED STATES' AND year >= 1992 AND year <= 1997 GROUP BY C_CITY, S_CITY, year ORDER BY year ASC, revenue DESC;", + "comment": " -- Q3.2", + }, + { + "query": "SELECT C_CITY, S_CITY, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND year >= 1992 AND year <= 1997 GROUP BY C_CITY, S_CITY, year ORDER BY year ASC, revenue DESC;", + "comment": " -- Q3.3", + }, + { + "query": "SELECT C_CITY, S_CITY, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND toYYYYMM(LO_ORDERDATE) = 199712 GROUP BY C_CITY, S_CITY, year ORDER BY year ASC, revenue DESC;", + "comment": " -- Q3.4", + }, + { + "query": "SELECT F_YEAR AS year, C_NATION, sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year, C_NATION ORDER BY year ASC, C_NATION ASC;", + "comment": " -- Q4.1", + }, + { + "query": "SELECT F_YEAR AS year, S_NATION, P_CATEGORY, sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (year = 1997 OR year = 1998) AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year, S_NATION, P_CATEGORY ORDER BY year ASC, S_NATION ASC, P_CATEGORY ASC;", + "comment": " -- Q4.2", + }, + { + "query": "SELECT F_YEAR AS year, S_CITY, P_BRAND, sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE S_NATION = 'UNITED STATES' AND (year = 1997 OR year = 1998) AND P_CATEGORY = 'MFGR#14' GROUP BY year, S_CITY, P_BRAND ORDER BY year ASC, S_CITY ASC, P_BRAND ASC;", + "comment": " -- Q4.", + }, + { + "query": "SELECT count() FROM hits", + "comment": "", + }, + { + "query": "SELECT count() FROM hits WHERE AdvEngineID != 0", + "comment": "", + }, + { + "query": "SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM hits", + "comment": "", + }, + { + "query": "SELECT sum(UserID) FROM hits", + "comment": "", + }, + { + "query": "SELECT uniq(UserID) FROM hits", + "comment": "", + }, + { + "query": "SELECT uniq(SearchPhrase) FROM hits", + "comment": "", + }, + { + "query": "SELECT min(EventDate), max(EventDate) FROM hits", + "comment": "", + }, + { + "query": "SELECT AdvEngineID, count() FROM hits WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC", + "comment": "", + }, + { + "query": "SELECT RegionID, uniq(UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM hits WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchPhrase, uniq(UserID) AS u FROM hits WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchEngineID, SearchPhrase, count() AS c FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT UserID, count() FROM hits GROUP BY UserID ORDER BY count() DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT UserID, SearchPhrase, count() FROM hits GROUP BY UserID, SearchPhrase LIMIT 10", + "comment": "", + }, + { + "query": "SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT UserID FROM hits WHERE UserID = 12345678901234567890", + "comment": "", + }, + { + "query": "SELECT count() FROM hits WHERE URL LIKE '%metrika%'", + "comment": "", + }, + { + "query": "SELECT SearchPhrase, any(URL), count() AS c FROM hits WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM hits WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT * FROM hits WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10", + "comment": "", + }, + { + "query": "SELECT SearchPhrase FROM hits WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10", + "comment": "", + }, + { + "query": "SELECT CounterID, avg(length(URL)) AS l, count() AS c FROM hits WHERE URL != '' GROUP BY CounterID HAVING c > 100000 ORDER BY l DESC LIMIT 25", + "comment": "", + }, + { + "query": "SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM hits WHERE Referer != '' GROUP BY key HAVING c > 100000 ORDER BY l DESC LIMIT 25", + "comment": "", + }, + { + "query": "SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM hits", + "comment": "", + }, + { + "query": "SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT URL, count() AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT 1, URL, count() AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT ClientIP AS x, x - 1, x - 2, x - 3, count() AS c FROM hits GROUP BY x, x - 1, x - 2, x - 3 ORDER BY c DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT Title, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10", + "comment": "", + }, + { + "query": "SELECT URL, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000", + "comment": "", + }, + { + "query": "SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000", + "comment": "", + }, + { + "query": "SELECT URLHash, EventDate, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://yandex.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100", + "comment": "", + }, + { + "query": "SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-31') AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://yandex.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000;", + "comment": "", + }, + { + "query": "SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= toDate('2013-07-01') AND EventDate <= toDate('2013-07-02') AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute;", + "comment": "", + } + ]; diff --git a/website/benchmark/versions/results/1.1.54378.json b/website/benchmark/versions/results/1.1.54378.json new file mode 100644 index 00000000000..987d4d50231 --- /dev/null +++ b/website/benchmark/versions/results/1.1.54378.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-04-16 1.1.54378", + "system_full": "ClickHouse 1.1.54378 2018-04-16", + "version": "1.1.54378", + "kind": "", + "comments": "", + "result": + [ + +[0.099, 0.017, 0.014], +[0.176, 0.005, 0.005], +[0.818, 0.018, 0.016], +[0.163, 0.011, 0.011], +[0.264, 0.039, 0.031], +[1.025, 0.026, 0.024], +[0.076, 0.004, 0.003], +[0.335, 0.018, 0.018], +[0.511, 0.034, 0.034], +[0.541, 0.090, 0.088], +[1.881, 0.506, 0.497], +[1.515, 0.546, 0.495], +[0.033, 0.003, 0.003], +[0.034, 0.016, 0.016], +[0.123, 0.105, 0.103], +[3.318, 0.090, 0.088], +[2.609, 0.146, 0.145], +[3.887, 0.067, 0.066], +[1.417, 0.067, 0.065], +[7.034, 0.332, 0.328], +[4.289, 0.177, 0.176], +[1.773, 0.204, 0.214], +[63.407, 2.564, 2.490], +[42.496, 2.011, 1.993], +[4.502, 0.218, 0.216], +[1.952, 0.120, 0.121], +[0.474, 0.059, 0.062], +[6.655, 0.106, 0.101], +[0.044, 0.010, 0.010], +[0.042, 0.007, 0.007], +[0.480, 0.060, 0.056], +[1.684, 0.042, 0.039], +[1.721, 0.091, 0.091], +[2.587, 0.207, 0.206], +[0.067, 0.044, 0.044], +[0.027, 0.008, 0.007], +[2.282, 0.428, 0.425], +[2.273, 0.501, 0.500], +[1.072, 0.133, 0.133], +[1.717, 0.163, 0.163], +[2.604, 0.431, 0.418], +[3.794, 0.546, 0.545], +[2.398, 0.538, 0.585], +[1.272, 0.564, 0.534], +[4.313, 1.445, 1.481], +[3.711, 0.860, 0.831], +[8.169, 3.717, 3.386], +[0.962, 0.029, 0.028], +[18.979, 0.538, 0.533], +[20.836, 0.642, 0.667], +[38.657, 1.461, 1.388], +[29.873, 0.735, 0.675], +[4.756, 0.189, 0.188], +[2.304, 0.161, 0.161], +[5.272, 0.196, 0.195], +[18.616, 0.557, 0.552], +[15.437, 1.136, 1.125], +[2.560, 2.523, 2.497], +[3.912, 0.561, 0.557], +[9.879, 0.774, 0.793], +[9.665, 4.632, 4.677], +[18.892, 2.283, 2.142], +[18.894, 2.264, 2.160], +[0.953, 0.921, 0.915], +[0.099, 0.075, 0.076], +[0.055, 0.038, 0.037], +[0.055, 0.033, 0.031], +[0.202, 0.159, 0.159], +[0.038, 0.011, 0.011], +[0.023, 0.009, 0.009], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/1.1.54394.json b/website/benchmark/versions/results/1.1.54394.json new file mode 100644 index 00000000000..b88336c2fcd --- /dev/null +++ b/website/benchmark/versions/results/1.1.54394.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-07-12 1.1.54394", + "system_full": "ClickHouse 1.1.54394 2018-07-12", + "version": "1.1.54394", + "kind": "", + "comments": "", + "result": + [ + +[0.141, 0.016, 0.014], +[0.152, 0.005, 0.005], +[0.820, 0.018, 0.016], +[0.161, 0.011, 0.011], +[0.266, 0.033, 0.032], +[1.024, 0.026, 0.024], +[0.076, 0.004, 0.003], +[0.335, 0.017, 0.017], +[0.508, 0.033, 0.032], +[0.553, 0.090, 0.090], +[1.839, 0.501, 0.497], +[1.519, 0.490, 0.513], +[0.035, 0.003, 0.003], +[0.033, 0.015, 0.015], +[0.116, 0.135, 0.105], +[3.324, 0.088, 0.085], +[2.614, 0.145, 0.144], +[3.882, 0.068, 0.066], +[1.421, 0.067, 0.066], +[7.042, 0.315, 0.312], +[4.326, 0.170, 0.168], +[1.810, 0.205, 0.198], +[63.409, 2.479, 2.463], +[42.496, 1.974, 1.963], +[4.501, 0.223, 0.211], +[1.947, 0.115, 0.114], +[0.488, 0.058, 0.060], +[6.636, 0.103, 0.099], +[0.050, 0.010, 0.010], +[0.043, 0.007, 0.006], +[0.482, 0.055, 0.055], +[1.669, 0.040, 0.041], +[1.713, 0.093, 0.091], +[2.563, 0.201, 0.202], +[0.064, 0.041, 0.041], +[0.028, 0.007, 0.007], +[2.305, 0.425, 0.424], +[2.270, 0.502, 0.506], +[1.070, 0.139, 0.138], +[1.689, 0.166, 0.165], +[2.616, 0.447, 0.459], +[3.835, 0.562, 0.540], +[2.452, 0.581, 0.560], +[1.282, 0.540, 0.544], +[4.321, 1.565, 1.467], +[3.678, 0.811, 0.812], +[8.241, 3.565, 3.693], +[0.961, 0.028, 0.027], +[18.976, 0.509, 0.482], +[20.838, 0.601, 0.737], +[38.809, 1.514, 1.353], +[29.889, 0.690, 0.628], +[4.546, 0.190, 0.188], +[2.314, 0.163, 0.165], +[5.264, 0.197, 0.193], +[18.615, 0.527, 0.516], +[15.439, 1.156, 1.138], +[2.552, 2.521, 2.509], +[3.918, 0.566, 0.557], +[9.883, 0.790, 0.794], +[9.662, 4.592, 4.830], +[18.976, 2.262, 2.254], +[18.926, 2.132, 2.173], +[0.945, 0.918, 0.921], +[0.099, 0.074, 0.074], +[0.053, 0.038, 0.038], +[0.052, 0.030, 0.030], +[0.199, 0.156, 0.155], +[0.038, 0.011, 0.011], +[0.023, 0.009, 0.009], +[0.011, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.01.0.json b/website/benchmark/versions/results/18.01.0.json new file mode 100644 index 00000000000..15ed8360b47 --- /dev/null +++ b/website/benchmark/versions/results/18.01.0.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-07-20 18.1", + "system_full": "ClickHouse 18.1.0 2018-07-20", + "version": "18.1.0", + "kind": "", + "comments": "", + "result": + [ + +[0.130, 0.015, 0.013], +[0.139, 0.005, 0.005], +[0.822, 0.017, 0.016], +[0.160, 0.011, 0.010], +[0.263, 0.051, 0.059], +[0.994, 0.025, 0.024], +[0.076, 0.004, 0.004], +[0.334, 0.017, 0.017], +[0.507, 0.031, 0.030], +[0.553, 0.087, 0.084], +[1.867, 0.486, 0.484], +[1.528, 0.493, 0.493], +[0.034, 0.003, 0.003], +[0.033, 0.015, 0.015], +[0.117, 0.099, 0.099], +[3.353, 0.087, 0.086], +[2.632, 0.145, 0.144], +[3.913, 0.067, 0.065], +[1.433, 0.063, 0.062], +[7.058, 0.303, 0.301], +[4.355, 0.163, 0.161], +[1.826, 0.187, 0.212], +[63.427, 2.395, 2.319], +[42.481, 1.974, 1.956], +[4.485, 0.213, 0.218], +[1.945, 0.123, 0.116], +[0.445, 0.056, 0.055], +[6.633, 0.102, 0.099], +[0.047, 0.010, 0.010], +[0.044, 0.007, 0.007], +[0.479, 0.055, 0.055], +[1.695, 0.040, 0.039], +[1.731, 0.092, 0.095], +[2.547, 0.197, 0.192], +[0.061, 0.041, 0.041], +[0.025, 0.007, 0.007], +[2.289, 0.429, 0.426], +[2.240, 0.506, 0.502], +[1.062, 0.128, 0.126], +[1.688, 0.155, 0.155], +[2.571, 0.456, 0.423], +[3.814, 0.538, 0.546], +[2.467, 0.563, 0.531], +[1.263, 0.545, 0.553], +[4.303, 1.526, 1.539], +[3.667, 0.911, 0.922], +[8.280, 3.588, 3.559], +[0.938, 0.028, 0.027], +[18.975, 0.519, 0.513], +[20.851, 0.588, 0.724], +[38.765, 1.508, 1.345], +[29.904, 0.684, 0.648], +[4.591, 0.180, 0.175], +[2.350, 0.151, 0.150], +[5.295, 0.185, 0.185], +[18.635, 0.520, 0.512], +[15.431, 1.169, 1.144], +[2.543, 2.542, 2.504], +[3.918, 0.545, 0.540], +[9.879, 0.781, 0.765], +[9.687, 4.567, 4.636], +[18.949, 2.314, 2.136], +[18.946, 2.168, 2.227], +[0.950, 0.902, 0.934], +[0.098, 0.073, 0.075], +[0.056, 0.038, 0.037], +[0.055, 0.030, 0.030], +[0.205, 0.157, 0.155], +[0.037, 0.011, 0.011], +[0.022, 0.009, 0.009], +[0.011, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.04.0.json b/website/benchmark/versions/results/18.04.0.json new file mode 100644 index 00000000000..65f41e6b9a4 --- /dev/null +++ b/website/benchmark/versions/results/18.04.0.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-07-28 18.4", + "system_full": "ClickHouse 18.4.0 2018-07-28", + "version": "18.4.0", + "kind": "", + "comments": "", + "result": + [ + +[0.145, 0.015, 0.013], +[0.042, 0.006, 0.005], +[0.662, 0.016, 0.015], +[0.161, 0.011, 0.010], +[0.268, 0.030, 0.029], +[1.026, 0.025, 0.023], +[0.076, 0.004, 0.003], +[0.334, 0.018, 0.018], +[0.501, 0.031, 0.031], +[0.551, 0.089, 0.086], +[1.893, 0.495, 0.489], +[1.521, 0.506, 0.479], +[0.033, 0.003, 0.003], +[0.032, 0.015, 0.015], +[0.163, 0.131, 0.103], +[3.347, 0.090, 0.087], +[2.623, 0.145, 0.146], +[3.895, 0.068, 0.067], +[1.421, 0.063, 0.064], +[7.053, 0.307, 0.304], +[4.320, 0.159, 0.159], +[1.811, 0.194, 0.200], +[63.418, 2.390, 2.315], +[42.482, 1.944, 1.934], +[4.507, 0.213, 0.210], +[1.947, 0.116, 0.119], +[0.481, 0.057, 0.056], +[6.653, 0.100, 0.098], +[0.045, 0.010, 0.010], +[0.043, 0.007, 0.007], +[0.481, 0.054, 0.055], +[1.683, 0.043, 0.044], +[1.732, 0.092, 0.091], +[2.570, 0.192, 0.193], +[0.056, 0.039, 0.039], +[0.025, 0.007, 0.007], +[2.313, 0.428, 0.426], +[2.253, 0.497, 0.498], +[1.076, 0.121, 0.121], +[1.728, 0.148, 0.149], +[2.579, 0.444, 0.414], +[3.796, 0.532, 0.531], +[2.427, 0.556, 0.563], +[1.267, 0.544, 0.542], +[4.314, 1.538, 1.516], +[3.662, 0.804, 0.869], +[8.244, 3.696, 3.698], +[0.956, 0.028, 0.027], +[18.975, 0.514, 0.507], +[20.853, 0.583, 0.726], +[38.834, 1.380, 1.363], +[29.884, 0.675, 0.640], +[4.554, 0.175, 0.173], +[2.360, 0.147, 0.145], +[5.300, 0.179, 0.179], +[18.661, 0.514, 0.505], +[15.432, 1.161, 1.153], +[2.528, 2.542, 2.512], +[3.929, 0.543, 0.533], +[9.838, 0.765, 0.761], +[9.589, 4.506, 4.642], +[18.961, 2.245, 2.185], +[18.935, 2.127, 2.154], +[0.950, 0.913, 0.889], +[0.098, 0.072, 0.072], +[0.054, 0.037, 0.037], +[0.054, 0.030, 0.030], +[0.203, 0.153, 0.158], +[0.037, 0.011, 0.011], +[0.023, 0.009, 0.009], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.05.1.json b/website/benchmark/versions/results/18.05.1.json new file mode 100644 index 00000000000..3d58829d607 --- /dev/null +++ b/website/benchmark/versions/results/18.05.1.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-07-31 18.5", + "system_full": "ClickHouse 18.5.1 2018-07-31", + "version": "18.5.1", + "kind": "", + "comments": "", + "result": + [ + +[0.130, 0.015, 0.013], +[0.147, 0.005, 0.005], +[0.819, 0.017, 0.015], +[0.162, 0.011, 0.011], +[0.263, 0.040, 0.054], +[1.003, 0.025, 0.023], +[0.076, 0.004, 0.003], +[0.335, 0.017, 0.017], +[0.509, 0.032, 0.031], +[0.551, 0.095, 0.087], +[1.829, 0.489, 0.496], +[1.510, 0.518, 0.492], +[0.035, 0.003, 0.003], +[0.032, 0.015, 0.015], +[0.122, 0.127, 0.101], +[3.329, 0.090, 0.087], +[2.609, 0.143, 0.141], +[3.895, 0.067, 0.066], +[1.433, 0.064, 0.064], +[7.038, 0.307, 0.305], +[4.335, 0.160, 0.160], +[1.817, 0.216, 0.214], +[63.378, 2.378, 2.313], +[42.494, 1.940, 1.929], +[4.510, 0.212, 0.209], +[1.955, 0.119, 0.117], +[0.496, 0.058, 0.056], +[6.639, 0.100, 0.097], +[0.046, 0.010, 0.010], +[0.044, 0.007, 0.006], +[0.525, 0.055, 0.056], +[1.739, 0.043, 0.041], +[1.749, 0.091, 0.091], +[2.566, 0.193, 0.189], +[0.061, 0.041, 0.041], +[0.026, 0.007, 0.007], +[2.331, 0.427, 0.426], +[2.279, 0.504, 0.502], +[1.054, 0.122, 0.121], +[1.735, 0.149, 0.150], +[2.649, 0.426, 0.415], +[3.799, 0.552, 0.564], +[2.437, 0.573, 0.522], +[1.255, 0.532, 0.556], +[4.340, 1.534, 1.446], +[3.647, 0.811, 0.846], +[8.212, 3.519, 3.542], +[0.951, 0.028, 0.027], +[18.978, 0.661, 0.508], +[20.848, 0.583, 0.575], +[38.808, 1.432, 1.348], +[29.875, 0.679, 0.651], +[4.778, 0.176, 0.174], +[2.370, 0.148, 0.146], +[5.302, 0.186, 0.178], +[18.666, 0.522, 0.514], +[15.419, 1.157, 1.141], +[2.527, 2.526, 2.513], +[3.948, 0.539, 0.544], +[9.857, 0.772, 0.750], +[9.827, 4.565, 4.514], +[18.957, 2.301, 2.151], +[18.952, 2.147, 2.239], +[0.940, 0.897, 0.907], +[0.099, 0.072, 0.073], +[0.055, 0.037, 0.037], +[0.054, 0.030, 0.029], +[0.193, 0.155, 0.152], +[0.035, 0.011, 0.010], +[0.023, 0.009, 0.009], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.06.0.json b/website/benchmark/versions/results/18.06.0.json new file mode 100644 index 00000000000..d033aabc4f2 --- /dev/null +++ b/website/benchmark/versions/results/18.06.0.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-08-01 18.6", + "system_full": "ClickHouse 18.6.0 2018-08-01", + "version": "18.6.0", + "kind": "", + "comments": "", + "result": + [ + +[0.125, 0.014, 0.013], +[0.156, 0.005, 0.005], +[0.818, 0.016, 0.015], +[0.162, 0.011, 0.011], +[0.265, 0.044, 0.031], +[1.023, 0.025, 0.023], +[0.076, 0.004, 0.004], +[0.335, 0.019, 0.017], +[0.508, 0.032, 0.031], +[0.551, 0.088, 0.086], +[1.844, 0.493, 0.491], +[1.520, 0.485, 0.492], +[0.035, 0.003, 0.003], +[0.033, 0.015, 0.015], +[0.155, 0.109, 0.129], +[3.314, 0.090, 0.088], +[2.611, 0.144, 0.142], +[3.902, 0.066, 0.065], +[1.423, 0.064, 0.062], +[7.049, 0.304, 0.330], +[4.330, 0.159, 0.158], +[1.834, 0.193, 0.176], +[63.516, 2.328, 2.310], +[42.645, 1.945, 1.913], +[4.521, 0.212, 0.217], +[1.923, 0.112, 0.114], +[0.479, 0.056, 0.055], +[6.627, 0.101, 0.097], +[0.047, 0.010, 0.009], +[0.043, 0.007, 0.006], +[0.482, 0.058, 0.055], +[1.693, 0.043, 0.043], +[1.744, 0.098, 0.093], +[2.565, 0.192, 0.192], +[0.059, 0.040, 0.040], +[0.026, 0.007, 0.007], +[2.325, 0.425, 0.426], +[2.265, 0.501, 0.499], +[1.043, 0.122, 0.122], +[1.718, 0.151, 0.150], +[2.627, 0.425, 0.441], +[3.801, 0.530, 0.528], +[2.398, 0.525, 0.520], +[1.238, 0.523, 0.543], +[4.345, 1.505, 1.513], +[3.667, 0.851, 0.852], +[8.282, 3.515, 3.493], +[0.962, 0.028, 0.028], +[18.978, 0.518, 0.514], +[20.849, 0.814, 0.578], +[38.796, 1.382, 1.331], +[29.874, 0.665, 0.650], +[4.545, 0.181, 0.174], +[2.356, 0.147, 0.145], +[5.302, 0.180, 0.179], +[18.680, 0.516, 0.509], +[15.430, 1.162, 1.158], +[2.515, 2.502, 2.538], +[3.927, 0.533, 0.525], +[9.878, 0.769, 0.767], +[9.608, 4.694, 4.443], +[19.021, 2.138, 2.202], +[18.958, 2.174, 2.204], +[0.956, 0.899, 0.929], +[0.099, 0.074, 0.073], +[0.055, 0.037, 0.037], +[0.051, 0.031, 0.030], +[0.203, 0.157, 0.156], +[0.040, 0.011, 0.011], +[0.024, 0.009, 0.009], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.10.3.json b/website/benchmark/versions/results/18.10.3.json new file mode 100644 index 00000000000..a29bdff3f7a --- /dev/null +++ b/website/benchmark/versions/results/18.10.3.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-08-13 18.10", + "system_full": "ClickHouse 18.10.3 2018-08-13", + "version": "18.10.3", + "kind": "", + "comments": "", + "result": + [ + +[0.138, 0.015, 0.013], +[0.119, 0.006, 0.006], +[0.820, 0.017, 0.016], +[0.161, 0.011, 0.011], +[0.270, 0.044, 0.046], +[1.009, 0.025, 0.024], +[0.076, 0.004, 0.004], +[0.335, 0.017, 0.017], +[0.505, 0.032, 0.030], +[0.554, 0.087, 0.090], +[1.887, 0.524, 0.504], +[1.530, 0.485, 0.490], +[0.034, 0.003, 0.003], +[0.032, 0.022, 0.020], +[0.174, 0.132, 0.134], +[3.331, 0.091, 0.087], +[2.632, 0.154, 0.152], +[3.881, 0.074, 0.072], +[1.421, 0.067, 0.067], +[7.055, 0.298, 0.291], +[4.370, 0.161, 0.160], +[1.828, 0.417, 0.423], +[63.020, 2.017, 1.993], +[42.473, 1.863, 1.855], +[4.489, 0.220, 0.215], +[1.947, 0.158, 0.208], +[0.357, 0.078, 0.092], +[6.607, 0.115, 0.103], +[0.043, 0.011, 0.010], +[0.043, 0.008, 0.008], +[0.483, 0.060, 0.061], +[1.687, 0.042, 0.042], +[1.732, 0.093, 0.093], +[2.572, 0.194, 0.192], +[0.066, 0.048, 0.048], +[0.028, 0.008, 0.008], +[2.290, 0.435, 0.438], +[2.276, 0.503, 0.506], +[1.054, 0.127, 0.126], +[1.702, 0.160, 0.165], +[2.545, 0.429, 0.464], +[3.846, 0.551, 0.535], +[2.413, 0.575, 0.554], +[1.244, 0.539, 0.582], +[4.310, 1.570, 1.539], +[3.635, 0.910, 0.868], +[8.212, 4.811, 4.268], +[0.947, 0.028, 0.027], +[18.972, 0.518, 0.506], +[20.843, 0.588, 0.572], +[38.776, 1.377, 1.363], +[29.917, 0.670, 0.630], +[4.779, 0.186, 0.182], +[2.330, 0.163, 0.153], +[5.283, 0.193, 0.187], +[18.637, 0.544, 0.518], +[15.417, 1.178, 1.161], +[2.396, 2.348, 2.330], +[3.916, 0.537, 0.539], +[9.855, 0.752, 0.735], +[9.330, 4.220, 4.258], +[18.911, 2.108, 2.111], +[18.849, 2.087, 2.145], +[0.942, 0.885, 0.891], +[0.102, 0.077, 0.073], +[0.062, 0.038, 0.038], +[0.052, 0.030, 0.030], +[0.251, 0.173, 0.193], +[0.035, 0.011, 0.011], +[0.023, 0.009, 0.009], +[0.011, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.12.17.json b/website/benchmark/versions/results/18.12.17.json new file mode 100644 index 00000000000..8a4ad751168 --- /dev/null +++ b/website/benchmark/versions/results/18.12.17.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-09-16 18.12", + "system_full": "ClickHouse 18.12.17 2018-09-16", + "version": "18.12.17", + "kind": "", + "comments": "", + "result": + [ + +[0.132, 0.018, 0.014], +[0.093, 0.006, 0.006], +[0.818, 0.020, 0.017], +[0.164, 0.012, 0.012], +[0.262, 0.042, 0.044], +[0.988, 0.028, 0.026], +[0.076, 0.004, 0.004], +[0.337, 0.020, 0.021], +[0.506, 0.038, 0.036], +[0.544, 0.098, 0.100], +[1.844, 0.533, 0.538], +[1.497, 0.510, 0.509], +[0.034, 0.004, 0.004], +[0.036, 0.016, 0.016], +[0.142, 0.148, 0.131], +[3.307, 0.098, 0.096], +[2.591, 0.160, 0.157], +[3.871, 0.082, 0.079], +[1.398, 0.074, 0.074], +[7.032, 0.349, 0.336], +[4.265, 0.170, 0.165], +[1.831, 0.221, 0.214], +[63.399, 2.641, 2.602], +[42.509, 2.107, 2.060], +[4.499, 0.233, 0.229], +[1.916, 0.146, 0.140], +[0.418, 0.064, 0.071], +[6.635, 0.123, 0.119], +[0.046, 0.011, 0.011], +[0.037, 0.008, 0.008], +[0.485, 0.062, 0.067], +[1.641, 0.044, 0.043], +[1.696, 0.097, 0.092], +[2.573, 0.200, 0.196], +[0.067, 0.046, 0.046], +[0.032, 0.010, 0.009], +[2.249, 0.429, 0.431], +[2.248, 0.513, 0.508], +[1.058, 0.132, 0.132], +[1.720, 0.162, 0.159], +[2.538, 0.437, 0.431], +[3.844, 0.542, 0.544], +[2.392, 0.533, 0.540], +[1.258, 0.541, 0.530], +[4.264, 1.392, 1.386], +[3.673, 0.799, 0.787], +[8.001, 2.947, 2.931], +[0.935, 0.060, 0.028], +[18.966, 0.610, 0.583], +[20.808, 0.629, 0.617], +[38.800, 1.481, 1.506], +[29.883, 0.663, 0.637], +[4.797, 0.190, 0.188], +[2.316, 0.167, 0.162], +[5.250, 0.199, 0.195], +[18.608, 0.545, 0.518], +[15.452, 1.180, 1.163], +[2.484, 2.458, 2.456], +[3.906, 0.493, 0.500], +[9.845, 0.714, 0.712], +[9.286, 4.143, 4.528], +[18.894, 2.139, 2.143], +[18.917, 2.145, 2.108], +[0.943, 0.872, 0.896], +[0.104, 0.081, 0.079], +[0.064, 0.045, 0.041], +[0.059, 0.036, 0.034], +[0.244, 0.183, 0.183], +[0.040, 0.012, 0.012], +[0.026, 0.011, 0.010], +[0.013, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.14.19.json b/website/benchmark/versions/results/18.14.19.json new file mode 100644 index 00000000000..b988dd2387c --- /dev/null +++ b/website/benchmark/versions/results/18.14.19.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-12-19 18.14", + "system_full": "ClickHouse 18.14.19 2018-12-19", + "version": "18.14.19", + "kind": "", + "comments": "", + "result": + [ + +[0.133, 0.016, 0.015], +[0.157, 0.006, 0.006], +[0.816, 0.018, 0.017], +[0.164, 0.012, 0.012], +[0.263, 0.036, 0.040], +[1.029, 0.027, 0.025], +[0.076, 0.004, 0.004], +[0.337, 0.018, 0.018], +[0.503, 0.034, 0.032], +[0.557, 0.088, 0.091], +[1.912, 0.517, 0.510], +[1.523, 0.506, 0.485], +[0.038, 0.004, 0.004], +[0.035, 0.015, 0.015], +[0.141, 0.109, 0.141], +[3.336, 0.088, 0.086], +[2.626, 0.147, 0.144], +[3.906, 0.068, 0.065], +[1.436, 0.067, 0.065], +[7.020, 0.316, 0.300], +[4.302, 0.169, 0.166], +[1.817, 0.197, 0.192], +[63.459, 2.171, 2.150], +[42.546, 1.915, 1.894], +[4.504, 0.219, 0.214], +[1.864, 0.155, 0.146], +[0.428, 0.074, 0.069], +[6.621, 0.111, 0.106], +[0.043, 0.010, 0.009], +[0.044, 0.008, 0.007], +[0.480, 0.059, 0.060], +[1.686, 0.041, 0.040], +[1.725, 0.090, 0.091], +[2.558, 0.195, 0.191], +[0.063, 0.046, 0.043], +[0.033, 0.008, 0.008], +[2.275, 0.434, 0.422], +[2.260, 0.507, 0.511], +[1.074, 0.123, 0.122], +[1.718, 0.151, 0.150], +[2.602, 0.429, 0.418], +[3.819, 0.552, 0.535], +[2.433, 0.508, 0.515], +[1.265, 0.543, 0.536], +[4.278, 1.386, 1.376], +[3.658, 0.865, 0.774], +[7.959, 2.910, 2.931], +[0.948, 0.033, 0.060], +[18.942, 0.572, 0.552], +[20.834, 0.579, 0.570], +[38.782, 1.378, 1.348], +[29.262, 0.607, 0.636], +[4.575, 0.185, 0.178], +[2.330, 0.158, 0.153], +[5.290, 0.193, 0.187], +[18.670, 0.513, 0.505], +[15.443, 1.150, 1.134], +[2.452, 2.435, 2.429], +[3.926, 0.474, 0.477], +[9.856, 0.684, 0.678], +[9.269, 4.033, 4.061], +[18.931, 2.123, 2.073], +[18.914, 2.101, 2.123], +[0.910, 0.860, 0.860], +[0.104, 0.076, 0.078], +[0.060, 0.040, 0.038], +[0.057, 0.033, 0.035], +[0.232, 0.173, 0.162], +[0.035, 0.025, 0.012], +[0.024, 0.013, 0.010], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/18.16.1.json b/website/benchmark/versions/results/18.16.1.json new file mode 100644 index 00000000000..1e570b5b946 --- /dev/null +++ b/website/benchmark/versions/results/18.16.1.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2018-12-21 18.16", + "system_full": "ClickHouse 18.16.1 2018-12-21", + "version": "18.16.1", + "kind": "", + "comments": "", + "result": + [ + +[0.133, 0.194, 0.016], +[0.215, 0.082, 0.007], +[0.818, 0.050, 0.021], +[0.156, 0.066, 0.016], +[0.269, 0.038, 0.045], +[1.159, 0.032, 0.029], +[0.086, 0.008, 0.008], +[0.335, 0.065, 0.024], +[0.488, 0.040, 0.040], +[0.550, 0.097, 0.097], +[1.906, 0.593, 0.584], +[1.503, 0.533, 0.522], +[0.038, 0.005, 0.005], +[0.036, 0.023, 0.017], +[0.185, 0.198, 0.162], +[3.335, 0.096, 0.057], +[2.631, 0.172, 0.142], +[3.884, 0.084, 0.057], +[1.463, 0.072, 0.072], +[7.189, 0.312, 0.306], +[4.632, 0.174, 0.171], +[1.865, 0.245, 0.194], +[63.399, 2.148, 2.072], +[42.517, 1.863, 1.819], +[4.507, 0.249, 0.221], +[2.078, 0.138, 0.141], +[0.455, 0.102, 0.070], +[11.873, 0.138, 0.124], +[0.046, 0.012, 0.012], +[0.032, 0.009, 0.009], +[0.482, 0.060, 0.060], +[1.682, 0.045, 0.045], +[1.719, 0.097, 0.096], +[2.567, 0.203, 0.202], +[0.063, 0.045, 0.044], +[0.027, 0.010, 0.010], +[2.283, 0.434, 0.437], +[2.256, 0.520, 0.521], +[1.078, 0.136, 0.135], +[1.693, 0.162, 0.161], +[2.589, 0.464, 0.460], +[3.809, 0.623, 0.589], +[2.391, 0.562, 0.579], +[1.265, 0.575, 0.579], +[4.293, 1.441, 1.485], +[3.656, 0.792, 0.796], +[7.960, 3.260, 3.240], +[0.923, 0.030, 0.029], +[18.973, 0.584, 0.581], +[20.841, 0.600, 0.593], +[38.786, 1.403, 1.398], +[39.036, 0.702, 0.684], +[4.554, 0.194, 0.195], +[2.325, 0.169, 0.165], +[5.239, 0.200, 0.197], +[18.609, 0.522, 0.517], +[15.427, 1.150, 1.153], +[2.475, 2.443, 2.442], +[3.870, 0.524, 0.523], +[9.837, 0.757, 0.748], +[9.334, 4.308, 4.309], +[18.947, 2.232, 2.243], +[18.972, 2.260, 2.283], +[0.991, 0.930, 0.932], +[0.116, 0.116, 0.093], +[0.061, 0.075, 0.050], +[0.056, 0.062, 0.036], +[0.272, 0.303, 0.222], +[0.041, 0.043, 0.014], +[0.025, 0.041, 0.012], +[0.014, 0.030, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.01.16.79.json b/website/benchmark/versions/results/19.01.16.79.json new file mode 100644 index 00000000000..d20ac409a7a --- /dev/null +++ b/website/benchmark/versions/results/19.01.16.79.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-04-02 19.1", + "system_full": "ClickHouse 19.1.16.79 2019-04-02", + "version": "19.1.16.79", + "kind": "", + "comments": "", + "result": + [ + +[0.139, 0.015, 0.012], +[0.041, 0.005, 0.005], +[0.817, 0.017, 0.016], +[0.160, 0.011, 0.010], +[0.264, 0.060, 0.034], +[0.987, 0.025, 0.023], +[0.075, 0.004, 0.004], +[0.335, 0.017, 0.017], +[0.507, 0.031, 0.031], +[0.555, 0.087, 0.087], +[1.861, 0.494, 0.490], +[1.510, 0.477, 0.488], +[0.033, 0.003, 0.003], +[0.031, 0.015, 0.015], +[0.156, 0.100, 0.129], +[3.293, 0.092, 0.088], +[2.613, 0.146, 0.141], +[3.902, 0.067, 0.065], +[1.432, 0.063, 0.062], +[7.049, 0.304, 0.304], +[4.342, 0.160, 0.159], +[1.827, 0.198, 0.197], +[63.412, 2.385, 2.319], +[42.497, 1.936, 1.924], +[4.499, 0.213, 0.209], +[1.975, 0.120, 0.112], +[0.460, 0.060, 0.057], +[6.642, 0.101, 0.095], +[0.045, 0.010, 0.010], +[0.042, 0.007, 0.007], +[0.482, 0.055, 0.058], +[1.667, 0.043, 0.042], +[1.720, 0.093, 0.094], +[2.556, 0.192, 0.188], +[0.059, 0.039, 0.039], +[0.026, 0.007, 0.007], +[2.299, 0.431, 0.425], +[2.258, 0.505, 0.502], +[1.050, 0.122, 0.121], +[1.715, 0.149, 0.153], +[2.605, 0.441, 0.423], +[3.786, 0.563, 0.520], +[2.393, 0.532, 0.532], +[1.268, 0.556, 0.533], +[4.330, 1.530, 1.482], +[3.647, 0.792, 0.833], +[8.224, 3.651, 3.551], +[0.941, 0.028, 0.027], +[18.967, 0.511, 0.507], +[20.837, 0.583, 0.583], +[38.807, 1.472, 1.356], +[29.895, 0.646, 0.669], +[4.550, 0.175, 0.173], +[2.371, 0.147, 0.145], +[5.296, 0.179, 0.177], +[18.656, 0.513, 0.501], +[15.434, 1.153, 1.145], +[2.549, 2.527, 2.507], +[3.922, 0.537, 0.554], +[9.872, 0.770, 0.784], +[9.630, 4.556, 4.342], +[18.955, 2.259, 2.280], +[18.975, 2.125, 2.130], +[0.930, 0.908, 0.909], +[0.099, 0.074, 0.073], +[0.053, 0.037, 0.036], +[0.055, 0.030, 0.029], +[0.201, 0.156, 0.158], +[0.034, 0.011, 0.011], +[0.023, 0.009, 0.009], +[0.011, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.03.9.12_adaptive.json b/website/benchmark/versions/results/19.03.9.12_adaptive.json new file mode 100644 index 00000000000..a61f28e7caa --- /dev/null +++ b/website/benchmark/versions/results/19.03.9.12_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-04-02 19.3", + "system_full": "ClickHouse 19.3.9.12 2019-04-02(adaptive)", + "version": "19.3.9.12", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.045, 0.003, 0.003], +[0.022, 0.002, 0.002], +[0.018, 0.003, 0.004], +[0.016, 0.002, 0.002], +[0.018, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.018, 0.002, 0.002], +[0.073, 0.029, 0.029], +[0.023, 0.011, 0.011], +[0.010, 0.002, 0.002], +[1.082, 0.342, 0.362], +[0.613, 0.344, 0.320], +[0.035, 0.003, 0.003], +[0.031, 0.014, 0.019], +[0.132, 0.102, 0.106], +[3.420, 0.095, 0.090], +[3.012, 0.748, 0.750], +[2.259, 0.060, 0.057], +[1.270, 0.047, 0.048], +[4.462, 0.226, 0.204], +[4.212, 0.184, 0.177], +[1.818, 0.219, 0.206], +[63.399, 2.050, 2.016], +[42.454, 1.432, 1.429], +[3.563, 0.822, 0.814], +[0.785, 0.133, 0.141], +[0.118, 0.079, 0.082], +[3.214, 0.089, 0.077], +[0.010, 0.001, 0.001], +[0.029, 0.008, 0.009], +[0.382, 0.025, 0.026], +[1.707, 0.031, 0.031], +[1.691, 0.079, 0.091], +[2.601, 0.200, 0.212], +[0.025, 0.012, 0.011], +[0.018, 0.009, 0.009], +[2.124, 0.417, 0.421], +[1.997, 0.462, 0.461], +[1.043, 0.130, 0.129], +[1.665, 0.145, 0.147], +[2.617, 0.485, 0.478], +[3.811, 0.606, 0.569], +[2.376, 0.559, 0.562], +[1.199, 0.566, 0.570], +[4.246, 1.434, 1.415], +[3.672, 0.828, 0.813], +[7.948, 3.037, 3.016], +[0.911, 0.048, 0.039], +[19.548, 0.495, 0.449], +[21.486, 0.568, 0.557], +[40.178, 1.318, 1.298], +[39.525, 0.609, 0.554], +[4.645, 0.196, 0.204], +[2.291, 0.171, 0.175], +[5.290, 0.234, 0.213], +[19.139, 0.450, 0.450], +[15.793, 0.791, 0.776], +[1.155, 1.141, 1.128], +[3.837, 0.491, 0.484], +[9.820, 0.730, 0.704], +[9.079, 4.077, 4.048], +[19.457, 2.042, 2.017], +[19.483, 2.137, 2.010], +[0.899, 0.829, 0.833], +[0.178, 0.135, 0.123], +[0.073, 0.051, 0.048], +[0.072, 0.037, 0.042], +[0.314, 0.274, 0.253], +[0.059, 0.017, 0.028], +[0.041, 0.012, 0.022], +[0.013, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.04.5.35.json b/website/benchmark/versions/results/19.04.5.35.json new file mode 100644 index 00000000000..884754b3370 --- /dev/null +++ b/website/benchmark/versions/results/19.04.5.35.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-05-13 19.4", + "system_full": "ClickHouse 19.4.5.35 2019-05-13", + "version": "19.4.5.35", + "kind": "", + "comments": "", + "result": + [ + +[0.072, 0.009, 0.009], +[0.075, 0.032, 0.032], +[0.066, 0.008, 0.008], +[0.055, 0.012, 0.012], +[0.072, 0.036, 0.036], +[0.124, 0.019, 0.018], +[0.101, 0.005, 0.004], +[0.335, 0.033, 0.032], +[0.503, 0.047, 0.047], +[0.528, 0.094, 0.093], +[1.898, 0.567, 0.540], +[1.502, 0.515, 0.508], +[0.035, 0.004, 0.004], +[0.032, 0.013, 0.015], +[0.134, 0.110, 0.104], +[3.333, 0.095, 0.095], +[2.608, 0.159, 0.152], +[2.203, 0.063, 0.062], +[1.316, 0.057, 0.058], +[5.163, 0.198, 0.186], +[4.884, 0.160, 0.153], +[1.903, 0.188, 0.184], +[63.457, 2.105, 2.057], +[42.487, 1.567, 1.545], +[4.497, 0.232, 0.226], +[1.435, 0.145, 0.166], +[0.262, 0.053, 0.059], +[2.915, 0.098, 0.081], +[0.102, 0.011, 0.011], +[0.045, 0.008, 0.007], +[0.481, 0.066, 0.066], +[1.682, 0.046, 0.046], +[1.710, 0.099, 0.101], +[2.556, 0.213, 0.206], +[0.075, 0.050, 0.046], +[0.031, 0.008, 0.008], +[2.245, 0.429, 0.420], +[2.246, 0.490, 0.483], +[1.073, 0.117, 0.119], +[1.735, 0.152, 0.157], +[2.644, 0.415, 0.401], +[3.812, 0.553, 0.556], +[2.404, 0.521, 0.515], +[1.262, 0.556, 0.561], +[4.287, 1.389, 1.380], +[3.628, 0.785, 0.777], +[8.027, 2.905, 2.912], +[0.945, 0.081, 0.028], +[18.915, 0.529, 0.503], +[20.803, 0.595, 0.580], +[38.783, 1.390, 1.363], +[38.820, 0.679, 0.639], +[4.565, 0.180, 0.183], +[2.309, 0.161, 0.150], +[5.264, 0.191, 0.183], +[18.646, 0.540, 0.521], +[15.425, 1.115, 1.092], +[2.306, 2.275, 2.265], +[3.941, 0.478, 0.488], +[9.883, 0.714, 0.704], +[9.498, 4.322, 4.348], +[18.919, 2.039, 2.010], +[18.883, 2.028, 1.980], +[0.916, 0.851, 0.867], +[0.114, 0.083, 0.077], +[0.058, 0.039, 0.041], +[0.053, 0.035, 0.028], +[0.248, 0.186, 0.182], +[0.041, 0.016, 0.014], +[0.025, 0.011, 0.010], +[0.013, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.05.4.22.json b/website/benchmark/versions/results/19.05.4.22.json new file mode 100644 index 00000000000..dc1505ce9d4 --- /dev/null +++ b/website/benchmark/versions/results/19.05.4.22.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-05-13 19.5", + "system_full": "ClickHouse 19.5.4.22 2019-05-13", + "kind": "", + "version": "19.5.4.22", + "comments": "", + "result": + [ + +[0.067, 0.010, 0.009], +[0.076, 0.032, 0.032], +[0.045, 0.008, 0.008], +[0.049, 0.011, 0.011], +[0.071, 0.035, 0.035], +[0.232, 0.020, 0.021], +[0.101, 0.005, 0.005], +[0.334, 0.035, 0.033], +[0.499, 0.047, 0.048], +[0.532, 0.096, 0.091], +[1.878, 0.562, 0.555], +[1.484, 0.513, 0.504], +[0.034, 0.003, 0.003], +[0.031, 0.015, 0.015], +[0.130, 0.114, 0.108], +[3.323, 0.095, 0.095], +[2.616, 0.160, 0.147], +[2.218, 0.063, 0.061], +[1.333, 0.059, 0.057], +[5.176, 0.211, 0.196], +[4.883, 0.167, 0.172], +[1.905, 0.194, 0.185], +[63.470, 2.105, 2.067], +[42.523, 1.606, 1.560], +[4.501, 0.231, 0.231], +[1.412, 0.159, 0.162], +[0.247, 0.060, 0.057], +[2.935, 0.075, 0.084], +[0.073, 0.010, 0.010], +[0.046, 0.008, 0.007], +[0.481, 0.063, 0.061], +[1.669, 0.042, 0.043], +[1.693, 0.097, 0.096], +[2.565, 0.211, 0.207], +[0.062, 0.044, 0.044], +[0.029, 0.008, 0.008], +[2.243, 0.438, 0.429], +[2.231, 0.493, 0.485], +[1.071, 0.119, 0.121], +[1.717, 0.143, 0.145], +[2.616, 0.422, 0.416], +[3.792, 0.523, 0.542], +[2.383, 0.535, 0.503], +[1.249, 0.568, 0.560], +[4.310, 1.423, 1.386], +[3.654, 0.776, 0.764], +[8.060, 2.973, 2.907], +[0.962, 0.058, 0.029], +[18.952, 0.492, 0.487], +[20.838, 0.593, 0.563], +[38.796, 1.366, 1.345], +[38.903, 0.672, 0.637], +[4.568, 0.181, 0.177], +[2.335, 0.148, 0.151], +[5.300, 0.194, 0.181], +[18.630, 0.520, 0.498], +[15.435, 1.086, 1.064], +[2.320, 2.304, 2.300], +[3.932, 0.477, 0.485], +[9.834, 0.713, 0.687], +[9.485, 4.400, 4.381], +[18.930, 2.046, 1.980], +[18.933, 2.012, 1.974], +[0.949, 0.882, 0.875], +[0.112, 0.097, 0.082], +[0.054, 0.043, 0.046], +[0.052, 0.029, 0.036], +[0.243, 0.207, 0.188], +[0.044, 0.014, 0.019], +[0.026, 0.010, 0.010], +[0.015, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.06.3.18_adaptive.json b/website/benchmark/versions/results/19.06.3.18_adaptive.json new file mode 100644 index 00000000000..6a3814b0cdb --- /dev/null +++ b/website/benchmark/versions/results/19.06.3.18_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-06-15 19.6", + "system_full": "ClickHouse 19.6.3.18 2019-06-15(adaptive)", + "version": "19.6.3.18", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.033, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.021, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.016, 0.002, 0.002], +[0.019, 0.003, 0.003], +[0.018, 0.001, 0.001], +[0.063, 0.024, 0.024], +[0.022, 0.008, 0.008], +[0.015, 0.002, 0.002], +[1.237, 0.349, 0.337], +[0.600, 0.297, 0.316], +[0.040, 0.003, 0.003], +[0.034, 0.014, 0.013], +[0.135, 0.109, 0.099], +[3.433, 0.097, 0.093], +[3.003, 0.670, 0.666], +[2.257, 0.063, 0.064], +[1.274, 0.051, 0.052], +[4.465, 0.202, 0.199], +[4.254, 0.161, 0.164], +[1.818, 0.177, 0.188], +[63.476, 2.082, 2.042], +[42.480, 1.577, 1.542], +[3.488, 0.723, 0.708], +[0.786, 0.147, 0.156], +[0.115, 0.066, 0.063], +[3.706, 0.087, 0.089], +[0.060, 0.012, 0.011], +[0.055, 0.009, 0.008], +[0.433, 0.067, 0.072], +[1.620, 0.049, 0.050], +[1.647, 0.109, 0.101], +[2.557, 0.207, 0.194], +[0.078, 0.050, 0.047], +[0.037, 0.009, 0.009], +[1.997, 0.420, 0.411], +[2.002, 0.465, 0.469], +[1.027, 0.117, 0.116], +[1.681, 0.141, 0.139], +[2.648, 0.398, 0.391], +[3.791, 0.510, 0.499], +[2.353, 0.529, 0.529], +[1.226, 0.547, 0.542], +[4.267, 1.333, 1.305], +[3.680, 0.733, 0.708], +[8.051, 2.804, 2.865], +[0.917, 0.042, 0.031], +[19.546, 0.463, 0.438], +[21.449, 0.555, 0.548], +[40.133, 1.226, 1.204], +[39.599, 0.606, 0.569], +[4.663, 0.176, 0.174], +[2.341, 0.161, 0.147], +[5.326, 0.188, 0.176], +[19.228, 0.468, 0.449], +[15.821, 1.027, 1.003], +[2.306, 2.271, 2.296], +[3.861, 0.479, 0.446], +[9.817, 0.688, 0.668], +[9.318, 4.049, 4.092], +[19.435, 1.893, 1.868], +[19.280, 1.873, 1.860], +[0.921, 0.865, 0.852], +[0.095, 0.059, 0.064], +[0.055, 0.028, 0.035], +[0.048, 0.021, 0.020], +[0.191, 0.135, 0.126], +[0.039, 0.012, 0.010], +[0.028, 0.008, 0.008], +[0.016, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.07.5.29_adaptive.json b/website/benchmark/versions/results/19.07.5.29_adaptive.json new file mode 100644 index 00000000000..a8b2a9d2cbb --- /dev/null +++ b/website/benchmark/versions/results/19.07.5.29_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-07-05 19.7", + "system_full": "ClickHouse 19.7.5.29 2019-07-05(adaptive)", + "version": "19.7.5.29", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.039, 0.003, 0.003], +[0.015, 0.002, 0.001], +[0.020, 0.003, 0.003], +[0.016, 0.002, 0.002], +[0.016, 0.002, 0.002], +[0.019, 0.003, 0.003], +[0.016, 0.001, 0.001], +[0.067, 0.025, 0.024], +[0.020, 0.008, 0.008], +[0.011, 0.002, 0.002], +[1.090, 0.342, 0.338], +[0.591, 0.306, 0.312], +[0.036, 0.003, 0.003], +[0.032, 0.014, 0.014], +[0.137, 0.107, 0.101], +[3.428, 0.100, 0.093], +[2.994, 0.681, 0.678], +[2.268, 0.066, 0.063], +[1.258, 0.052, 0.047], +[4.453, 0.211, 0.188], +[4.249, 0.163, 0.160], +[1.831, 0.176, 0.173], +[63.471, 2.099, 2.050], +[42.482, 1.609, 1.601], +[3.497, 0.735, 0.714], +[0.799, 0.147, 0.176], +[0.116, 0.056, 0.058], +[3.715, 0.088, 0.086], +[0.064, 0.011, 0.010], +[0.047, 0.008, 0.008], +[0.432, 0.065, 0.066], +[1.635, 0.049, 0.048], +[1.658, 0.105, 0.105], +[2.558, 0.201, 0.198], +[0.066, 0.040, 0.040], +[0.034, 0.008, 0.008], +[2.066, 0.432, 0.433], +[2.033, 0.491, 0.479], +[1.050, 0.122, 0.122], +[1.696, 0.147, 0.146], +[2.613, 0.427, 0.389], +[3.795, 0.508, 0.500], +[2.339, 0.509, 0.526], +[1.202, 0.541, 0.548], +[4.288, 1.364, 1.337], +[3.671, 0.734, 0.723], +[8.031, 2.884, 2.832], +[0.896, 0.038, 0.031], +[19.524, 0.461, 0.441], +[21.437, 0.578, 0.549], +[40.135, 1.244, 1.213], +[39.587, 0.612, 0.601], +[4.660, 0.181, 0.178], +[2.325, 0.161, 0.149], +[5.324, 0.203, 0.178], +[19.194, 0.483, 0.460], +[15.815, 1.027, 1.021], +[2.272, 2.248, 2.269], +[3.857, 0.466, 0.455], +[9.839, 0.694, 0.669], +[9.343, 4.105, 4.058], +[19.485, 1.906, 1.864], +[19.375, 1.905, 1.862], +[0.924, 0.848, 0.844], +[0.091, 0.066, 0.058], +[0.052, 0.029, 0.029], +[0.046, 0.024, 0.020], +[0.185, 0.132, 0.140], +[0.037, 0.010, 0.011], +[0.027, 0.008, 0.009], +[0.015, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.08.3.8_adaptive.json b/website/benchmark/versions/results/19.08.3.8_adaptive.json new file mode 100644 index 00000000000..87afb885c3d --- /dev/null +++ b/website/benchmark/versions/results/19.08.3.8_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-06-11 19.8", + "system_full": "ClickHouse 19.8.3.8 2019-06-11(adaptive)", + "version": "19.8.3.8", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.036, 0.003, 0.003], +[0.014, 0.002, 0.001], +[0.022, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.016, 0.002, 0.002], +[0.019, 0.003, 0.003], +[0.017, 0.001, 0.001], +[0.063, 0.026, 0.027], +[0.019, 0.008, 0.008], +[0.009, 0.002, 0.001], +[1.119, 0.338, 0.340], +[0.593, 0.300, 0.300], +[0.040, 0.003, 0.003], +[0.035, 0.014, 0.013], +[0.162, 0.107, 0.107], +[3.426, 0.100, 0.091], +[3.003, 0.671, 0.667], +[2.275, 0.062, 0.063], +[1.285, 0.054, 0.055], +[4.459, 0.210, 0.192], +[4.257, 0.179, 0.163], +[1.813, 0.178, 0.183], +[63.480, 2.122, 2.062], +[42.481, 1.598, 1.581], +[3.489, 0.717, 0.709], +[0.796, 0.124, 0.195], +[0.114, 0.054, 0.052], +[3.719, 0.088, 0.085], +[0.063, 0.011, 0.011], +[0.046, 0.009, 0.008], +[0.411, 0.060, 0.058], +[1.630, 0.044, 0.043], +[1.666, 0.098, 0.097], +[2.572, 0.200, 0.194], +[0.072, 0.041, 0.043], +[0.038, 0.009, 0.008], +[2.024, 0.439, 0.425], +[2.018, 0.486, 0.484], +[1.032, 0.122, 0.120], +[1.691, 0.153, 0.151], +[2.577, 0.398, 0.386], +[3.811, 0.511, 0.494], +[2.366, 0.527, 0.515], +[1.183, 0.535, 0.537], +[4.240, 1.323, 1.311], +[3.667, 0.733, 0.717], +[7.945, 2.772, 2.820], +[0.863, 0.046, 0.028], +[19.550, 0.424, 0.417], +[21.446, 0.539, 0.520], +[40.092, 1.202, 1.167], +[39.601, 0.617, 0.556], +[4.662, 0.175, 0.174], +[2.349, 0.156, 0.145], +[5.350, 0.181, 0.179], +[19.239, 0.455, 0.438], +[15.829, 1.015, 0.996], +[2.297, 2.269, 2.278], +[3.865, 0.468, 0.440], +[9.817, 0.674, 0.645], +[9.106, 3.916, 4.283], +[19.440, 1.872, 1.838], +[19.427, 1.858, 1.818], +[0.894, 0.835, 0.847], +[0.088, 0.059, 0.060], +[0.048, 0.027, 0.030], +[0.046, 0.022, 0.024], +[0.180, 0.137, 0.124], +[0.037, 0.011, 0.010], +[0.025, 0.008, 0.009], +[0.015, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.09.5.36_adaptive.json b/website/benchmark/versions/results/19.09.5.36_adaptive.json new file mode 100644 index 00000000000..6a4de63a0dc --- /dev/null +++ b/website/benchmark/versions/results/19.09.5.36_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-07-20 19.9", + "system_full": "ClickHouse 19.9.5.36 2019-07-20(adaptive)", + "version": "19.9.5.36", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.034, 0.003, 0.003], +[0.014, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.014, 0.002, 0.002], +[0.015, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.016, 0.001, 0.001], +[0.120, 0.024, 0.027], +[0.021, 0.008, 0.008], +[0.016, 0.001, 0.001], +[1.225, 0.331, 0.331], +[0.620, 0.301, 0.296], +[0.038, 0.003, 0.003], +[0.034, 0.013, 0.013], +[0.125, 0.108, 0.100], +[3.432, 0.095, 0.090], +[2.993, 0.667, 0.673], +[2.271, 0.060, 0.059], +[1.282, 0.047, 0.051], +[4.467, 0.201, 0.185], +[4.258, 0.167, 0.157], +[1.845, 0.180, 0.180], +[63.456, 2.040, 2.008], +[42.482, 1.578, 1.566], +[3.485, 0.714, 0.717], +[0.789, 0.176, 0.176], +[0.120, 0.063, 0.061], +[3.680, 0.092, 0.084], +[0.060, 0.010, 0.010], +[0.053, 0.008, 0.007], +[0.429, 0.060, 0.054], +[1.646, 0.042, 0.045], +[1.659, 0.105, 0.106], +[2.557, 0.207, 0.195], +[0.072, 0.041, 0.043], +[0.041, 0.008, 0.008], +[2.012, 0.439, 0.424], +[2.021, 0.486, 0.480], +[1.054, 0.126, 0.124], +[1.666, 0.158, 0.167], +[2.565, 0.437, 0.392], +[3.795, 0.497, 0.529], +[2.355, 0.524, 0.517], +[1.191, 0.540, 0.538], +[4.245, 1.306, 1.298], +[3.654, 0.728, 0.717], +[7.948, 2.782, 2.789], +[0.893, 0.061, 0.027], +[19.518, 0.425, 0.408], +[21.473, 0.524, 0.513], +[40.168, 1.177, 1.137], +[39.602, 0.574, 0.588], +[4.663, 0.169, 0.176], +[2.346, 0.155, 0.144], +[5.320, 0.193, 0.176], +[19.215, 0.437, 0.418], +[15.836, 0.996, 0.982], +[2.297, 2.258, 2.274], +[3.824, 0.448, 0.454], +[9.803, 0.657, 0.632], +[9.051, 3.826, 3.919], +[19.430, 1.820, 1.798], +[19.387, 1.847, 1.820], +[0.885, 0.815, 0.823], +[0.089, 0.065, 0.057], +[0.056, 0.028, 0.033], +[0.041, 0.022, 0.022], +[0.199, 0.128, 0.120], +[0.036, 0.010, 0.010], +[0.025, 0.008, 0.009], +[0.017, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.10.1.5_adaptive.json b/website/benchmark/versions/results/19.10.1.5_adaptive.json new file mode 100644 index 00000000000..b66d70caab6 --- /dev/null +++ b/website/benchmark/versions/results/19.10.1.5_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-07-12 19.10", + "system_full": "ClickHouse 19.10.1.5 2019-07-12(adaptive)", + "version": "19.10.1.5", + "type": "adaptive", + "comments": "", + "result": + [ + +[0.045, 0.003, 0.003], +[0.017, 0.001, 0.001], +[0.018, 0.003, 0.003], +[0.023, 0.002, 0.001], +[0.016, 0.002, 0.002], +[0.021, 0.003, 0.003], +[0.017, 0.001, 0.001], +[0.062, 0.024, 0.023], +[0.021, 0.008, 0.008], +[0.011, 0.001, 0.001], +[1.159, 0.338, 0.334], +[0.620, 0.323, 0.307], +[0.021, 0.002, 0.001], +[0.009, 0.001, 0.001], +[0.017, 0.004, 0.004], +[3.646, 0.094, 0.089], +[2.997, 0.679, 0.667], +[2.240, 0.060, 0.059], +[1.177, 0.060, 0.056], +[4.103, 0.224, 0.194], +[3.934, 0.167, 0.171], +[1.754, 0.188, 0.177], +[63.474, 2.051, 2.021], +[42.421, 1.550, 1.518], +[3.482, 0.725, 0.731], +[0.683, 0.161, 0.136], +[0.087, 0.058, 0.055], +[3.057, 0.061, 0.055], +[0.095, 0.010, 0.010], +[0.062, 0.008, 0.007], +[0.425, 0.062, 0.065], +[1.645, 0.047, 0.045], +[1.655, 0.117, 0.113], +[2.541, 0.220, 0.198], +[0.058, 0.044, 0.044], +[0.019, 0.008, 0.008], +[2.034, 0.434, 0.420], +[2.035, 0.485, 0.484], +[1.036, 0.116, 0.118], +[1.672, 0.150, 0.148], +[2.584, 0.397, 0.385], +[3.829, 0.512, 0.488], +[2.373, 0.518, 0.504], +[1.191, 0.546, 0.539], +[4.394, 1.359, 1.319], +[3.649, 0.754, 0.735], +[7.906, 2.846, 2.808], +[0.906, 0.044, 0.027], +[19.556, 0.441, 0.410], +[21.468, 0.529, 0.521], +[40.145, 1.194, 1.165], +[39.573, 0.559, 0.529], +[4.652, 0.180, 0.171], +[2.321, 0.154, 0.147], +[5.312, 0.188, 0.178], +[19.213, 0.444, 0.424], +[15.806, 1.006, 0.981], +[2.265, 2.257, 2.274], +[3.826, 0.470, 0.449], +[9.797, 0.666, 0.639], +[9.059, 3.902, 3.855], +[19.428, 1.865, 1.804], +[19.377, 1.843, 1.815], +[0.885, 0.828, 0.828], +[0.090, 0.061, 0.061], +[0.052, 0.029, 0.027], +[0.045, 0.021, 0.022], +[0.195, 0.129, 0.137], +[0.035, 0.011, 0.011], +[0.027, 0.009, 0.008], +[0.013, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.11.14.1_adaptive.json b/website/benchmark/versions/results/19.11.14.1_adaptive.json new file mode 100644 index 00000000000..11e9a4abffb --- /dev/null +++ b/website/benchmark/versions/results/19.11.14.1_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-12-04 19.11", + "system_full": "ClickHouse 19.11.14.1 2019-12-04(adaptive)", + "version": "19.11.14.1", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.037, 0.003, 0.003], +[0.018, 0.001, 0.001], +[0.019, 0.003, 0.003], +[0.017, 0.002, 0.001], +[0.015, 0.002, 0.002], +[0.020, 0.003, 0.003], +[0.018, 0.001, 0.001], +[0.066, 0.025, 0.028], +[0.020, 0.008, 0.008], +[0.011, 0.001, 0.001], +[1.138, 0.350, 0.340], +[0.580, 0.309, 0.305], +[0.034, 0.003, 0.004], +[0.031, 0.014, 0.013], +[0.137, 0.103, 0.107], +[3.421, 0.099, 0.090], +[3.002, 0.653, 0.647], +[2.253, 0.064, 0.060], +[1.258, 0.056, 0.052], +[4.436, 0.230, 0.208], +[4.214, 0.163, 0.176], +[1.854, 0.208, 0.208], +[63.371, 2.164, 2.111], +[42.444, 1.585, 1.551], +[3.448, 0.708, 0.699], +[0.793, 0.154, 0.169], +[0.104, 0.081, 0.075], +[3.178, 0.080, 0.091], +[0.039, 0.010, 0.010], +[0.060, 0.007, 0.007], +[0.427, 0.073, 0.071], +[1.616, 0.046, 0.047], +[1.654, 0.098, 0.096], +[2.561, 0.211, 0.211], +[0.064, 0.048, 0.048], +[0.019, 0.008, 0.007], +[1.989, 0.396, 0.394], +[1.991, 0.456, 0.453], +[1.020, 0.132, 0.129], +[1.640, 0.168, 0.161], +[2.523, 0.430, 0.406], +[3.826, 0.511, 0.505], +[2.356, 0.507, 0.509], +[1.177, 0.538, 0.534], +[4.299, 1.294, 1.286], +[3.662, 0.732, 0.704], +[7.885, 2.777, 2.784], +[0.911, 0.053, 0.027], +[19.526, 0.431, 0.419], +[21.478, 0.557, 0.536], +[40.140, 1.248, 1.209], +[39.497, 0.589, 0.542], +[4.644, 0.195, 0.187], +[2.302, 0.167, 0.164], +[5.303, 0.203, 0.196], +[19.169, 0.439, 0.425], +[15.754, 0.803, 0.777], +[2.251, 2.236, 2.251], +[3.831, 0.475, 0.454], +[9.802, 0.666, 0.656], +[9.119, 3.820, 3.843], +[19.443, 1.864, 1.825], +[19.451, 1.848, 1.819], +[0.854, 0.808, 0.799], +[0.153, 0.119, 0.111], +[0.071, 0.048, 0.045], +[0.065, 0.037, 0.035], +[0.319, 0.253, 0.247], +[0.051, 0.019, 0.018], +[0.033, 0.013, 0.014], +[0.013, 0.002, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/19.13.7.57_adaptive.json b/website/benchmark/versions/results/19.13.7.57_adaptive.json new file mode 100644 index 00000000000..bacbe36fbce --- /dev/null +++ b/website/benchmark/versions/results/19.13.7.57_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-11-06 19.13", + "system_full": "ClickHouse 19.13.7.57 2019-11-06(adaptive)", + "version": "19.13.7.57", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.032, 0.003, 0.003], +[0.014, 0.002, 0.002], +[0.019, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.017, 0.002, 0.002], +[0.018, 0.003, 0.003], +[0.018, 0.001, 0.001], +[0.065, 0.025, 0.028], +[0.022, 0.009, 0.009], +[0.013, 0.002, 0.002], +[1.218, 0.344, 0.339], +[0.597, 0.317, 0.318], +[0.033, 0.003, 0.003], +[0.031, 0.013, 0.017], +[0.141, 0.106, 0.096], +[3.401, 0.092, 0.087], +[3.037, 0.748, 0.740], +[2.263, 0.059, 0.057], +[1.277, 0.045, 0.057], +[4.439, 0.223, 0.213], +[4.197, 0.178, 0.174], +[1.817, 0.223, 0.195], +[63.350, 2.157, 2.109], +[42.422, 1.557, 1.551], +[3.545, 0.808, 0.798], +[0.791, 0.185, 0.154], +[0.110, 0.082, 0.079], +[3.174, 0.093, 0.106], +[0.047, 0.030, 0.031], +[0.023, 0.008, 0.008], +[0.306, 0.069, 0.067], +[1.635, 0.049, 0.047], +[1.669, 0.096, 0.096], +[2.579, 0.216, 0.207], +[0.060, 0.040, 0.043], +[0.017, 0.008, 0.008], +[2.057, 0.421, 0.403], +[2.015, 0.464, 0.458], +[1.039, 0.122, 0.127], +[1.676, 0.151, 0.144], +[2.624, 0.421, 0.394], +[3.777, 0.536, 0.508], +[2.352, 0.523, 0.494], +[1.205, 0.537, 0.532], +[4.218, 1.303, 1.273], +[3.674, 0.724, 0.714], +[7.911, 2.748, 2.734], +[0.910, 0.057, 0.029], +[19.529, 0.434, 0.415], +[21.471, 0.577, 0.527], +[40.121, 1.221, 1.191], +[39.482, 0.566, 0.544], +[4.644, 0.191, 0.191], +[2.312, 0.168, 0.159], +[5.286, 0.204, 0.199], +[19.174, 0.449, 0.431], +[15.773, 0.772, 0.755], +[2.270, 2.254, 2.254], +[3.855, 0.469, 0.455], +[9.782, 0.667, 0.640], +[9.127, 3.834, 3.826], +[19.407, 1.852, 1.794], +[19.405, 1.838, 1.803], +[0.850, 0.803, 0.799], +[0.146, 0.118, 0.111], +[0.070, 0.048, 0.050], +[0.067, 0.038, 0.035], +[0.318, 0.238, 0.256], +[0.058, 0.019, 0.018], +[0.034, 0.013, 0.013], +[0.013, 0.003, 0.005] + ] + } +] diff --git a/website/benchmark/versions/results/19.14.13.4_adaptive.json b/website/benchmark/versions/results/19.14.13.4_adaptive.json new file mode 100644 index 00000000000..27e969d7046 --- /dev/null +++ b/website/benchmark/versions/results/19.14.13.4_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-07-21 19.14", + "system_full": "ClickHouse 19.14.13.4 2020-07-21(adaptive)", + "version": "19.14.13.4", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.037, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.021, 0.003, 0.003], +[0.016, 0.002, 0.002], +[0.016, 0.002, 0.002], +[0.015, 0.003, 0.003], +[0.018, 0.002, 0.002], +[0.071, 0.040, 0.030], +[0.022, 0.009, 0.009], +[0.014, 0.002, 0.002], +[1.241, 0.353, 0.357], +[0.550, 0.346, 0.341], +[0.034, 0.004, 0.003], +[0.031, 0.019, 0.018], +[0.124, 0.101, 0.106], +[3.424, 0.101, 0.095], +[3.021, 0.745, 0.742], +[2.268, 0.057, 0.055], +[1.266, 0.052, 0.051], +[4.452, 0.219, 0.226], +[4.183, 0.188, 0.198], +[1.823, 0.227, 0.221], +[63.392, 2.282, 2.249], +[42.447, 1.526, 1.529], +[3.546, 0.809, 0.813], +[0.763, 0.173, 0.141], +[0.119, 0.079, 0.094], +[3.172, 0.100, 0.085], +[0.025, 0.006, 0.006], +[0.026, 0.009, 0.008], +[0.365, 0.027, 0.026], +[1.690, 0.034, 0.032], +[1.668, 0.090, 0.089], +[2.592, 0.207, 0.211], +[0.027, 0.011, 0.011], +[0.018, 0.009, 0.009], +[2.114, 0.410, 0.403], +[2.003, 0.457, 0.444], +[1.025, 0.130, 0.131], +[1.645, 0.160, 0.156], +[2.559, 0.463, 0.454], +[3.838, 0.579, 0.570], +[2.380, 0.552, 0.549], +[1.202, 0.600, 0.583], +[4.286, 1.424, 1.415], +[3.680, 0.808, 0.823], +[7.918, 3.030, 3.035], +[0.906, 0.044, 0.044], +[19.539, 0.495, 0.483], +[21.464, 0.603, 0.562], +[40.117, 1.299, 1.279], +[39.494, 0.622, 0.607], +[4.644, 0.209, 0.216], +[2.249, 0.165, 0.183], +[5.262, 0.223, 0.218], +[19.142, 0.461, 0.449], +[15.763, 0.824, 0.828], +[1.168, 1.135, 1.139], +[3.836, 0.492, 0.509], +[9.800, 0.729, 0.705], +[9.106, 4.009, 4.031], +[19.505, 1.972, 1.949], +[19.475, 2.047, 1.972], +[0.858, 0.846, 0.825], +[0.163, 0.122, 0.117], +[0.067, 0.048, 0.049], +[0.072, 0.043, 0.039], +[0.336, 0.262, 0.277], +[0.050, 0.027, 0.018], +[0.030, 0.023, 0.012], +[0.013, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.15.7.30_adaptive.json b/website/benchmark/versions/results/19.15.7.30_adaptive.json new file mode 100644 index 00000000000..376058e1748 --- /dev/null +++ b/website/benchmark/versions/results/19.15.7.30_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2019-12-29 19.15", + "system_full": "ClickHouse 19.15.7.30 2019-12-29(adaptive)", + "version": "19.15.7.30", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.036, 0.003, 0.003], +[0.014, 0.002, 0.002], +[0.016, 0.003, 0.003], +[0.016, 0.002, 0.002], +[0.015, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.016, 0.002, 0.002], +[0.067, 0.039, 0.039], +[0.021, 0.008, 0.008], +[0.011, 0.002, 0.002], +[1.241, 0.346, 0.359], +[0.572, 0.336, 0.340], +[0.034, 0.003, 0.003], +[0.032, 0.019, 0.014], +[0.130, 0.102, 0.098], +[3.424, 0.092, 0.092], +[3.032, 0.737, 0.746], +[2.283, 0.055, 0.055], +[1.275, 0.049, 0.049], +[4.531, 0.222, 0.216], +[4.224, 0.190, 0.174], +[1.808, 0.212, 0.211], +[63.378, 2.088, 2.059], +[42.433, 1.451, 1.445], +[3.535, 0.788, 0.792], +[0.783, 0.166, 0.146], +[0.117, 0.075, 0.083], +[3.194, 0.081, 0.091], +[0.025, 0.005, 0.005], +[0.025, 0.008, 0.008], +[0.400, 0.022, 0.023], +[1.716, 0.032, 0.032], +[1.684, 0.082, 0.088], +[2.604, 0.201, 0.203], +[0.029, 0.014, 0.015], +[0.019, 0.008, 0.009], +[2.128, 0.415, 0.408], +[2.008, 0.445, 0.442], +[1.031, 0.119, 0.119], +[1.673, 0.149, 0.143], +[2.623, 0.468, 0.491], +[3.772, 0.566, 0.556], +[2.345, 0.545, 0.540], +[1.198, 0.581, 0.604], +[4.245, 1.422, 1.421], +[3.641, 0.797, 0.799], +[7.931, 3.072, 3.036], +[0.899, 0.046, 0.037], +[19.526, 0.453, 0.454], +[21.447, 0.556, 0.583], +[39.978, 1.279, 1.248], +[39.505, 0.567, 0.563], +[4.637, 0.205, 0.204], +[2.262, 0.170, 0.170], +[5.273, 0.211, 0.214], +[19.142, 0.448, 0.435], +[15.800, 0.780, 0.782], +[1.154, 1.138, 1.134], +[3.820, 0.508, 0.524], +[9.790, 0.722, 0.689], +[9.068, 4.062, 4.036], +[19.494, 2.033, 1.911], +[19.512, 2.044, 1.967], +[0.860, 0.839, 0.830], +[0.151, 0.129, 0.118], +[0.068, 0.048, 0.053], +[0.068, 0.043, 0.037], +[0.312, 0.251, 0.273], +[0.051, 0.023, 0.026], +[0.034, 0.024, 0.022], +[0.014, 0.002, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.16.19.85_adaptive.json b/website/benchmark/versions/results/19.16.19.85_adaptive.json new file mode 100644 index 00000000000..259b6310720 --- /dev/null +++ b/website/benchmark/versions/results/19.16.19.85_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-04-27 19.16", + "system_full": "ClickHouse 19.16.19.85 2020-04-27(adaptive)", + "version": "19.16.19.85", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.035, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.015, 0.002, 0.002], +[0.014, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.107, 0.029, 0.039], +[0.021, 0.010, 0.010], +[0.015, 0.002, 0.002], +[1.238, 0.359, 0.360], +[0.573, 0.324, 0.311], +[0.032, 0.003, 0.003], +[0.032, 0.018, 0.013], +[0.133, 0.106, 0.106], +[3.420, 0.095, 0.092], +[3.048, 0.746, 0.735], +[2.274, 0.064, 0.058], +[1.277, 0.050, 0.056], +[4.456, 0.250, 0.240], +[4.146, 0.197, 0.183], +[1.821, 0.223, 0.220], +[63.395, 2.248, 2.241], +[42.449, 1.517, 1.519], +[3.568, 0.798, 0.795], +[0.793, 0.147, 0.142], +[0.128, 0.081, 0.066], +[3.217, 0.085, 0.096], +[0.026, 0.006, 0.005], +[0.027, 0.008, 0.009], +[0.363, 0.025, 0.024], +[1.716, 0.034, 0.030], +[1.683, 0.099, 0.086], +[2.517, 0.216, 0.197], +[0.030, 0.014, 0.015], +[0.018, 0.009, 0.008], +[2.142, 0.410, 0.407], +[2.009, 0.449, 0.435], +[1.020, 0.125, 0.123], +[1.677, 0.146, 0.152], +[2.577, 0.468, 0.488], +[3.794, 0.572, 0.550], +[2.362, 0.567, 0.528], +[1.204, 0.544, 0.548], +[4.506, 1.420, 1.415], +[3.668, 0.804, 0.803], +[7.920, 3.041, 3.031], +[0.917, 0.032, 0.043], +[19.548, 0.498, 0.477], +[21.481, 0.573, 0.563], +[40.165, 1.294, 1.261], +[39.503, 0.619, 0.606], +[4.642, 0.207, 0.203], +[2.278, 0.169, 0.170], +[5.301, 0.209, 0.214], +[19.193, 0.451, 0.458], +[15.769, 0.793, 0.782], +[1.171, 1.139, 1.129], +[3.804, 0.511, 0.511], +[9.797, 0.724, 0.704], +[9.091, 4.112, 4.069], +[19.481, 1.974, 1.937], +[19.489, 2.049, 1.976], +[0.849, 0.820, 0.830], +[0.144, 0.117, 0.115], +[0.068, 0.052, 0.051], +[0.075, 0.035, 0.040], +[0.321, 0.263, 0.267], +[0.052, 0.018, 0.024], +[0.032, 0.012, 0.012], +[0.012, 0.011, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/19.17.10.1_adaptive.json b/website/benchmark/versions/results/19.17.10.1_adaptive.json new file mode 100644 index 00000000000..e7471d40b3f --- /dev/null +++ b/website/benchmark/versions/results/19.17.10.1_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-07-22 19.17", + "system_full": "ClickHouse 19.17.10.1 2020-07-22(adaptive)", + "version": "19.17.10.1", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.032, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.014, 0.002, 0.002], +[0.016, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.018, 0.002, 0.002], +[0.144, 0.030, 0.038], +[0.023, 0.011, 0.011], +[0.014, 0.002, 0.002], +[1.242, 0.340, 0.347], +[0.605, 0.325, 0.327], +[0.033, 0.003, 0.003], +[0.031, 0.014, 0.017], +[0.130, 0.107, 0.108], +[3.419, 0.104, 0.095], +[3.005, 0.751, 0.754], +[2.318, 0.055, 0.049], +[1.282, 0.048, 0.040], +[4.472, 0.235, 0.219], +[4.206, 0.171, 0.171], +[1.862, 0.212, 0.205], +[63.410, 2.035, 2.023], +[42.454, 1.429, 1.427], +[3.558, 0.811, 0.802], +[0.771, 0.168, 0.149], +[0.123, 0.089, 0.070], +[3.175, 0.092, 0.102], +[0.010, 0.001, 0.001], +[0.026, 0.009, 0.009], +[0.340, 0.025, 0.024], +[1.713, 0.031, 0.031], +[1.689, 0.089, 0.079], +[2.601, 0.214, 0.197], +[0.025, 0.011, 0.011], +[0.018, 0.009, 0.009], +[2.105, 0.415, 0.411], +[2.001, 0.454, 0.458], +[1.032, 0.125, 0.124], +[1.659, 0.150, 0.161], +[2.585, 0.486, 0.474], +[3.798, 0.580, 0.578], +[2.360, 0.558, 0.542], +[1.197, 0.558, 0.563], +[4.244, 1.423, 1.425], +[3.638, 0.813, 0.807], +[7.920, 3.054, 3.030], +[0.909, 0.060, 0.040], +[19.522, 0.477, 0.454], +[21.467, 0.575, 0.563], +[40.174, 1.311, 1.269], +[39.523, 0.619, 0.584], +[4.649, 0.217, 0.201], +[2.274, 0.173, 0.167], +[5.286, 0.227, 0.216], +[19.157, 0.467, 0.464], +[15.790, 0.783, 0.784], +[1.156, 1.141, 1.129], +[3.850, 0.488, 0.518], +[9.832, 0.722, 0.685], +[9.068, 4.054, 4.058], +[19.490, 2.038, 2.035], +[19.504, 2.129, 1.977], +[0.910, 0.842, 0.848], +[0.166, 0.142, 0.122], +[0.078, 0.050, 0.048], +[0.069, 0.038, 0.042], +[0.335, 0.255, 0.256], +[0.053, 0.025, 0.019], +[0.031, 0.012, 0.022], +[0.012, 0.003, 0.002] + ] + } +] diff --git a/website/benchmark/versions/results/20.01.16.120_adaptive.json b/website/benchmark/versions/results/20.01.16.120_adaptive.json new file mode 100644 index 00000000000..91c22297b07 --- /dev/null +++ b/website/benchmark/versions/results/20.01.16.120_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-06-26 20.1", + "system_full": "ClickHouse 20.1.16.120 2020-06-26(adaptive)", + "version": "20.1.16.120", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.036, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.017, 0.004, 0.004], +[0.019, 0.002, 0.002], +[0.017, 0.002, 0.002], +[0.015, 0.003, 0.003], +[0.018, 0.002, 0.002], +[0.238, 0.032, 0.022], +[0.018, 0.009, 0.009], +[0.014, 0.002, 0.002], +[1.227, 0.336, 0.336], +[0.623, 0.346, 0.328], +[0.032, 0.003, 0.003], +[0.031, 0.014, 0.014], +[0.127, 0.095, 0.101], +[3.415, 0.109, 0.108], +[3.271, 1.135, 1.135], +[2.268, 0.057, 0.065], +[1.415, 0.049, 0.055], +[4.452, 0.210, 0.223], +[4.192, 0.195, 0.195], +[1.769, 0.238, 0.230], +[63.306, 2.245, 2.191], +[42.435, 1.591, 1.558], +[3.901, 1.178, 1.182], +[0.778, 0.194, 0.143], +[0.115, 0.076, 0.094], +[4.628, 0.097, 0.095], +[0.012, 0.001, 0.001], +[0.029, 0.009, 0.008], +[0.268, 0.027, 0.025], +[1.709, 0.034, 0.032], +[1.662, 0.093, 0.095], +[2.554, 0.222, 0.208], +[0.033, 0.015, 0.016], +[0.018, 0.009, 0.008], +[2.092, 0.425, 0.411], +[2.011, 0.461, 0.452], +[1.012, 0.130, 0.128], +[1.636, 0.160, 0.154], +[2.553, 0.501, 0.482], +[3.775, 0.595, 0.568], +[2.336, 0.548, 0.518], +[1.217, 0.554, 0.552], +[4.248, 1.415, 1.410], +[3.650, 0.815, 0.799], +[7.889, 2.996, 2.999], +[0.906, 0.035, 0.038], +[19.528, 0.487, 0.479], +[21.456, 0.586, 0.560], +[40.019, 1.302, 1.277], +[39.505, 0.621, 0.584], +[4.627, 0.186, 0.184], +[2.321, 0.152, 0.153], +[5.298, 0.193, 0.192], +[19.189, 0.460, 0.444], +[15.722, 0.766, 0.757], +[1.193, 1.173, 1.169], +[3.824, 0.529, 0.510], +[9.801, 0.735, 0.711], +[9.054, 4.012, 4.011], +[19.468, 2.084, 2.060], +[19.447, 2.108, 2.078], +[0.907, 0.857, 0.845], +[0.174, 0.146, 0.130], +[0.067, 0.045, 0.048], +[0.074, 0.047, 0.035], +[0.314, 0.237, 0.260], +[0.056, 0.013, 0.022], +[0.043, 0.010, 0.010], +[0.014, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.03.21.2_adaptive.json b/website/benchmark/versions/results/20.03.21.2_adaptive.json new file mode 100644 index 00000000000..324c7f76139 --- /dev/null +++ b/website/benchmark/versions/results/20.03.21.2_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-11-02 20.3", + "system_full": "ClickHouse 20.3.21.2 2020-11-02(adaptive)", + "version": "20.3.21.2", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.038, 0.003, 0.003], +[0.018, 0.002, 0.002], +[0.021, 0.004, 0.004], +[0.019, 0.002, 0.002], +[0.018, 0.002, 0.002], +[0.015, 0.003, 0.003], +[0.019, 0.002, 0.002], +[0.067, 0.028, 0.027], +[0.021, 0.009, 0.009], +[0.012, 0.002, 0.002], +[1.232, 0.332, 0.336], +[0.595, 0.316, 0.319], +[0.036, 0.004, 0.004], +[0.032, 0.014, 0.013], +[0.156, 0.098, 0.097], +[3.409, 0.100, 0.097], +[3.255, 1.130, 1.118], +[2.251, 0.063, 0.061], +[1.402, 0.052, 0.054], +[4.425, 0.245, 0.234], +[4.160, 0.189, 0.191], +[1.763, 0.228, 0.227], +[63.662, 2.106, 2.094], +[42.534, 1.413, 1.384], +[3.879, 1.163, 1.163], +[0.785, 0.185, 0.148], +[0.120, 0.081, 0.071], +[4.663, 0.110, 0.089], +[0.011, 0.001, 0.001], +[0.027, 0.010, 0.009], +[0.341, 0.025, 0.022], +[1.719, 0.033, 0.032], +[1.687, 0.085, 0.084], +[2.584, 0.196, 0.191], +[0.028, 0.015, 0.016], +[0.019, 0.009, 0.009], +[2.053, 0.315, 0.317], +[2.037, 0.355, 0.347], +[1.172, 0.122, 0.125], +[1.671, 0.145, 0.155], +[2.552, 0.483, 0.507], +[3.795, 0.557, 0.556], +[2.323, 0.516, 0.548], +[1.184, 0.526, 0.525], +[4.187, 1.384, 1.385], +[3.674, 0.827, 0.822], +[7.857, 2.964, 2.991], +[0.905, 0.062, 0.037], +[19.548, 0.516, 0.473], +[21.492, 0.549, 0.543], +[40.083, 1.304, 1.276], +[39.617, 0.624, 0.602], +[4.629, 0.186, 0.192], +[2.391, 0.157, 0.163], +[5.297, 0.195, 0.195], +[19.173, 0.464, 0.467], +[15.765, 0.794, 0.789], +[1.204, 1.180, 1.175], +[3.823, 0.503, 0.493], +[9.800, 0.708, 0.710], +[9.058, 4.009, 3.968], +[19.437, 1.976, 1.998], +[19.368, 2.031, 2.052], +[0.884, 0.838, 0.831], +[0.169, 0.117, 0.115], +[0.067, 0.046, 0.042], +[0.079, 0.042, 0.044], +[0.293, 0.245, 0.233], +[0.050, 0.014, 0.019], +[0.029, 0.018, 0.029], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.04.9.110_adaptive.json b/website/benchmark/versions/results/20.04.9.110_adaptive.json new file mode 100644 index 00000000000..1e286c9f984 --- /dev/null +++ b/website/benchmark/versions/results/20.04.9.110_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-08-20 20.4", + "system_full": "ClickHouse 20.4.9.110 2020-08-20(adaptive)", + "version": "20.4.9.110", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.034, 0.003, 0.003], +[0.061, 0.002, 0.002], +[0.018, 0.004, 0.004], +[0.019, 0.002, 0.002], +[0.016, 0.002, 0.002], +[0.017, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.058, 0.030, 0.026], +[0.021, 0.009, 0.008], +[0.011, 0.002, 0.002], +[1.115, 0.329, 0.321], +[0.626, 0.338, 0.330], +[0.032, 0.004, 0.004], +[0.030, 0.014, 0.013], +[0.118, 0.129, 0.097], +[3.428, 0.103, 0.094], +[3.001, 0.470, 0.466], +[2.266, 0.056, 0.061], +[1.414, 0.056, 0.050], +[4.436, 0.206, 0.211], +[4.218, 0.170, 0.166], +[1.825, 0.218, 0.218], +[63.353, 2.084, 2.009], +[42.428, 1.361, 1.341], +[3.240, 0.513, 0.515], +[0.769, 0.152, 0.151], +[0.107, 0.068, 0.069], +[4.636, 0.092, 0.089], +[0.012, 0.001, 0.001], +[0.035, 0.010, 0.010], +[0.262, 0.024, 0.024], +[1.695, 0.033, 0.029], +[1.671, 0.099, 0.091], +[2.564, 0.220, 0.207], +[0.027, 0.012, 0.012], +[0.020, 0.010, 0.010], +[1.986, 0.327, 0.330], +[2.008, 0.365, 0.362], +[0.997, 0.122, 0.121], +[1.657, 0.154, 0.151], +[2.537, 0.467, 0.466], +[3.767, 0.572, 0.559], +[2.315, 0.521, 0.546], +[1.126, 0.519, 0.540], +[4.176, 1.361, 1.375], +[3.667, 0.824, 0.819], +[7.799, 2.987, 2.913], +[0.890, 0.058, 0.043], +[19.493, 0.543, 0.513], +[21.321, 0.585, 0.575], +[39.970, 1.294, 1.275], +[39.519, 0.656, 0.618], +[4.625, 0.179, 0.174], +[2.327, 0.150, 0.153], +[5.305, 0.202, 0.197], +[19.199, 0.458, 0.447], +[15.612, 0.784, 0.773], +[1.152, 1.136, 1.134], +[3.821, 0.470, 0.470], +[9.794, 0.712, 0.688], +[9.036, 4.000, 3.941], +[19.410, 1.997, 1.967], +[19.237, 2.013, 2.008], +[0.884, 0.828, 0.831], +[0.172, 0.135, 0.117], +[0.067, 0.046, 0.051], +[0.072, 0.037, 0.041], +[0.293, 0.220, 0.243], +[0.053, 0.026, 0.013], +[0.032, 0.021, 0.015], +[0.015, 0.003, 0.010] + ] + } +] diff --git a/website/benchmark/versions/results/20.05.5.74_adaptive.json b/website/benchmark/versions/results/20.05.5.74_adaptive.json new file mode 100644 index 00000000000..cbed25cdf08 --- /dev/null +++ b/website/benchmark/versions/results/20.05.5.74_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-08-20 20.5", + "system_full": "ClickHouse 20.5.5.74 2020-08-20(adaptive)", + "version": "20.5.5.74", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.017, 0.003, 0.003], +[0.012, 0.002, 0.002], +[0.017, 0.004, 0.004], +[0.011, 0.002, 0.002], +[0.014, 0.002, 0.002], +[0.010, 0.003, 0.003], +[0.019, 0.002, 0.002], +[0.075, 0.029, 0.026], +[0.018, 0.008, 0.008], +[0.009, 0.002, 0.002], +[0.987, 0.327, 0.319], +[0.649, 0.326, 0.291], +[0.034, 0.004, 0.004], +[0.031, 0.014, 0.014], +[0.139, 0.101, 0.097], +[3.401, 0.097, 0.090], +[2.982, 0.493, 0.470], +[2.267, 0.061, 0.054], +[1.421, 0.050, 0.046], +[4.444, 0.206, 0.206], +[4.235, 0.170, 0.171], +[1.830, 0.210, 0.213], +[63.495, 1.978, 1.955], +[42.433, 1.320, 1.303], +[3.312, 0.528, 0.515], +[0.879, 0.168, 0.163], +[0.115, 0.065, 0.071], +[4.676, 0.097, 0.093], +[0.010, 0.001, 0.001], +[0.027, 0.009, 0.009], +[0.313, 0.025, 0.023], +[1.700, 0.031, 0.029], +[1.684, 0.096, 0.096], +[2.569, 0.221, 0.207], +[0.027, 0.013, 0.012], +[0.020, 0.010, 0.009], +[1.965, 0.327, 0.331], +[2.019, 0.371, 0.363], +[1.054, 0.124, 0.121], +[1.669, 0.157, 0.144], +[2.538, 0.442, 0.427], +[3.784, 0.537, 0.544], +[2.356, 0.512, 0.524], +[1.179, 0.526, 0.516], +[4.170, 1.364, 1.314], +[3.675, 0.821, 0.795], +[7.808, 2.835, 2.806], +[0.897, 0.049, 0.040], +[19.506, 0.490, 0.478], +[21.397, 0.535, 0.574], +[39.973, 1.263, 1.231], +[39.497, 0.623, 0.580], +[4.624, 0.180, 0.177], +[2.332, 0.149, 0.149], +[5.305, 0.183, 0.185], +[19.203, 0.456, 0.443], +[15.583, 0.771, 0.783], +[1.159, 1.134, 1.123], +[3.818, 0.443, 0.460], +[9.802, 0.690, 0.662], +[9.004, 3.952, 3.935], +[19.402, 1.989, 1.922], +[19.316, 1.972, 1.892], +[0.876, 0.826, 0.797], +[0.177, 0.119, 0.112], +[0.067, 0.043, 0.049], +[0.076, 0.044, 0.039], +[0.312, 0.238, 0.245], +[0.047, 0.022, 0.017], +[0.040, 0.014, 0.015], +[0.014, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.06.11.1_adaptive.json b/website/benchmark/versions/results/20.06.11.1_adaptive.json new file mode 100644 index 00000000000..816df9b93e5 --- /dev/null +++ b/website/benchmark/versions/results/20.06.11.1_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-12-09 20.6", + "system_full": "ClickHouse 20.6.11.1 2020-12-09(adaptive)", + "version": "2020-12-09", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.021, 0.004, 0.004], +[0.012, 0.002, 0.002], +[0.019, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.013, 0.003, 0.003], +[0.010, 0.003, 0.004], +[0.016, 0.002, 0.002], +[0.063, 0.029, 0.026], +[0.018, 0.008, 0.009], +[0.009, 0.002, 0.002], +[0.988, 0.318, 0.320], +[0.631, 0.304, 0.315], +[0.032, 0.004, 0.004], +[0.031, 0.014, 0.015], +[0.114, 0.102, 0.133], +[3.430, 0.098, 0.093], +[3.001, 0.496, 0.488], +[2.257, 0.057, 0.059], +[1.412, 0.054, 0.054], +[4.430, 0.215, 0.214], +[4.211, 0.174, 0.172], +[1.831, 0.216, 0.217], +[63.359, 2.025, 2.017], +[42.442, 1.349, 1.339], +[3.265, 0.533, 0.540], +[0.796, 0.155, 0.178], +[0.117, 0.070, 0.065], +[4.640, 0.093, 0.101], +[0.011, 0.002, 0.001], +[0.027, 0.011, 0.010], +[0.322, 0.022, 0.025], +[1.729, 0.031, 0.031], +[1.692, 0.095, 0.092], +[2.558, 0.212, 0.196], +[0.025, 0.015, 0.013], +[0.020, 0.010, 0.010], +[2.065, 0.340, 0.335], +[2.015, 0.380, 0.378], +[1.016, 0.132, 0.131], +[1.644, 0.161, 0.156], +[2.509, 0.441, 0.447], +[3.769, 0.577, 0.530], +[2.357, 0.538, 0.530], +[1.175, 0.545, 0.532], +[4.177, 1.338, 1.320], +[3.671, 0.800, 0.804], +[7.821, 2.833, 2.787], +[0.908, 0.122, 0.075], +[19.423, 0.477, 0.463], +[21.438, 0.591, 0.545], +[39.980, 1.269, 1.253], +[39.490, 0.625, 0.589], +[4.620, 0.196, 0.182], +[2.322, 0.155, 0.158], +[5.295, 0.193, 0.198], +[19.194, 0.452, 0.449], +[15.789, 0.785, 0.765], +[1.160, 1.127, 1.128], +[3.854, 0.470, 0.451], +[9.914, 0.727, 0.703], +[9.172, 4.111, 4.067], +[19.409, 2.006, 1.927], +[19.235, 1.968, 1.900], +[0.868, 0.831, 0.798], +[0.173, 0.116, 0.116], +[0.072, 0.047, 0.048], +[0.077, 0.039, 0.045], +[0.294, 0.227, 0.228], +[0.048, 0.016, 0.025], +[0.037, 0.026, 0.018], +[0.015, 0.012, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.07.4.11_adaptive.json b/website/benchmark/versions/results/20.07.4.11_adaptive.json new file mode 100644 index 00000000000..e4e14d1b4ed --- /dev/null +++ b/website/benchmark/versions/results/20.07.4.11_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-10-09 20.7", + "system_full": "ClickHouse 20.7.4.11 2020-10-09(adaptive)", + "version": "20.7.4.11", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.017, 0.004, 0.004], +[0.012, 0.002, 0.002], +[0.017, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.013, 0.003, 0.003], +[0.010, 0.003, 0.003], +[0.016, 0.002, 0.003], +[0.073, 0.030, 0.027], +[0.019, 0.008, 0.009], +[0.009, 0.002, 0.003], +[1.109, 0.324, 0.320], +[0.639, 0.323, 0.287], +[0.031, 0.004, 0.004], +[0.031, 0.014, 0.014], +[0.119, 0.136, 0.094], +[3.426, 0.109, 0.100], +[2.994, 0.161, 0.161], +[2.740, 0.063, 0.060], +[1.402, 0.052, 0.054], +[4.421, 0.187, 0.190], +[4.250, 0.157, 0.138], +[1.880, 0.219, 0.201], +[63.407, 1.970, 1.960], +[42.465, 1.336, 1.321], +[2.928, 0.207, 0.218], +[1.134, 0.155, 0.142], +[0.119, 0.070, 0.072], +[4.686, 0.089, 0.089], +[0.010, 0.001, 0.002], +[0.029, 0.010, 0.010], +[0.322, 0.024, 0.026], +[1.696, 0.033, 0.030], +[1.681, 0.094, 0.092], +[2.576, 0.204, 0.197], +[0.029, 0.015, 0.015], +[0.020, 0.010, 0.011], +[2.019, 0.330, 0.328], +[2.029, 0.370, 0.378], +[1.031, 0.126, 0.134], +[1.638, 0.150, 0.150], +[2.582, 0.454, 0.413], +[3.750, 0.550, 0.518], +[2.374, 0.533, 0.504], +[1.189, 0.526, 0.509], +[4.192, 1.329, 1.299], +[3.656, 0.811, 0.788], +[7.830, 2.852, 2.762], +[0.888, 0.128, 0.056], +[19.419, 0.541, 0.491], +[21.463, 0.543, 0.536], +[40.010, 1.276, 1.247], +[39.537, 0.660, 0.607], +[4.633, 0.192, 0.189], +[2.296, 0.150, 0.154], +[5.319, 0.186, 0.181], +[19.258, 0.449, 0.449], +[15.793, 0.767, 0.766], +[1.147, 1.125, 1.121], +[3.821, 0.474, 0.466], +[9.793, 0.690, 0.666], +[9.069, 3.982, 4.006], +[19.417, 1.995, 1.882], +[19.408, 1.973, 1.894], +[0.870, 0.807, 0.799], +[0.148, 0.119, 0.111], +[0.075, 0.046, 0.052], +[0.072, 0.041, 0.039], +[0.318, 0.230, 0.226], +[0.068, 0.018, 0.025], +[0.029, 0.015, 0.014], +[0.018, 0.004, 0.004] + ] + } +] diff --git a/website/benchmark/versions/results/20.08.19.4.json b/website/benchmark/versions/results/20.08.19.4.json new file mode 100644 index 00000000000..d2b2b41ceb0 --- /dev/null +++ b/website/benchmark/versions/results/20.08.19.4.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-07-11 20.8", + "system_full": "ClickHouse 20.8.19.4 2021-07-11", + "version": "20.8.19.4", + "kind": "", + "comments": "", + "result": + [ + +[0.042, 0.011, 0.012], +[0.064, 0.031, 0.032], +[0.037, 0.010, 0.010], +[0.038, 0.015, 0.015], +[0.053, 0.033, 0.029], +[0.065, 0.016, 0.017], +[0.060, 0.005, 0.006], +[0.335, 0.031, 0.029], +[0.498, 0.052, 0.053], +[0.507, 0.082, 0.084], +[1.930, 0.553, 0.551], +[1.488, 0.498, 0.476], +[0.029, 0.005, 0.005], +[0.030, 0.014, 0.014], +[0.133, 0.106, 0.102], +[3.315, 0.116, 0.103], +[2.560, 0.138, 0.145], +[2.209, 0.087, 0.063], +[1.218, 0.061, 0.056], +[5.103, 0.201, 0.192], +[4.795, 0.166, 0.161], +[1.884, 0.174, 0.196], +[63.515, 2.121, 2.103], +[42.496, 1.428, 1.386], +[4.439, 0.193, 0.187], +[1.458, 0.156, 0.155], +[0.238, 0.057, 0.063], +[5.102, 0.088, 0.085], +[0.008, 0.002, 0.002], +[0.029, 0.011, 0.011], +[0.479, 0.028, 0.026], +[1.723, 0.028, 0.031], +[1.730, 0.091, 0.093], +[2.545, 0.199, 0.195], +[0.029, 0.016, 0.016], +[0.021, 0.010, 0.010], +[2.223, 0.345, 0.329], +[2.234, 0.380, 0.382], +[1.029, 0.115, 0.113], +[1.703, 0.146, 0.138], +[2.568, 0.452, 0.430], +[3.769, 0.545, 0.548], +[2.370, 0.542, 0.518], +[1.239, 0.526, 0.528], +[4.186, 1.388, 1.335], +[3.652, 0.845, 0.824], +[7.822, 2.888, 2.870], +[0.937, 0.134, 0.052], +[18.851, 0.550, 0.538], +[20.853, 0.578, 0.608], +[38.786, 1.395, 1.372], +[38.926, 0.706, 0.656], +[4.545, 0.182, 0.184], +[2.309, 0.143, 0.142], +[5.274, 0.182, 0.182], +[18.667, 0.502, 0.524], +[15.385, 0.807, 0.816], +[1.203, 1.191, 1.180], +[3.875, 0.457, 0.468], +[9.798, 0.714, 0.673], +[9.077, 3.991, 3.987], +[18.866, 2.138, 2.090], +[18.921, 2.146, 2.070], +[0.904, 0.854, 0.846], +[0.118, 0.088, 0.114], +[0.061, 0.043, 0.039], +[0.058, 0.045, 0.037], +[0.212, 0.188, 0.161], +[0.046, 0.014, 0.012], +[0.026, 0.021, 0.009], +[0.015, 0.004, 0.011] + ] + } +] diff --git a/website/benchmark/versions/results/20.08.19.4_adaptive.json b/website/benchmark/versions/results/20.08.19.4_adaptive.json new file mode 100644 index 00000000000..4f1ef56cdd0 --- /dev/null +++ b/website/benchmark/versions/results/20.08.19.4_adaptive.json @@ -0,0 +1,86 @@ + +[ + { + "system": "2021-07-11 20.8", + "system_full": "ClickHouse 20.8.19.4 2021-07-11(adaptive)", + "version": "20.8.19.4", + "kind": "adaptive", + "kind": "", + "comments": "", + "result": + [ + +[0.021, 0.004, 0.004], +[0.013, 0.002, 0.002], +[0.017, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.013, 0.002, 0.003], +[0.011, 0.003, 0.003], +[0.018, 0.002, 0.002], +[0.076, 0.028, 0.021], +[0.018, 0.009, 0.008], +[0.010, 0.002, 0.002], +[1.009, 0.331, 0.329], +[0.604, 0.311, 0.319], +[0.029, 0.004, 0.004], +[0.033, 0.014, 0.014], +[0.119, 0.096, 0.098], +[3.418, 0.102, 0.098], +[3.006, 0.165, 0.157], +[2.750, 0.060, 0.066], +[1.390, 0.050, 0.051], +[4.429, 0.202, 0.214], +[4.188, 0.150, 0.159], +[1.854, 0.200, 0.206], +[63.423, 2.078, 2.029], +[42.477, 1.373, 1.340], +[2.925, 0.205, 0.209], +[1.147, 0.154, 0.146], +[0.109, 0.071, 0.085], +[4.647, 0.088, 0.091], +[0.010, 0.001, 0.001], +[0.027, 0.009, 0.010], +[0.339, 0.024, 0.025], +[1.723, 0.031, 0.031], +[1.670, 0.093, 0.091], +[2.589, 0.205, 0.212], +[0.035, 0.016, 0.017], +[0.020, 0.011, 0.010], +[1.997, 0.337, 0.323], +[2.027, 0.359, 0.365], +[1.043, 0.118, 0.116], +[1.673, 0.149, 0.146], +[2.558, 0.453, 0.425], +[3.782, 0.546, 0.525], +[2.372, 0.525, 0.502], +[1.178, 0.529, 0.515], +[4.370, 1.352, 1.305], +[3.655, 0.798, 0.791], +[7.837, 2.829, 2.802], +[0.896, 0.153, 0.086], +[19.375, 0.554, 0.506], +[21.462, 0.552, 0.559], +[40.026, 1.272, 1.253], +[39.523, 0.656, 0.638], +[4.624, 0.190, 0.191], +[2.287, 0.149, 0.148], +[5.342, 0.186, 0.191], +[19.237, 0.460, 0.446], +[15.673, 0.782, 0.759], +[1.148, 1.127, 1.125], +[3.812, 0.452, 0.447], +[9.773, 0.672, 0.652], +[8.977, 3.897, 3.828], +[19.403, 1.921, 1.925], +[19.249, 1.966, 1.929], +[0.863, 0.810, 0.787], +[0.144, 0.115, 0.114], +[0.078, 0.048, 0.045], +[0.068, 0.044, 0.045], +[0.291, 0.245, 0.221], +[0.062, 0.015, 0.014], +[0.042, 0.014, 0.011], +[0.015, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.09.7.11_adaptive.json b/website/benchmark/versions/results/20.09.7.11_adaptive.json new file mode 100644 index 00000000000..cd2bbae0158 --- /dev/null +++ b/website/benchmark/versions/results/20.09.7.11_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-12-07 20.9", + "system_full": "ClickHouse 20.9.7.11 2020-12-07(adaptive)", + "version": "20.9.7.11", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.020, 0.004, 0.004], +[0.011, 0.002, 0.002], +[0.017, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.015, 0.003, 0.003], +[0.011, 0.003, 0.003], +[0.019, 0.002, 0.002], +[0.059, 0.031, 0.030], +[0.019, 0.009, 0.008], +[0.011, 0.002, 0.002], +[0.863, 0.338, 0.338], +[0.583, 0.306, 0.311], +[0.028, 0.004, 0.005], +[0.030, 0.014, 0.014], +[0.151, 0.093, 0.096], +[3.419, 0.110, 0.096], +[2.907, 0.154, 0.151], +[2.774, 0.054, 0.060], +[1.319, 0.044, 0.048], +[4.413, 0.185, 0.189], +[4.250, 0.147, 0.153], +[1.861, 0.204, 0.191], +[63.419, 2.030, 1.990], +[42.508, 1.353, 1.333], +[2.928, 0.208, 0.208], +[1.150, 0.179, 0.157], +[0.105, 0.079, 0.067], +[4.658, 0.097, 0.092], +[0.010, 0.002, 0.002], +[0.029, 0.010, 0.010], +[0.303, 0.025, 0.024], +[1.725, 0.031, 0.030], +[1.665, 0.085, 0.083], +[2.586, 0.197, 0.192], +[0.029, 0.016, 0.016], +[0.018, 0.010, 0.010], +[2.014, 0.309, 0.306], +[2.069, 0.340, 0.341], +[1.097, 0.121, 0.116], +[1.658, 0.142, 0.141], +[2.564, 0.470, 0.439], +[3.798, 0.538, 0.529], +[2.328, 0.537, 0.492], +[1.157, 0.515, 0.514], +[4.157, 1.346, 1.311], +[3.663, 0.796, 0.777], +[7.840, 2.878, 2.741], +[0.899, 0.126, 0.048], +[19.450, 0.536, 0.485], +[21.523, 0.539, 0.531], +[40.028, 1.275, 1.236], +[39.551, 0.647, 0.618], +[4.656, 0.178, 0.176], +[2.335, 0.151, 0.139], +[5.338, 0.176, 0.177], +[19.315, 0.452, 0.433], +[15.803, 0.770, 0.759], +[1.150, 1.122, 1.129], +[3.845, 0.449, 0.457], +[9.791, 0.689, 0.666], +[9.005, 3.932, 3.850], +[19.470, 1.954, 1.916], +[19.262, 1.966, 1.901], +[0.873, 0.798, 0.802], +[0.154, 0.114, 0.113], +[0.066, 0.041, 0.042], +[0.077, 0.056, 0.042], +[0.294, 0.243, 0.245], +[0.053, 0.016, 0.015], +[0.043, 0.011, 0.024], +[0.015, 0.005, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.10.7.4_adaptive.json b/website/benchmark/versions/results/20.10.7.4_adaptive.json new file mode 100644 index 00000000000..166c3d462ac --- /dev/null +++ b/website/benchmark/versions/results/20.10.7.4_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2020-12-24 20.10", + "system_full": "ClickHouse 20.10.7.4 2020-12-24(adaptive)", + "version": "20.10.7.4", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.019, 0.004, 0.004], +[0.011, 0.002, 0.002], +[0.019, 0.004, 0.004], +[0.009, 0.002, 0.002], +[0.013, 0.002, 0.002], +[0.010, 0.004, 0.004], +[0.017, 0.002, 0.002], +[0.078, 0.028, 0.024], +[0.016, 0.011, 0.008], +[0.010, 0.002, 0.002], +[1.227, 0.346, 0.338], +[0.579, 0.341, 0.335], +[0.031, 0.004, 0.004], +[0.032, 0.015, 0.019], +[0.125, 0.097, 0.099], +[3.426, 0.101, 0.098], +[3.020, 0.149, 0.146], +[2.789, 0.060, 0.057], +[1.430, 0.049, 0.058], +[4.427, 0.242, 0.224], +[4.169, 0.193, 0.186], +[1.818, 0.238, 0.239], +[63.459, 2.557, 2.539], +[42.438, 1.662, 1.627], +[2.923, 0.209, 0.199], +[1.165, 0.178, 0.154], +[0.120, 0.082, 0.087], +[4.645, 0.105, 0.120], +[0.011, 0.001, 0.001], +[0.026, 0.011, 0.010], +[0.262, 0.023, 0.022], +[1.686, 0.031, 0.032], +[1.681, 0.095, 0.092], +[2.548, 0.238, 0.227], +[0.029, 0.015, 0.015], +[0.019, 0.010, 0.010], +[1.941, 0.320, 0.316], +[2.027, 0.356, 0.362], +[1.055, 0.146, 0.146], +[1.618, 0.170, 0.182], +[2.513, 0.484, 0.423], +[3.774, 0.555, 0.534], +[2.363, 0.566, 0.516], +[1.180, 0.534, 0.544], +[4.235, 1.425, 1.384], +[3.644, 0.819, 0.801], +[8.000, 3.018, 2.921], +[0.879, 0.165, 0.035], +[19.441, 0.484, 0.463], +[21.474, 0.583, 0.606], +[40.105, 1.336, 1.341], +[39.496, 0.607, 0.587], +[4.623, 0.194, 0.194], +[2.320, 0.167, 0.162], +[5.290, 0.189, 0.200], +[19.186, 0.507, 0.486], +[15.783, 0.730, 0.715], +[1.133, 1.110, 1.115], +[3.838, 0.471, 0.469], +[9.799, 0.692, 0.677], +[9.014, 3.920, 3.907], +[19.380, 1.999, 1.962], +[19.270, 1.979, 1.950], +[0.844, 0.775, 0.759], +[0.169, 0.110, 0.116], +[0.064, 0.041, 0.048], +[0.091, 0.042, 0.037], +[0.301, 0.245, 0.226], +[0.050, 0.021, 0.020], +[0.034, 0.011, 0.014], +[0.015, 0.004, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.11.7.16_adaptive.json b/website/benchmark/versions/results/20.11.7.16_adaptive.json new file mode 100644 index 00000000000..bf706fbdeb1 --- /dev/null +++ b/website/benchmark/versions/results/20.11.7.16_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-02-03 20.11", + "system_full": "ClickHouse 20.11.7.16 2021-02-03(adaptive)", + "version": "2021-02-03", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.023, 0.004, 0.004], +[0.013, 0.002, 0.002], +[0.019, 0.005, 0.004], +[0.009, 0.002, 0.002], +[0.013, 0.003, 0.003], +[0.010, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.085, 0.030, 0.027], +[0.016, 0.009, 0.009], +[0.009, 0.002, 0.002], +[1.034, 0.337, 0.347], +[0.558, 0.330, 0.313], +[0.036, 0.004, 0.004], +[0.032, 0.016, 0.015], +[0.118, 0.099, 0.103], +[3.423, 0.102, 0.097], +[3.129, 0.146, 0.139], +[2.774, 0.064, 0.056], +[1.368, 0.058, 0.050], +[4.421, 0.259, 0.226], +[4.154, 0.204, 0.192], +[1.790, 0.239, 0.236], +[63.324, 2.536, 2.532], +[42.446, 1.660, 1.645], +[2.928, 0.211, 0.206], +[1.151, 0.137, 0.168], +[0.116, 0.079, 0.072], +[4.668, 0.125, 0.118], +[0.010, 0.001, 0.001], +[0.036, 0.010, 0.010], +[0.268, 0.020, 0.021], +[1.723, 0.032, 0.032], +[1.674, 0.104, 0.091], +[2.554, 0.237, 0.248], +[0.029, 0.016, 0.016], +[0.019, 0.009, 0.009], +[1.927, 0.319, 0.325], +[1.997, 0.363, 0.351], +[1.071, 0.147, 0.145], +[1.617, 0.176, 0.188], +[2.512, 0.455, 0.465], +[3.782, 0.551, 0.542], +[2.367, 0.530, 0.557], +[1.195, 0.560, 0.527], +[4.303, 1.411, 1.411], +[3.652, 0.830, 0.811], +[7.991, 3.038, 2.955], +[0.892, 0.143, 0.039], +[19.450, 0.493, 0.506], +[21.273, 0.604, 0.617], +[40.013, 1.335, 1.313], +[39.502, 0.628, 0.578], +[4.624, 0.202, 0.200], +[2.295, 0.158, 0.163], +[5.295, 0.193, 0.187], +[19.195, 0.524, 0.500], +[15.579, 0.730, 0.731], +[1.139, 1.114, 1.112], +[3.826, 0.492, 0.481], +[9.809, 0.707, 0.677], +[8.992, 3.956, 3.902], +[19.401, 1.989, 1.952], +[19.272, 1.963, 1.901], +[0.811, 0.772, 0.765], +[0.140, 0.110, 0.108], +[0.065, 0.047, 0.049], +[0.074, 0.040, 0.046], +[0.294, 0.231, 0.226], +[0.047, 0.021, 0.020], +[0.039, 0.011, 0.025], +[0.012, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/20.12.8.5_adaptive.json b/website/benchmark/versions/results/20.12.8.5_adaptive.json new file mode 100644 index 00000000000..8322c449144 --- /dev/null +++ b/website/benchmark/versions/results/20.12.8.5_adaptive.json @@ -0,0 +1,86 @@ + +[ + { + "system": "2021-03-02 20.12", + "system_full": "ClickHouse 20.12.8.5 2021-03-02(adaptive)", + "version": "2021-03-02", + "time": "2021-03-02", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.019, 0.004, 0.004], +[0.011, 0.002, 0.002], +[0.015, 0.004, 0.005], +[0.009, 0.002, 0.002], +[0.014, 0.003, 0.003], +[0.010, 0.004, 0.003], +[0.016, 0.002, 0.002], +[0.076, 0.028, 0.020], +[0.016, 0.010, 0.008], +[0.014, 0.002, 0.002], +[1.224, 0.345, 0.336], +[0.600, 0.315, 0.331], +[0.029, 0.004, 0.004], +[0.033, 0.015, 0.016], +[0.120, 0.099, 0.095], +[3.426, 0.106, 0.119], +[3.117, 0.242, 0.234], +[2.595, 0.061, 0.052], +[1.436, 0.061, 0.060], +[4.433, 0.230, 0.222], +[4.213, 0.190, 0.205], +[1.810, 0.260, 0.239], +[63.332, 2.577, 2.558], +[42.473, 1.621, 1.564], +[3.025, 0.304, 0.309], +[0.956, 0.174, 0.170], +[0.126, 0.084, 0.104], +[4.624, 0.102, 0.131], +[0.011, 0.001, 0.001], +[0.027, 0.009, 0.010], +[0.240, 0.023, 0.021], +[1.711, 0.030, 0.027], +[1.681, 0.099, 0.100], +[2.541, 0.232, 0.235], +[0.029, 0.015, 0.015], +[0.017, 0.011, 0.010], +[2.008, 0.342, 0.326], +[2.037, 0.374, 0.358], +[1.067, 0.158, 0.156], +[1.612, 0.181, 0.167], +[2.496, 0.504, 0.427], +[3.784, 0.561, 0.557], +[2.386, 0.576, 0.543], +[1.213, 0.554, 0.538], +[4.249, 1.422, 1.398], +[3.692, 0.825, 0.814], +[8.007, 2.959, 2.938], +[0.919, 0.155, 0.035], +[19.417, 0.492, 0.457], +[21.492, 0.593, 0.566], +[40.016, 1.311, 1.271], +[39.555, 0.602, 0.589], +[4.636, 0.217, 0.203], +[2.307, 0.161, 0.166], +[5.319, 0.209, 0.206], +[19.166, 0.501, 0.482], +[15.822, 0.718, 0.725], +[1.096, 1.070, 1.073], +[3.831, 0.496, 0.493], +[9.821, 0.731, 0.688], +[9.067, 3.928, 3.881], +[19.446, 1.976, 1.898], +[19.444, 2.021, 1.946], +[0.819, 0.777, 0.765], +[0.146, 0.115, 0.114], +[0.065, 0.051, 0.044], +[0.071, 0.047, 0.044], +[0.288, 0.211, 0.219], +[0.050, 0.019, 0.021], +[0.028, 0.012, 0.010], +[0.015, 0.010, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/21.01.9.41_adaptive.json b/website/benchmark/versions/results/21.01.9.41_adaptive.json new file mode 100644 index 00000000000..541fdc92ffb --- /dev/null +++ b/website/benchmark/versions/results/21.01.9.41_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-04-13 21.1", + "system_full": "ClickHouse 21.1.9.41 2021-04-13(adaptive)", + "version": "21.1.9.41", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.020, 0.004, 0.004], +[0.011, 0.002, 0.002], +[0.015, 0.004, 0.004], +[0.009, 0.002, 0.002], +[0.014, 0.003, 0.003], +[0.010, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.081, 0.027, 0.026], +[0.017, 0.008, 0.010], +[0.016, 0.002, 0.002], +[1.227, 0.335, 0.302], +[0.656, 0.324, 0.325], +[0.033, 0.005, 0.004], +[0.034, 0.016, 0.015], +[0.128, 0.098, 0.159], +[3.429, 0.097, 0.096], +[3.031, 0.251, 0.227], +[3.755, 0.083, 0.081], +[1.369, 0.045, 0.052], +[4.412, 0.230, 0.219], +[4.211, 0.208, 0.177], +[1.788, 0.239, 0.222], +[63.467, 2.373, 2.366], +[42.459, 1.503, 1.478], +[3.071, 0.292, 0.278], +[1.660, 0.156, 0.151], +[0.360, 0.106, 0.097], +[4.666, 0.113, 0.111], +[0.011, 0.001, 0.001], +[0.027, 0.009, 0.010], +[0.277, 0.021, 0.023], +[1.700, 0.032, 0.028], +[1.670, 0.089, 0.092], +[2.554, 0.233, 0.245], +[0.028, 0.015, 0.014], +[0.020, 0.009, 0.009], +[1.918, 0.321, 0.326], +[2.024, 0.354, 0.360], +[1.097, 0.152, 0.143], +[1.621, 0.184, 0.194], +[2.496, 0.462, 0.443], +[3.811, 0.554, 0.557], +[2.351, 0.568, 0.549], +[1.207, 0.551, 0.545], +[4.253, 1.418, 1.384], +[3.671, 0.832, 0.817], +[7.993, 3.087, 2.952], +[0.922, 0.156, 0.059], +[19.390, 0.480, 0.464], +[21.473, 0.577, 0.559], +[40.078, 1.308, 1.275], +[39.505, 0.602, 0.584], +[4.637, 0.207, 0.194], +[2.299, 0.167, 0.165], +[5.295, 0.199, 0.204], +[19.197, 0.476, 0.489], +[15.831, 0.725, 0.711], +[1.103, 1.073, 1.068], +[3.845, 0.483, 0.477], +[9.822, 0.724, 0.689], +[9.006, 3.969, 3.947], +[19.443, 1.984, 1.918], +[19.262, 1.993, 1.926], +[0.823, 0.789, 0.754], +[0.161, 0.110, 0.111], +[0.064, 0.045, 0.048], +[0.070, 0.038, 0.048], +[0.290, 0.224, 0.235], +[0.050, 0.014, 0.013], +[0.040, 0.012, 0.022], +[0.013, 0.009, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/21.02.10.48_adaptive.json b/website/benchmark/versions/results/21.02.10.48_adaptive.json new file mode 100644 index 00000000000..3c710e3e236 --- /dev/null +++ b/website/benchmark/versions/results/21.02.10.48_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-04-17 21.2", + "system_full": "ClickHouse 21.2.10.48 2021-04-17(adaptive)", + "version": "21.2.10.48", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.023, 0.005, 0.005], +[0.012, 0.003, 0.003], +[0.017, 0.005, 0.005], +[0.011, 0.003, 0.003], +[0.015, 0.003, 0.003], +[0.011, 0.004, 0.004], +[0.015, 0.002, 0.002], +[0.060, 0.030, 0.021], +[0.018, 0.009, 0.008], +[0.009, 0.002, 0.002], +[1.213, 0.335, 0.324], +[0.799, 0.347, 0.310], +[0.032, 0.004, 0.004], +[0.031, 0.016, 0.017], +[0.117, 0.098, 0.101], +[3.424, 0.105, 0.096], +[3.014, 0.229, 0.234], +[3.778, 0.078, 0.077], +[1.376, 0.058, 0.057], +[4.413, 0.235, 0.198], +[4.229, 0.191, 0.201], +[1.805, 0.254, 0.228], +[63.464, 2.504, 2.457], +[42.441, 1.549, 1.610], +[3.070, 0.283, 0.279], +[1.675, 0.168, 0.168], +[0.329, 0.085, 0.085], +[4.712, 0.131, 0.104], +[0.010, 0.002, 0.002], +[0.026, 0.009, 0.009], +[0.301, 0.022, 0.022], +[1.726, 0.032, 0.032], +[1.664, 0.093, 0.101], +[2.568, 0.258, 0.247], +[0.031, 0.015, 0.016], +[0.018, 0.010, 0.010], +[1.946, 0.333, 0.317], +[2.054, 0.361, 0.371], +[1.065, 0.147, 0.150], +[1.611, 0.176, 0.164], +[2.519, 0.445, 0.442], +[3.793, 0.586, 0.572], +[2.383, 0.560, 0.555], +[1.202, 0.551, 0.557], +[4.249, 1.419, 1.410], +[3.666, 0.823, 0.793], +[7.991, 3.022, 2.988], +[0.921, 0.078, 0.040], +[19.494, 0.495, 0.472], +[21.495, 0.584, 0.589], +[40.170, 1.361, 1.331], +[39.540, 0.620, 0.591], +[4.635, 0.202, 0.201], +[2.313, 0.175, 0.176], +[5.267, 0.206, 0.191], +[19.183, 0.500, 0.493], +[15.799, 0.739, 0.723], +[1.083, 1.060, 1.060], +[3.830, 0.495, 0.470], +[9.810, 0.730, 0.724], +[9.028, 3.997, 3.907], +[19.429, 2.052, 1.930], +[19.247, 2.000, 1.937], +[0.825, 0.767, 0.785], +[0.170, 0.121, 0.113], +[0.066, 0.046, 0.044], +[0.075, 0.044, 0.038], +[0.296, 0.236, 0.241], +[0.047, 0.036, 0.026], +[0.036, 0.015, 0.021], +[0.017, 0.007, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/21.03.20.1_adaptive.json b/website/benchmark/versions/results/21.03.20.1_adaptive.json new file mode 100644 index 00000000000..38aaf75dc91 --- /dev/null +++ b/website/benchmark/versions/results/21.03.20.1_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-01-26 21.3", + "system_full": "ClickHouse 21.3.20.1 2022-01-26(adaptive)", + "version": "21.3.20.1", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.023, 0.004, 0.004], +[0.012, 0.003, 0.003], +[0.016, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.013, 0.003, 0.003], +[0.012, 0.004, 0.004], +[0.016, 0.002, 0.002], +[0.057, 0.026, 0.022], +[0.018, 0.007, 0.007], +[0.010, 0.002, 0.002], +[1.109, 0.317, 0.335], +[0.661, 0.338, 0.334], +[0.034, 0.004, 0.004], +[0.032, 0.016, 0.015], +[0.155, 0.100, 0.091], +[3.424, 0.111, 0.099], +[3.005, 0.234, 0.225], +[2.609, 0.064, 0.057], +[1.401, 0.056, 0.057], +[4.437, 0.220, 0.213], +[4.219, 0.196, 0.186], +[1.801, 0.231, 0.211], +[63.368, 2.297, 2.282], +[42.444, 1.494, 1.453], +[3.023, 0.297, 0.293], +[1.007, 0.149, 0.139], +[0.149, 0.079, 0.083], +[4.692, 0.122, 0.093], +[0.011, 0.002, 0.001], +[0.026, 0.010, 0.010], +[0.338, 0.022, 0.022], +[1.727, 0.035, 0.031], +[1.684, 0.090, 0.097], +[2.597, 0.218, 0.237], +[0.030, 0.014, 0.014], +[0.021, 0.010, 0.010], +[2.004, 0.330, 0.316], +[2.060, 0.364, 0.357], +[1.073, 0.149, 0.142], +[1.644, 0.166, 0.170], +[2.549, 0.445, 0.437], +[3.802, 0.568, 0.540], +[2.363, 0.558, 0.551], +[1.233, 0.562, 0.556], +[4.255, 1.424, 1.390], +[3.692, 0.809, 0.820], +[8.017, 3.001, 2.985], +[0.925, 0.134, 0.065], +[19.426, 0.507, 0.506], +[21.481, 0.571, 0.556], +[40.204, 1.353, 1.283], +[39.541, 0.624, 0.642], +[4.633, 0.192, 0.193], +[2.313, 0.174, 0.170], +[5.278, 0.196, 0.197], +[19.189, 0.503, 0.490], +[15.698, 0.724, 0.701], +[1.088, 1.064, 1.059], +[3.822, 0.467, 0.427], +[9.807, 0.712, 0.675], +[9.042, 3.844, 3.828], +[19.440, 2.069, 1.911], +[19.342, 1.989, 1.950], +[0.836, 0.754, 0.775], +[0.152, 0.111, 0.122], +[0.069, 0.046, 0.042], +[0.070, 0.040, 0.051], +[0.300, 0.228, 0.225], +[0.053, 0.015, 0.034], +[0.030, 0.011, 0.013], +[0.015, 0.003, 0.012] + ] + } +] diff --git a/website/benchmark/versions/results/21.04.7.3_adaptive.json b/website/benchmark/versions/results/21.04.7.3_adaptive.json new file mode 100644 index 00000000000..bb5edb4e447 --- /dev/null +++ b/website/benchmark/versions/results/21.04.7.3_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-05-20 21.4", + "system_full": "ClickHouse 21.4.7.3 2021-05-20(adaptive)", + "version": "21.4.7.3", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.021, 0.004, 0.004], +[0.013, 0.003, 0.003], +[0.016, 0.005, 0.005], +[0.010, 0.003, 0.003], +[0.014, 0.003, 0.003], +[0.010, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.060, 0.027, 0.026], +[0.015, 0.012, 0.009], +[0.009, 0.002, 0.002], +[1.193, 0.327, 0.326], +[0.633, 0.344, 0.327], +[0.032, 0.005, 0.004], +[0.032, 0.015, 0.015], +[0.121, 0.105, 0.103], +[3.414, 0.098, 0.102], +[11.208, 0.217, 0.209], +[0.237, 0.072, 0.074], +[1.411, 0.051, 0.126], +[4.444, 0.218, 0.203], +[4.219, 0.175, 0.175], +[1.788, 0.233, 0.236], +[22.316, 1.279, 1.214], +[25.837, 1.361, 1.323], +[12.063, 1.031, 1.059], +[0.299, 0.169, 0.160], +[0.114, 0.078, 0.083], +[0.945, 0.061, 0.059], +[0.011, 0.002, 0.002], +[0.030, 0.010, 0.008], +[0.058, 0.022, 0.021], +[1.447, 0.034, 0.030], +[1.518, 0.107, 0.095], +[2.377, 0.218, 0.217], +[0.028, 0.013, 0.013], +[0.019, 0.010, 0.010], +[1.541, 0.324, 0.330], +[1.872, 0.376, 0.379], +[1.032, 0.130, 0.139], +[1.472, 0.157, 0.150], +[1.846, 0.445, 0.432], +[3.153, 0.560, 0.556], +[1.698, 0.529, 0.543], +[1.213, 0.569, 0.543], +[3.571, 1.416, 1.398], +[2.978, 0.853, 0.810], +[7.312, 3.041, 2.953], +[0.916, 0.101, 0.058], +[19.296, 0.431, 0.426], +[2.597, 0.519, 0.501], +[21.286, 1.229, 1.181], +[18.186, 0.611, 0.571], +[3.249, 0.207, 0.173], +[1.183, 0.161, 0.163], +[3.153, 0.194, 0.185], +[0.764, 0.432, 0.429], +[15.719, 0.706, 0.686], +[1.082, 1.065, 1.064], +[2.955, 0.455, 0.444], +[8.856, 0.658, 0.626], +[5.234, 3.545, 3.447], +[1.934, 1.909, 1.853], +[1.853, 1.938, 1.840], +[0.802, 0.772, 0.765], +[0.130, 0.109, 0.108], +[0.062, 0.051, 0.053], +[0.054, 0.042, 0.043], +[0.239, 0.242, 0.214], +[0.039, 0.012, 0.011], +[0.031, 0.010, 0.018], +[0.020, 0.014, 0.004] + ] + } +] diff --git a/website/benchmark/versions/results/21.05.9.4_adaptive.json b/website/benchmark/versions/results/21.05.9.4_adaptive.json new file mode 100644 index 00000000000..05f7bfa2a0b --- /dev/null +++ b/website/benchmark/versions/results/21.05.9.4_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-07-10 21.5", + "system_full": "ClickHouse 21.5.9.4 2021-07-10(adaptive)", + "version": "21.5.9.4", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.022, 0.004, 0.004], +[0.014, 0.002, 0.003], +[0.016, 0.004, 0.004], +[0.009, 0.002, 0.003], +[0.013, 0.003, 0.003], +[0.011, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.072, 0.026, 0.026], +[0.016, 0.007, 0.007], +[0.008, 0.002, 0.002], +[1.156, 0.347, 0.318], +[0.608, 0.336, 0.324], +[0.033, 0.004, 0.004], +[0.029, 0.015, 0.014], +[0.112, 0.102, 0.095], +[3.432, 0.115, 0.114], +[11.038, 0.224, 0.225], +[2.938, 0.086, 0.084], +[1.367, 0.044, 0.047], +[4.466, 0.223, 0.214], +[4.230, 0.195, 0.181], +[1.826, 0.228, 0.223], +[22.627, 1.270, 1.250], +[26.679, 1.423, 1.379], +[28.615, 1.147, 1.084], +[0.607, 0.139, 0.155], +[0.112, 0.079, 0.080], +[1.520, 0.063, 0.056], +[0.010, 0.002, 0.002], +[0.026, 0.010, 0.009], +[0.426, 0.023, 0.020], +[1.734, 0.036, 0.031], +[1.696, 0.093, 0.084], +[2.596, 0.236, 0.224], +[0.030, 0.015, 0.015], +[0.020, 0.011, 0.010], +[1.970, 0.314, 0.313], +[2.073, 0.367, 0.358], +[1.068, 0.163, 0.136], +[1.622, 0.174, 0.169], +[2.539, 0.467, 0.469], +[3.845, 0.562, 0.563], +[2.331, 0.540, 0.539], +[1.217, 0.570, 0.551], +[4.257, 1.407, 1.449], +[3.663, 0.813, 0.832], +[8.009, 2.986, 3.047], +[0.922, 0.141, 0.066], +[19.407, 0.458, 0.454], +[21.493, 0.551, 0.540], +[40.210, 1.275, 1.246], +[39.569, 0.604, 0.596], +[4.654, 0.213, 0.170], +[2.342, 0.157, 0.167], +[5.301, 0.190, 0.200], +[19.220, 0.498, 0.482], +[15.816, 0.692, 0.692], +[1.096, 1.063, 1.059], +[3.858, 0.461, 0.431], +[9.850, 0.673, 0.637], +[9.063, 3.608, 3.629], +[19.417, 1.977, 1.908], +[19.431, 2.004, 1.907], +[0.838, 0.800, 0.777], +[0.175, 0.110, 0.108], +[0.076, 0.046, 0.054], +[0.080, 0.042, 0.035], +[0.327, 0.250, 0.235], +[0.037, 0.017, 0.011], +[0.031, 0.021, 0.009], +[0.014, 0.015, 0.017] + ] + } +] diff --git a/website/benchmark/versions/results/21.06.9.7_adaptive.json b/website/benchmark/versions/results/21.06.9.7_adaptive.json new file mode 100644 index 00000000000..5ddb79105e4 --- /dev/null +++ b/website/benchmark/versions/results/21.06.9.7_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-09-03 21.6", + "system_full": "ClickHouse 21.6.9.7 2021-09-03(adaptive)", + "version": "21.6.9.7", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.021, 0.003, 0.003], +[0.012, 0.002, 0.002], +[0.018, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.014, 0.003, 0.002], +[0.014, 0.003, 0.003], +[0.014, 0.002, 0.002], +[0.066, 0.028, 0.027], +[0.059, 0.009, 0.007], +[0.010, 0.002, 0.002], +[1.226, 0.350, 0.349], +[0.573, 0.341, 0.322], +[0.032, 0.005, 0.006], +[0.033, 0.015, 0.015], +[0.112, 0.095, 0.099], +[3.431, 0.112, 0.143], +[3.106, 0.231, 0.226], +[2.623, 0.072, 0.060], +[1.397, 0.058, 0.056], +[4.432, 0.237, 0.225], +[4.207, 0.208, 0.196], +[1.801, 0.233, 0.250], +[63.562, 2.649, 2.593], +[42.495, 1.661, 1.672], +[3.054, 0.292, 0.266], +[1.139, 0.176, 0.177], +[0.130, 0.070, 0.088], +[4.663, 0.108, 0.126], +[0.010, 0.001, 0.002], +[0.026, 0.009, 0.009], +[0.323, 0.023, 0.022], +[1.709, 0.034, 0.033], +[1.703, 0.091, 0.092], +[2.612, 0.234, 0.237], +[0.027, 0.014, 0.014], +[0.029, 0.009, 0.009], +[1.967, 0.330, 0.323], +[2.053, 0.371, 0.358], +[1.059, 0.140, 0.130], +[1.663, 0.165, 0.149], +[2.534, 0.450, 0.448], +[3.803, 0.575, 0.549], +[2.410, 0.557, 0.532], +[1.230, 0.557, 0.563], +[4.250, 1.433, 1.387], +[3.695, 0.832, 0.823], +[8.005, 3.048, 2.991], +[0.925, 0.108, 0.052], +[19.516, 0.509, 0.473], +[21.482, 0.587, 0.571], +[40.055, 1.356, 1.337], +[39.576, 0.644, 0.607], +[4.633, 0.201, 0.207], +[2.300, 0.173, 0.178], +[5.286, 0.205, 0.201], +[19.282, 0.497, 0.506], +[15.767, 0.751, 0.724], +[1.095, 1.065, 1.065], +[3.838, 0.448, 0.448], +[9.826, 0.674, 0.645], +[9.114, 3.490, 3.507], +[19.456, 1.993, 1.956], +[19.353, 2.015, 1.956], +[0.824, 0.785, 0.779], +[0.166, 0.116, 0.106], +[0.091, 0.052, 0.066], +[0.076, 0.039, 0.048], +[0.305, 0.240, 0.251], +[0.046, 0.027, 0.027], +[0.056, 0.023, 0.011], +[0.017, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/21.07.11.3_adaptive.json b/website/benchmark/versions/results/21.07.11.3_adaptive.json new file mode 100644 index 00000000000..76b181ad253 --- /dev/null +++ b/website/benchmark/versions/results/21.07.11.3_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-09-24 21.7", + "system_full": "ClickHouse 21.7.11.3 2021-09-24(adaptive)", + "version": "2021-09-24", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.017, 0.004, 0.003], +[0.011, 0.002, 0.002], +[0.016, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.014, 0.002, 0.002], +[0.013, 0.003, 0.003], +[0.016, 0.003, 0.002], +[0.056, 0.025, 0.029], +[0.013, 0.007, 0.012], +[0.010, 0.002, 0.002], +[1.197, 0.338, 0.385], +[0.744, 0.328, 0.309], +[0.030, 0.005, 0.004], +[0.030, 0.017, 0.015], +[0.117, 0.092, 0.104], +[3.411, 0.106, 0.101], +[2.986, 0.233, 0.229], +[2.630, 0.065, 0.066], +[1.400, 0.057, 0.054], +[4.443, 0.230, 0.226], +[4.219, 0.199, 0.205], +[1.785, 0.229, 0.238], +[63.427, 2.479, 2.411], +[42.498, 1.582, 1.532], +[3.005, 0.297, 0.279], +[1.045, 0.157, 0.189], +[0.118, 0.082, 0.089], +[4.653, 0.136, 0.098], +[0.010, 0.001, 0.001], +[0.025, 0.012, 0.009], +[0.319, 0.023, 0.023], +[1.723, 0.032, 0.032], +[1.682, 0.092, 0.091], +[2.607, 0.242, 0.225], +[0.027, 0.014, 0.014], +[0.019, 0.010, 0.009], +[1.980, 0.323, 0.328], +[2.042, 0.363, 0.371], +[1.064, 0.135, 0.143], +[1.659, 0.172, 0.172], +[2.533, 0.484, 0.440], +[3.839, 0.598, 0.547], +[2.378, 0.583, 0.562], +[1.223, 0.562, 0.556], +[4.259, 1.447, 1.399], +[3.713, 0.862, 0.825], +[8.055, 3.074, 3.056], +[0.921, 0.124, 0.044], +[19.450, 0.493, 0.476], +[21.501, 0.585, 0.568], +[40.294, 1.348, 1.326], +[39.583, 0.622, 0.615], +[4.634, 0.220, 0.196], +[2.309, 0.173, 0.176], +[5.275, 0.214, 0.206], +[19.186, 0.524, 0.479], +[15.773, 0.744, 0.727], +[1.093, 1.071, 1.070], +[3.858, 0.469, 0.437], +[9.827, 0.661, 0.650], +[9.028, 3.542, 3.558], +[19.424, 2.010, 1.953], +[19.434, 2.045, 1.965], +[0.814, 0.792, 0.775], +[0.158, 0.113, 0.119], +[0.069, 0.043, 0.047], +[0.073, 0.045, 0.056], +[0.301, 0.243, 0.231], +[0.047, 0.013, 0.018], +[0.037, 0.015, 0.011], +[0.013, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/21.08.15.7_adaptive.json b/website/benchmark/versions/results/21.08.15.7_adaptive.json new file mode 100644 index 00000000000..8607d494cb4 --- /dev/null +++ b/website/benchmark/versions/results/21.08.15.7_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-02-25 21.8", + "system_full": "ClickHouse 21.8.15.7 2022-02-25(adaptive)", + "version": "21.8.15.7", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.019, 0.004, 0.003], +[0.010, 0.003, 0.002], +[0.017, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.014, 0.003, 0.002], +[0.012, 0.003, 0.003], +[0.016, 0.002, 0.003], +[0.060, 0.029, 0.025], +[0.014, 0.007, 0.007], +[0.009, 0.002, 0.002], +[1.137, 0.343, 0.351], +[0.752, 0.350, 0.329], +[0.030, 0.006, 0.004], +[0.031, 0.016, 0.015], +[0.113, 0.098, 0.095], +[3.412, 0.112, 0.102], +[2.999, 0.230, 0.224], +[2.620, 0.068, 0.073], +[1.394, 0.055, 0.056], +[4.439, 0.232, 0.242], +[4.215, 0.218, 0.187], +[1.819, 0.230, 0.226], +[63.446, 2.475, 2.432], +[42.542, 1.559, 1.545], +[2.991, 0.297, 0.279], +[1.020, 0.154, 0.144], +[0.124, 0.078, 0.083], +[4.689, 0.123, 0.118], +[0.012, 0.001, 0.001], +[0.027, 0.011, 0.009], +[0.286, 0.025, 0.022], +[1.709, 0.034, 0.032], +[1.692, 0.092, 0.095], +[2.611, 0.230, 0.226], +[0.026, 0.013, 0.014], +[0.020, 0.010, 0.010], +[1.973, 0.339, 0.326], +[2.014, 0.380, 0.372], +[1.034, 0.139, 0.145], +[1.661, 0.162, 0.164], +[2.537, 0.475, 0.450], +[3.815, 0.588, 0.544], +[2.368, 0.559, 0.550], +[1.201, 0.575, 0.563], +[4.254, 1.424, 1.416], +[3.701, 0.841, 0.837], +[8.060, 3.123, 3.095], +[0.922, 0.176, 0.068], +[19.380, 0.486, 0.476], +[21.535, 0.622, 0.582], +[40.306, 1.364, 1.329], +[39.580, 0.622, 0.605], +[4.635, 0.199, 0.194], +[2.337, 0.174, 0.165], +[5.270, 0.194, 0.196], +[19.226, 0.522, 0.508], +[15.791, 0.734, 0.727], +[1.093, 1.067, 1.070], +[3.841, 0.479, 0.452], +[9.837, 0.660, 0.670], +[9.048, 3.597, 3.576], +[19.469, 1.964, 1.962], +[19.474, 1.983, 1.981], +[0.839, 0.781, 0.772], +[0.149, 0.112, 0.110], +[0.072, 0.059, 0.053], +[0.086, 0.047, 0.041], +[0.308, 0.227, 0.234], +[0.047, 0.023, 0.013], +[0.029, 0.012, 0.014], +[0.026, 0.004, 0.005] + ] + } +] diff --git a/website/benchmark/versions/results/21.09.6.24_adaptive.json b/website/benchmark/versions/results/21.09.6.24_adaptive.json new file mode 100644 index 00000000000..447f7bef853 --- /dev/null +++ b/website/benchmark/versions/results/21.09.6.24_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2021-12-02 21.9", + "system_full": "ClickHouse 21.9.6.24 2021-12-02(adaptive)", + "version": "2021-12-02", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.022, 0.004, 0.003], +[0.014, 0.002, 0.002], +[0.018, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.015, 0.003, 0.002], +[0.012, 0.003, 0.003], +[0.017, 0.002, 0.002], +[0.071, 0.035, 0.032], +[0.017, 0.007, 0.007], +[0.045, 0.002, 0.002], +[0.920, 0.346, 0.332], +[0.573, 0.286, 0.285], +[0.032, 0.006, 0.005], +[0.028, 0.014, 0.018], +[0.116, 0.099, 0.101], +[3.420, 0.110, 0.099], +[2.968, 0.295, 0.235], +[2.711, 0.067, 0.064], +[1.415, 0.065, 0.060], +[4.427, 0.222, 0.241], +[4.183, 0.180, 0.184], +[1.854, 0.233, 0.244], +[63.467, 2.360, 2.306], +[42.527, 1.533, 1.488], +[3.013, 0.286, 0.275], +[1.009, 0.163, 0.178], +[0.150, 0.078, 0.084], +[4.553, 0.123, 0.097], +[0.011, 0.001, 0.001], +[0.026, 0.010, 0.008], +[0.249, 0.023, 0.022], +[1.679, 0.031, 0.032], +[1.660, 0.096, 0.097], +[2.579, 0.228, 0.215], +[0.031, 0.012, 0.013], +[0.022, 0.009, 0.009], +[1.960, 0.321, 0.322], +[2.028, 0.348, 0.357], +[1.069, 0.126, 0.143], +[1.648, 0.157, 0.156], +[2.569, 0.454, 0.427], +[3.799, 0.571, 0.563], +[2.363, 0.552, 0.529], +[1.174, 0.511, 0.501], +[4.261, 1.418, 1.387], +[3.693, 0.856, 0.806], +[7.945, 2.932, 2.995], +[0.919, 0.120, 0.082], +[19.420, 0.502, 0.473], +[21.480, 0.609, 0.590], +[40.279, 1.340, 1.318], +[39.461, 0.624, 0.604], +[4.635, 0.206, 0.202], +[2.313, 0.164, 0.166], +[5.325, 0.201, 0.191], +[19.250, 0.491, 0.481], +[15.833, 0.713, 0.710], +[1.100, 1.070, 1.067], +[3.816, 0.426, 0.424], +[9.758, 0.568, 0.546], +[8.369, 2.686, 2.746], +[19.439, 1.971, 1.923], +[19.436, 1.985, 1.931], +[0.781, 0.723, 0.728], +[0.150, 0.143, 0.108], +[0.078, 0.054, 0.048], +[0.071, 0.050, 0.075], +[0.313, 0.224, 0.214], +[0.044, 0.016, 0.025], +[0.032, 0.014, 0.020], +[0.022, 0.015, 0.010] + ] + } +] diff --git a/website/benchmark/versions/results/21.10.6.2_adaptive.json b/website/benchmark/versions/results/21.10.6.2_adaptive.json new file mode 100644 index 00000000000..bc77e0e8889 --- /dev/null +++ b/website/benchmark/versions/results/21.10.6.2_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-01-24 21.10", + "system_full": "ClickHouse 21.10.6.2 2022-01-24(adaptive)", + "version": "21.10.6.2", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.021, 0.004, 0.004], +[0.012, 0.002, 0.002], +[0.016, 0.004, 0.004], +[0.010, 0.002, 0.002], +[0.013, 0.003, 0.002], +[0.011, 0.003, 0.003], +[0.015, 0.002, 0.002], +[0.062, 0.032, 0.035], +[0.019, 0.007, 0.007], +[0.045, 0.002, 0.002], +[1.173, 0.329, 0.302], +[0.622, 0.276, 0.278], +[0.033, 0.004, 0.006], +[0.035, 0.015, 0.014], +[0.125, 0.094, 0.099], +[3.431, 0.106, 0.098], +[2.989, 0.241, 0.226], +[2.769, 0.062, 0.059], +[1.413, 0.051, 0.048], +[4.433, 0.245, 0.211], +[4.198, 0.183, 0.172], +[1.837, 0.223, 0.232], +[63.368, 2.339, 2.324], +[42.447, 1.474, 1.469], +[3.007, 0.296, 0.274], +[1.002, 0.167, 0.176], +[0.143, 0.093, 0.082], +[4.599, 0.118, 0.098], +[0.010, 0.001, 0.001], +[0.026, 0.009, 0.007], +[0.321, 0.023, 0.023], +[1.734, 0.031, 0.030], +[1.696, 0.090, 0.100], +[2.586, 0.217, 0.222], +[0.026, 0.013, 0.013], +[0.020, 0.008, 0.009], +[2.026, 0.322, 0.312], +[2.059, 0.355, 0.348], +[1.109, 0.130, 0.137], +[1.656, 0.152, 0.148], +[2.565, 0.420, 0.410], +[3.843, 0.565, 0.536], +[2.362, 0.510, 0.521], +[1.177, 0.494, 0.480], +[4.206, 1.336, 1.315], +[3.694, 0.828, 0.802], +[7.829, 2.826, 2.838], +[0.933, 0.101, 0.050], +[19.482, 0.505, 0.470], +[21.289, 0.582, 0.565], +[39.997, 1.372, 1.307], +[39.358, 0.656, 0.609], +[4.636, 0.200, 0.177], +[2.328, 0.155, 0.167], +[5.278, 0.198, 0.177], +[19.227, 0.519, 0.507], +[15.788, 0.749, 0.717], +[1.093, 1.063, 1.060], +[3.848, 0.412, 0.418], +[9.738, 0.557, 0.528], +[8.344, 2.681, 2.646], +[19.421, 1.944, 1.914], +[19.217, 1.937, 1.929], +[0.798, 0.711, 0.704], +[0.165, 0.111, 0.109], +[0.081, 0.045, 0.040], +[0.071, 0.060, 0.052], +[0.294, 0.245, 0.223], +[0.053, 0.025, 0.016], +[0.030, 0.018, 0.023], +[0.017, 0.005, 0.012] + ] + } +] diff --git a/website/benchmark/versions/results/21.11.11.1_adaptive.json b/website/benchmark/versions/results/21.11.11.1_adaptive.json new file mode 100644 index 00000000000..9228e607e77 --- /dev/null +++ b/website/benchmark/versions/results/21.11.11.1_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-01-23 21.11", + "system_full": "ClickHouse 21.11.11.1 2022-01-23(adaptive)", + "version": "21.11.11.1", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.031, 0.004, 0.003], +[0.017, 0.002, 0.002], +[0.024, 0.004, 0.004], +[0.015, 0.002, 0.002], +[0.019, 0.002, 0.002], +[0.019, 0.003, 0.003], +[0.020, 0.002, 0.002], +[0.060, 0.023, 0.022], +[0.023, 0.007, 0.007], +[0.052, 0.002, 0.002], +[1.080, 0.333, 0.317], +[0.795, 0.262, 0.279], +[0.038, 0.004, 0.004], +[0.037, 0.014, 0.016], +[0.116, 0.099, 0.094], +[3.492, 0.085, 0.074], +[2.783, 0.223, 0.218], +[2.676, 0.055, 0.051], +[1.449, 0.054, 0.052], +[4.442, 0.202, 0.196], +[4.269, 0.180, 0.170], +[1.907, 0.225, 0.213], +[63.380, 2.320, 2.283], +[42.459, 1.482, 1.417], +[2.926, 0.288, 0.279], +[1.156, 0.140, 0.141], +[0.135, 0.080, 0.069], +[3.156, 0.090, 0.092], +[0.018, 0.002, 0.001], +[0.031, 0.008, 0.007], +[0.396, 0.020, 0.021], +[1.745, 0.030, 0.030], +[1.695, 0.086, 0.084], +[2.610, 0.220, 0.204], +[0.031, 0.011, 0.011], +[0.027, 0.008, 0.008], +[2.071, 0.313, 0.306], +[2.075, 0.333, 0.334], +[1.133, 0.118, 0.113], +[1.693, 0.136, 0.131], +[2.607, 0.391, 0.372], +[3.839, 0.520, 0.506], +[2.428, 0.521, 0.494], +[1.176, 0.473, 0.463], +[4.198, 1.345, 1.298], +[3.697, 0.809, 0.768], +[7.942, 2.859, 2.861], +[0.935, 0.064, 0.031], +[19.534, 0.490, 0.468], +[21.514, 0.556, 0.536], +[40.022, 1.329, 1.267], +[33.001, 0.673, 0.575], +[4.655, 0.167, 0.153], +[2.383, 0.135, 0.128], +[5.388, 0.163, 0.155], +[19.305, 0.509, 0.484], +[15.801, 0.738, 0.714], +[1.080, 1.059, 1.061], +[3.817, 0.371, 0.364], +[9.735, 0.504, 0.484], +[8.324, 2.674, 2.645], +[19.419, 1.884, 1.860], +[19.383, 1.888, 1.861], +[0.785, 0.691, 0.691], +[0.141, 0.108, 0.106], +[0.078, 0.040, 0.050], +[0.085, 0.048, 0.039], +[0.292, 0.232, 0.240], +[0.052, 0.025, 0.022], +[0.036, 0.020, 0.013], +[0.022, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/21.12.4.1_adaptive.json b/website/benchmark/versions/results/21.12.4.1_adaptive.json new file mode 100644 index 00000000000..d5d4ff0db8d --- /dev/null +++ b/website/benchmark/versions/results/21.12.4.1_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-01-23 21.12", + "system_full": "ClickHouse 21.12.4.1 2022-01-23(adaptive)", + "version": "21.12.4.1", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.036, 0.003, 0.003], +[0.020, 0.002, 0.002], +[0.025, 0.004, 0.004], +[0.019, 0.002, 0.002], +[0.020, 0.002, 0.002], +[0.020, 0.003, 0.003], +[0.022, 0.002, 0.002], +[0.067, 0.023, 0.027], +[0.024, 0.007, 0.007], +[0.060, 0.002, 0.002], +[1.121, 0.334, 0.318], +[0.607, 0.259, 0.260], +[0.038, 0.004, 0.004], +[0.039, 0.018, 0.014], +[0.120, 0.107, 0.096], +[3.493, 0.078, 0.075], +[2.812, 0.229, 0.226], +[2.654, 0.058, 0.052], +[1.437, 0.051, 0.051], +[4.449, 0.213, 0.200], +[4.273, 0.176, 0.166], +[1.864, 0.222, 0.213], +[63.700, 2.301, 2.246], +[42.592, 1.463, 1.410], +[2.933, 0.307, 0.283], +[1.117, 0.163, 0.134], +[0.135, 0.083, 0.077], +[3.151, 0.083, 0.080], +[0.019, 0.001, 0.001], +[0.033, 0.008, 0.008], +[0.407, 0.022, 0.021], +[1.755, 0.035, 0.030], +[1.719, 0.084, 0.087], +[2.633, 0.224, 0.212], +[0.032, 0.011, 0.010], +[0.028, 0.008, 0.008], +[2.071, 0.311, 0.300], +[2.107, 0.331, 0.327], +[1.158, 0.115, 0.112], +[1.707, 0.133, 0.128], +[2.621, 0.378, 0.369], +[3.826, 0.522, 0.488], +[2.445, 0.522, 0.521], +[1.181, 0.466, 0.461], +[4.246, 1.319, 1.293], +[3.692, 0.805, 0.761], +[7.928, 2.828, 2.907], +[0.920, 0.071, 0.033], +[19.573, 0.501, 0.472], +[21.520, 0.565, 0.550], +[40.245, 1.305, 1.263], +[33.018, 0.620, 0.574], +[4.656, 0.157, 0.151], +[2.426, 0.149, 0.134], +[5.345, 0.169, 0.154], +[19.347, 0.520, 0.493], +[15.793, 0.753, 0.728], +[1.082, 1.058, 1.057], +[3.851, 0.355, 0.359], +[9.758, 0.508, 0.485], +[8.379, 2.657, 2.712], +[19.461, 1.884, 1.886], +[19.397, 1.896, 1.854], +[0.785, 0.696, 0.689], +[0.137, 0.116, 0.117], +[0.076, 0.040, 0.060], +[0.080, 0.043, 0.043], +[0.320, 0.253, 0.235], +[0.064, 0.014, 0.022], +[0.048, 0.014, 0.030], +[0.020, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/22.1.4.30_adaptive.json b/website/benchmark/versions/results/22.1.4.30_adaptive.json new file mode 100644 index 00000000000..0cc2ea8f48a --- /dev/null +++ b/website/benchmark/versions/results/22.1.4.30_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-02-25 22.1", + "system_full": "ClickHouse 22.1.4.30 2022-02-25(adaptive)", + "version": "22.1.4.30", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.026, 0.003, 0.003], +[0.021, 0.002, 0.002], +[0.024, 0.004, 0.004], +[0.019, 0.002, 0.002], +[0.020, 0.002, 0.002], +[0.020, 0.003, 0.003], +[0.021, 0.002, 0.002], +[0.076, 0.026, 0.023], +[0.026, 0.007, 0.008], +[0.057, 0.002, 0.002], +[1.235, 0.335, 0.323], +[0.605, 0.295, 0.259], +[0.037, 0.006, 0.006], +[0.039, 0.016, 0.016], +[0.134, 0.102, 0.105], +[3.485, 0.082, 0.074], +[2.802, 0.224, 0.232], +[2.682, 0.062, 0.056], +[1.436, 0.051, 0.051], +[4.452, 0.225, 0.190], +[4.245, 0.189, 0.172], +[1.951, 0.225, 0.212], +[63.456, 2.290, 2.253], +[42.478, 1.453, 1.417], +[2.930, 0.301, 0.280], +[1.145, 0.144, 0.137], +[0.136, 0.079, 0.078], +[3.136, 0.105, 0.081], +[0.020, 0.001, 0.001], +[0.035, 0.008, 0.008], +[0.391, 0.021, 0.021], +[1.751, 0.032, 0.031], +[1.697, 0.083, 0.085], +[2.610, 0.208, 0.212], +[0.033, 0.012, 0.012], +[0.026, 0.008, 0.008], +[1.972, 0.319, 0.304], +[2.083, 0.335, 0.328], +[1.087, 0.127, 0.113], +[1.670, 0.128, 0.123], +[2.615, 0.381, 0.364], +[3.817, 0.521, 0.486], +[2.390, 0.511, 0.502], +[1.148, 0.472, 0.463], +[4.492, 1.327, 1.318], +[3.722, 0.828, 0.818], +[8.008, 2.936, 2.951], +[0.941, 0.058, 0.031], +[19.524, 0.518, 0.500], +[21.530, 0.565, 0.549], +[40.236, 1.326, 1.276], +[32.973, 0.629, 0.603], +[4.656, 0.161, 0.158], +[2.387, 0.138, 0.127], +[5.362, 0.164, 0.154], +[19.264, 0.502, 0.487], +[15.836, 0.733, 0.726], +[1.083, 1.061, 1.059], +[3.832, 0.359, 0.350], +[9.755, 0.511, 0.485], +[8.436, 2.740, 2.725], +[19.561, 2.010, 2.005], +[19.328, 2.023, 2.008], +[0.774, 0.702, 0.714], +[0.152, 0.113, 0.111], +[0.069, 0.050, 0.056], +[0.086, 0.052, 0.041], +[0.328, 0.247, 0.264], +[0.056, 0.015, 0.016], +[0.040, 0.016, 0.017], +[0.019, 0.003, 0.003] + ] + } +] diff --git a/website/benchmark/versions/results/22.2.3.5_adaptive.json b/website/benchmark/versions/results/22.2.3.5_adaptive.json new file mode 100644 index 00000000000..d7c61f7fe03 --- /dev/null +++ b/website/benchmark/versions/results/22.2.3.5_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-02-25 22.2", + "system_full": "ClickHouse 22.2.3.5 2022-02-25(adaptive)", + "version": "22.2.3.5", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.028, 0.004, 0.004], +[0.021, 0.003, 0.003], +[0.025, 0.004, 0.005], +[0.018, 0.003, 0.003], +[0.020, 0.003, 0.003], +[0.021, 0.004, 0.004], +[0.024, 0.002, 0.002], +[0.086, 0.036, 0.032], +[0.024, 0.011, 0.009], +[0.055, 0.002, 0.002], +[1.146, 0.330, 0.323], +[0.605, 0.282, 0.259], +[0.040, 0.007, 0.006], +[0.040, 0.014, 0.014], +[0.131, 0.112, 0.091], +[3.459, 0.083, 0.078], +[2.788, 0.227, 0.220], +[2.657, 0.063, 0.053], +[1.441, 0.050, 0.058], +[4.444, 0.207, 0.197], +[4.273, 0.178, 0.177], +[1.864, 0.217, 0.218], +[63.697, 2.299, 2.228], +[42.575, 1.462, 1.420], +[2.934, 0.314, 0.286], +[1.139, 0.151, 0.156], +[0.136, 0.074, 0.082], +[3.131, 0.088, 0.083], +[0.022, 0.002, 0.001], +[0.035, 0.009, 0.009], +[0.371, 0.023, 0.021], +[1.697, 0.034, 0.034], +[1.706, 0.087, 0.082], +[2.618, 0.209, 0.195], +[0.019, 0.002, 0.001], +[0.027, 0.009, 0.008], +[2.084, 0.314, 0.303], +[2.126, 0.344, 0.329], +[1.116, 0.117, 0.115], +[1.682, 0.137, 0.131], +[2.646, 0.386, 0.371], +[3.832, 0.514, 0.487], +[2.412, 0.505, 0.492], +[1.179, 0.457, 0.463], +[4.264, 1.362, 1.369], +[3.729, 0.856, 0.796], +[7.967, 3.005, 2.935], +[0.937, 0.059, 0.033], +[19.577, 0.544, 0.523], +[21.519, 0.593, 0.556], +[40.278, 1.335, 1.281], +[33.029, 0.672, 0.644], +[4.674, 0.161, 0.158], +[2.398, 0.136, 0.130], +[5.391, 0.175, 0.157], +[19.313, 0.515, 0.496], +[15.716, 0.736, 0.713], +[1.089, 1.067, 1.066], +[3.787, 0.366, 0.346], +[9.775, 0.510, 0.497], +[8.451, 2.676, 2.772], +[19.577, 2.037, 2.001], +[19.571, 2.024, 2.008], +[0.814, 0.718, 0.712], +[0.158, 0.107, 0.112], +[0.082, 0.041, 0.045], +[0.096, 0.068, 0.045], +[0.305, 0.250, 0.250], +[0.055, 0.027, 0.015], +[0.037, 0.013, 0.012], +[0.025, 0.004, 0.005] + ] + } +] diff --git a/website/benchmark/versions/results/22.3.3.44_adaptive.json b/website/benchmark/versions/results/22.3.3.44_adaptive.json new file mode 100644 index 00000000000..bc236a438d6 --- /dev/null +++ b/website/benchmark/versions/results/22.3.3.44_adaptive.json @@ -0,0 +1,85 @@ + +[ + { + "system": "2022-04-06 22.3", + "system_full": "ClickHouse 22.3.3.44 2022-04-06(adaptive)", + "version": "22.3.3.44", + "kind": "adaptive", + "comments": "", + "result": + [ + +[0.031, 0.004, 0.004], +[0.021, 0.003, 0.003], +[0.026, 0.005, 0.005], +[0.019, 0.002, 0.003], +[0.021, 0.003, 0.003], +[0.024, 0.004, 0.004], +[0.023, 0.002, 0.002], +[0.130, 0.032, 0.029], +[0.027, 0.008, 0.008], +[0.057, 0.002, 0.002], +[1.145, 0.328, 0.313], +[0.635, 0.267, 0.297], +[0.039, 0.006, 0.009], +[0.038, 0.017, 0.015], +[0.129, 0.106, 0.104], +[3.447, 0.081, 0.070], +[2.784, 0.233, 0.223], +[2.670, 0.057, 0.056], +[1.449, 0.054, 0.052], +[4.461, 0.217, 0.196], +[4.270, 0.179, 0.171], +[1.845, 0.220, 0.205], +[63.657, 2.305, 2.263], +[42.601, 1.471, 1.432], +[2.952, 0.294, 0.280], +[1.143, 0.142, 0.146], +[0.134, 0.084, 0.080], +[3.128, 0.085, 0.082], +[0.021, 0.002, 0.002], +[0.034, 0.008, 0.008], +[0.365, 0.022, 0.021], +[1.751, 0.033, 0.032], +[1.691, 0.088, 0.088], +[2.620, 0.216, 0.203], +[0.018, 0.002, 0.001], +[0.026, 0.008, 0.008], +[2.045, 0.318, 0.309], +[2.074, 0.343, 0.336], +[1.119, 0.120, 0.119], +[1.674, 0.141, 0.138], +[2.637, 0.389, 0.374], +[3.842, 0.510, 0.488], +[2.378, 0.516, 0.495], +[1.185, 0.468, 0.457], +[4.261, 1.369, 1.354], +[3.745, 0.826, 0.791], +[8.026, 2.927, 2.951], +[0.908, 0.060, 0.039], +[19.572, 0.550, 0.522], +[21.498, 0.584, 0.549], +[40.296, 1.307, 1.275], +[33.134, 0.662, 0.630], +[4.665, 0.163, 0.154], +[2.389, 0.143, 0.142], +[5.373, 0.167, 0.165], +[19.294, 0.513, 0.480], +[15.836, 0.749, 0.722], +[1.083, 1.064, 1.066], +[3.812, 0.370, 0.360], +[9.763, 0.513, 0.487], +[8.477, 2.733, 2.725], +[19.572, 2.006, 2.019], +[19.497, 2.026, 2.032], +[0.784, 0.708, 0.716], +[0.148, 0.109, 0.126], +[0.073, 0.047, 0.041], +[0.077, 0.042, 0.054], +[0.317, 0.247, 0.257], +[0.061, 0.022, 0.014], +[0.038, 0.017, 0.020], +[0.021, 0.004, 0.005] + ] + } +] diff --git a/website/benchmark/versions/scripts/benchmarks.sh b/website/benchmark/versions/scripts/benchmarks.sh new file mode 100644 index 00000000000..4e6baff42ef --- /dev/null +++ b/website/benchmark/versions/scripts/benchmarks.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash + +CH_QUERIES_FILE="ch_queries.sql" +SSB_QUERIES_FILE="ssb_queries.sql" +BROWN_QUERIES_FILE="brown_queries.sql" +CH_TABLE="hits_100m_obfuscated" +TRIES=3 + +if [ -x ./clickhouse ] +then + CLICKHOUSE_CLIENT="./clickhouse client" +elif command -v clickhouse-client >/dev/null 2>&1 +then + CLICKHOUSE_CLIENT="clickhouse-client" +else + echo "clickhouse-client is not found" + exit 1 +fi + +${CLICKHOUSE_CLIENT} --query 'SELECT version();' + +echo "Brown Benchmark:" + +cat "$BROWN_QUERIES_FILE" | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(${CLICKHOUSE_CLIENT} --time --format=Null --max_memory_usage=100G --query="$query" 2>&1) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + done + echo "]," +done + +echo "SSB Benchmark:" + +cat "$SSB_QUERIES_FILE" | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(${CLICKHOUSE_CLIENT} --time --format=Null --max_memory_usage=100G --query="$query" 2>&1) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + done + echo "]," +done + + +echo "ClickHouse Benchmark:" + +cat "$CH_QUERIES_FILE" | sed "s/{table}/${CH_TABLE}/g" | while read query; do + sync + echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null + + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(${CLICKHOUSE_CLIENT} --time --format=Null --max_memory_usage=100G --query="$query" 2>&1) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + done + echo "]," +done diff --git a/website/benchmark/versions/scripts/brown_queries.sql b/website/benchmark/versions/scripts/brown_queries.sql new file mode 100644 index 00000000000..f22175646b9 --- /dev/null +++ b/website/benchmark/versions/scripts/brown_queries.sql @@ -0,0 +1,16 @@ +SELECT machine_name, MIN(cpu) AS cpu_min, MAX(cpu) AS cpu_max, AVG(cpu) AS cpu_avg, MIN(net_in) AS net_in_min, MAX(net_in) AS net_in_max, AVG(net_in) AS net_in_avg, MIN(net_out) AS net_out_min, MAX(net_out) AS net_out_max, AVG(net_out) AS net_out_avg FROM ( SELECT machine_name, ifNull(cpu_user, 0.0) AS cpu, ifNull(bytes_in, 0.0) AS net_in, ifNull(bytes_out, 0.0) AS net_out FROM mgbench.logs1 WHERE machine_name IN ('anansi','aragog','urd') AND log_time >= toDateTime('2017-01-11 00:00:00')) AS r GROUP BY machine_name; -- Q1.1: What is the CPU/network utilization for each web server since midnight? +SELECT machine_name, log_time FROM mgbench.logs1 WHERE (machine_name LIKE 'cslab%' OR machine_name LIKE 'mslab%') AND load_one IS NULL AND log_time >= toDateTime('2017-01-10 00:00:00') ORDER BY machine_name, log_time; -- Q1.2: Which computer lab machines have been offline in the past day? +SELECT dt, hr, AVG(load_fifteen) AS load_fifteen_avg, AVG(load_five) AS load_five_avg, AVG(load_one) AS load_one_avg, AVG(mem_free) AS mem_free_avg, AVG(swap_free) AS swap_free_avg FROM ( SELECT CAST(log_time AS DATE) AS dt, toHour(log_time) AS hr, load_fifteen, load_five, load_one, mem_free, swap_free FROM mgbench.logs1 WHERE machine_name = 'babbage' AND load_fifteen IS NOT NULL AND load_five IS NOT NULL AND load_one IS NOT NULL AND mem_free IS NOT NULL AND swap_free IS NOT NULL AND log_time >= toDateTime('2017-01-01 00:00:00')) AS r GROUP BY dt, hr ORDER BY dt, hr; -- Q1.3: What are the hourly average metrics during the past 10 days for a specific workstation? +SELECT machine_name, COUNT(*) AS spikes FROM mgbench.logs1 WHERE machine_group = 'Servers' AND cpu_wio > 0.99 AND log_time >= toDateTime('2016-12-01 00:00:00') AND log_time < toDateTime('2017-01-01 00:00:00') GROUP BY machine_name ORDER BY spikes DESC LIMIT 10; -- Q1.4: Over 1 month, how often was each server blocked on disk I/O? +SELECT machine_name, dt, MIN(mem_free) AS mem_free_min FROM ( SELECT machine_name, CAST(log_time AS DATE) AS dt, mem_free FROM mgbench.logs1 WHERE machine_group = 'DMZ' AND mem_free IS NOT NULL ) AS r GROUP BY machine_name, dt HAVING MIN(mem_free) < 10000 ORDER BY machine_name, dt; -- Q1.5: Which externally reachable VMs have run low on memory? +SELECT dt, hr, SUM(net_in) AS net_in_sum, SUM(net_out) AS net_out_sum, SUM(net_in) + SUM(net_out) AS both_sum FROM ( SELECT CAST(log_time AS DATE) AS dt, toHour(log_time) AS hr, ifNull(bytes_in, 0.0) / 1000000000.0 AS net_in, ifNull(bytes_out, 0.0) / 1000000000.0 AS net_out FROM mgbench.logs1 WHERE machine_name IN ('allsorts','andes','bigred','blackjack','bonbon','cadbury','chiclets','cotton','crows','dove','fireball','hearts','huey','lindt','milkduds','milkyway','mnm','necco','nerds','orbit','peeps','poprocks','razzles','runts','smarties','smuggler','spree','stride','tootsie','trident','wrigley','york') ) AS r GROUP BY dt, hr ORDER BY both_sum DESC LIMIT 10; -- Q1.6: What is the total hourly network traffic across all file servers? +SELECT * FROM mgbench.logs2 WHERE status_code >= 500 AND log_time >= toDateTime('2012-12-18 00:00:00') ORDER BY log_time; -- Q2.1: Which requests have caused server errors within the past 2 weeks? +SELECT * FROM mgbench.logs2 WHERE status_code >= 200 AND status_code < 300 AND request LIKE '%/etc/passwd%' AND log_time >= toDateTime('2012-05-06 00:00:00') AND log_time < toDateTime('2012-05-20 00:00:00'); -- Q2.3: What was the average path depth for top-level requests in the past month? +SELECT top_level, AVG(length(request) - length(replaceOne(request, '/',''))) AS depth_avg FROM ( SELECT substring(request, 1, len) AS top_level, request FROM ( SELECT position('/', substring(request, 2)) AS len, request FROM mgbench.logs2 WHERE status_code >= 200 AND status_code < 300 AND log_time >= toDateTime('2012-12-01 00:00:00')) AS r WHERE len > 0 ) AS s WHERE top_level IN ('/about','/courses','/degrees','/events','/grad','/industry','/news','/people','/publications','/research','/teaching','/ugrad') GROUP BY top_level ORDER BY top_level; -- Q2.2: During a specific 2-week period, was the user password file leaked? +SELECT client_ip, COUNT(*) AS num_requests FROM mgbench.logs2 WHERE log_time >= toDateTime('2012-10-01 00:00:00') GROUP BY client_ip HAVING COUNT(*) >= 100000 ORDER BY num_requests DESC; -- Q2.4: During the last 3 months, which clients have made an excessive number of requests? +SELECT dt, COUNT(DISTINCT client_ip) FROM ( SELECT CAST(log_time AS DATE) AS dt, client_ip FROM mgbench.logs2) AS r GROUP BY dt ORDER BY dt; -- Q2.5: What are the daily unique visitors? +SELECT AVG(transfer) / 125000000.0 AS transfer_avg, MAX(transfer) / 125000000.0 AS transfer_max FROM ( SELECT log_time, SUM(object_size) AS transfer FROM mgbench.logs2 GROUP BY log_time) AS r; -- Q2.6: What are the average and maximum data transfer rates (Gbps)? +SELECT * FROM mgbench.logs3 WHERE event_type = 'temperature' AND event_value <= 32.0 AND log_time >= '2019-11-29 17:00:00'; -- Q3.1: Did the indoor temperature reach freezing over the weekend? +SELECT device_name, device_floor, COUNT(*) AS ct FROM mgbench.logs3 WHERE event_type = 'door_open' AND log_time >= '2019-06-01 00:00:00' GROUP BY device_name, device_floor ORDER BY ct DESC; -- Q3.4: Over the past 6 months, how frequently were each door opened? +SELECT DISTINCT device_name, device_type, device_floor, if (dt >= toDate('2018-12-01') AND dt < toDate('2019-03-01'), 'WINTER', 'SUMMER') FROM (SELECT dt, device_name, device_type, device_floor FROM ( SELECT dt, hr, device_name, device_type, device_floor, AVG(event_value) AS temperature_hourly_avg FROM ( SELECT CAST(log_time AS DATE) AS dt, toHour(log_time) AS hr, device_name, device_type, device_floor, event_value FROM mgbench.logs3 WHERE event_type = 'temperature' ) AS r GROUP BY dt, hr, device_name, device_type, device_floor ) AS s GROUP BY dt, device_name, device_type, device_floor HAVING MAX(temperature_hourly_avg) - MIN(temperature_hourly_avg) >= 25.0 ) -- Q3.5: Where in the building do large temperature variations occur in winter and summer? +SELECT yr, mo, SUM(coffee_hourly_avg) AS coffee_monthly_sum, AVG(coffee_hourly_avg) AS coffee_monthly_avg, SUM(printer_hourly_avg) AS printer_monthly_sum, AVG(printer_hourly_avg) AS printer_monthly_avg, SUM(projector_hourly_avg) AS projector_monthly_sum, AVG(projector_hourly_avg) AS projector_monthly_avg, SUM(vending_hourly_avg) AS vending_monthly_sum, AVG(vending_hourly_avg) AS vending_monthly_avg FROM ( SELECT dt, yr, mo, hr, AVG(coffee) AS coffee_hourly_avg, AVG(printer) AS printer_hourly_avg, AVG(projector) AS projector_hourly_avg, AVG(vending) AS vending_hourly_avg FROM ( SELECT CAST(log_time AS DATE) AS dt, toYear(log_time) AS yr, EXTRACT(MONTH FROM log_time) AS mo, toHour(log_time) AS hr, CASE WHEN device_name LIKE 'coffee%' THEN event_value END AS coffee, CASE WHEN device_name LIKE 'printer%' THEN event_value END AS printer, CASE WHEN device_name LIKE 'projector%' THEN event_value END AS projector, CASE WHEN device_name LIKE 'vending%' THEN event_value END AS vending FROM mgbench.logs3 WHERE device_type = 'meter' ) AS r GROUP BY dt, yr, mo, hr ) AS s GROUP BY yr, mo ORDER BY yr, mo; -- Q3.6: For each device category, what are the monthly power consumption metrics? \ No newline at end of file diff --git a/website/benchmark/versions/scripts/ch_queries.sql b/website/benchmark/versions/scripts/ch_queries.sql new file mode 100644 index 00000000000..89c4616c642 --- /dev/null +++ b/website/benchmark/versions/scripts/ch_queries.sql @@ -0,0 +1,43 @@ +SELECT count() FROM {table}; +SELECT count() FROM {table} WHERE AdvEngineID != 0; +SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM {table} ; +SELECT sum(UserID) FROM {table} ; +SELECT uniq(UserID) FROM {table} ; +SELECT uniq(SearchPhrase) FROM {table} ; +SELECT min(EventDate), max(EventDate) FROM {table} ; +SELECT AdvEngineID, count() FROM {table} WHERE AdvEngineID != 0 GROUP BY AdvEngineID ORDER BY count() DESC; +SELECT RegionID, uniq(UserID) AS u FROM {table} GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, sum(AdvEngineID), count() AS c, avg(ResolutionWidth), uniq(UserID) FROM {table} GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, uniq(UserID) AS u FROM {table} WHERE MobilePhoneModel != '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, uniq(UserID) AS u FROM {table} WHERE MobilePhoneModel != '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, count() AS c FROM {table} WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, uniq(UserID) AS u FROM {table} WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, count() AS c FROM {table} WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, count() FROM {table} GROUP BY UserID ORDER BY count() DESC LIMIT 10; +SELECT UserID, SearchPhrase, count() FROM {table} GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10; +SELECT UserID, SearchPhrase, count() FROM {table} GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM {table} GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10; +SELECT UserID FROM {table} WHERE UserID = 12345678901234567890; +SELECT count() FROM {table} WHERE URL LIKE '%metrika%'; +SELECT SearchPhrase, any(URL), count() AS c FROM {table} WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM {table} WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM {table} WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10; +SELECT CounterID, avg(length(URL)) AS l, count() AS c FROM {table} WHERE URL != '' GROUP BY CounterID HAVING c > 100000 ORDER BY l DESC LIMIT 25; +SELECT domainWithoutWWW(Referer) AS key, avg(length(Referer)) AS l, count() AS c, any(Referer) FROM {table} WHERE Referer != '' GROUP BY key HAVING c > 100000 ORDER BY l DESC LIMIT 25; +SELECT sum(ResolutionWidth), sum(ResolutionWidth + 1), sum(ResolutionWidth + 2), sum(ResolutionWidth + 3), sum(ResolutionWidth + 4), sum(ResolutionWidth + 5), sum(ResolutionWidth + 6), sum(ResolutionWidth + 7), sum(ResolutionWidth + 8), sum(ResolutionWidth + 9), sum(ResolutionWidth + 10), sum(ResolutionWidth + 11), sum(ResolutionWidth + 12), sum(ResolutionWidth + 13), sum(ResolutionWidth + 14), sum(ResolutionWidth + 15), sum(ResolutionWidth + 16), sum(ResolutionWidth + 17), sum(ResolutionWidth + 18), sum(ResolutionWidth + 19), sum(ResolutionWidth + 20), sum(ResolutionWidth + 21), sum(ResolutionWidth + 22), sum(ResolutionWidth + 23), sum(ResolutionWidth + 24), sum(ResolutionWidth + 25), sum(ResolutionWidth + 26), sum(ResolutionWidth + 27), sum(ResolutionWidth + 28), sum(ResolutionWidth + 29), sum(ResolutionWidth + 30), sum(ResolutionWidth + 31), sum(ResolutionWidth + 32), sum(ResolutionWidth + 33), sum(ResolutionWidth + 34), sum(ResolutionWidth + 35), sum(ResolutionWidth + 36), sum(ResolutionWidth + 37), sum(ResolutionWidth + 38), sum(ResolutionWidth + 39), sum(ResolutionWidth + 40), sum(ResolutionWidth + 41), sum(ResolutionWidth + 42), sum(ResolutionWidth + 43), sum(ResolutionWidth + 44), sum(ResolutionWidth + 45), sum(ResolutionWidth + 46), sum(ResolutionWidth + 47), sum(ResolutionWidth + 48), sum(ResolutionWidth + 49), sum(ResolutionWidth + 50), sum(ResolutionWidth + 51), sum(ResolutionWidth + 52), sum(ResolutionWidth + 53), sum(ResolutionWidth + 54), sum(ResolutionWidth + 55), sum(ResolutionWidth + 56), sum(ResolutionWidth + 57), sum(ResolutionWidth + 58), sum(ResolutionWidth + 59), sum(ResolutionWidth + 60), sum(ResolutionWidth + 61), sum(ResolutionWidth + 62), sum(ResolutionWidth + 63), sum(ResolutionWidth + 64), sum(ResolutionWidth + 65), sum(ResolutionWidth + 66), sum(ResolutionWidth + 67), sum(ResolutionWidth + 68), sum(ResolutionWidth + 69), sum(ResolutionWidth + 70), sum(ResolutionWidth + 71), sum(ResolutionWidth + 72), sum(ResolutionWidth + 73), sum(ResolutionWidth + 74), sum(ResolutionWidth + 75), sum(ResolutionWidth + 76), sum(ResolutionWidth + 77), sum(ResolutionWidth + 78), sum(ResolutionWidth + 79), sum(ResolutionWidth + 80), sum(ResolutionWidth + 81), sum(ResolutionWidth + 82), sum(ResolutionWidth + 83), sum(ResolutionWidth + 84), sum(ResolutionWidth + 85), sum(ResolutionWidth + 86), sum(ResolutionWidth + 87), sum(ResolutionWidth + 88), sum(ResolutionWidth + 89) FROM {table}; +SELECT SearchEngineID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} WHERE SearchPhrase != '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} WHERE SearchPhrase != '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, count() AS c, sum(Refresh), avg(ResolutionWidth) FROM {table} GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, count() AS c FROM {table} GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, count() AS c FROM {table} GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +SELECT ClientIP AS x, x - 1, x - 2, x - 3, count() AS c FROM {table} GROUP BY x, x - 1, x - 2, x - 3 ORDER BY c DESC LIMIT 10; +SELECT URL, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND notEmpty(URL) GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT DontCountHits AND NOT Refresh AND notEmpty(Title) GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND IsLink AND NOT IsDownload GROUP BY URL ORDER BY PageViews DESC LIMIT 1000; +SELECT TraficSourceID, SearchEngineID, AdvEngineID, ((SearchEngineID = 0 AND AdvEngineID = 0) ? Referer : '') AS Src, URL AS Dst, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 1000; +SELECT URLHash, EventDate, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND TraficSourceID IN (-1, 6) AND RefererHash = halfMD5('http://example.ru/') GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 100; +SELECT WindowClientWidth, WindowClientHeight, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND NOT Refresh AND NOT DontCountHits AND URLHash = halfMD5('http://example.ru/') GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10000; +SELECT toStartOfMinute(EventTime) AS Minute, count() AS PageViews FROM {table} WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-02' AND NOT Refresh AND NOT DontCountHits GROUP BY Minute ORDER BY Minute; diff --git a/website/benchmark/versions/scripts/runner.sh b/website/benchmark/versions/scripts/runner.sh new file mode 100644 index 00000000000..7678738b6cb --- /dev/null +++ b/website/benchmark/versions/scripts/runner.sh @@ -0,0 +1,3 @@ +#!/bin/bash +echo "VERSION = $1" +echo "DATE = $2" \ No newline at end of file diff --git a/website/benchmark/versions/scripts/ssb_queries.sql b/website/benchmark/versions/scripts/ssb_queries.sql new file mode 100644 index 00000000000..375cf511489 --- /dev/null +++ b/website/benchmark/versions/scripts/ssb_queries.sql @@ -0,0 +1,13 @@ +SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE F_YEAR = 1993 AND LO_DISCOUNT BETWEEN 1 AND 3 AND LO_QUANTITY < 25; -- Q1.1 +SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE toYYYYMM(LO_ORDERDATE) = 199401 AND LO_DISCOUNT BETWEEN 4 AND 6 AND LO_QUANTITY BETWEEN 26 AND 35; -- Q1.2 +SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE toRelativeWeekNum(LO_ORDERDATE) - toRelativeWeekNum(toDate('1994-01-01')) = 6 AND F_YEAR = 1994 AND LO_DISCOUNT BETWEEN 5 AND 7 AND LO_QUANTITY BETWEEN 26 AND 35; -- Q1.3 +SELECT sum(LO_REVENUE), F_YEAR AS year, P_BRAND FROM lineorder_flat WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' GROUP BY year, P_BRAND ORDER BY year, P_BRAND; -- Q2.1 +SELECT sum(LO_REVENUE), F_YEAR AS year, P_BRAND FROM lineorder_flat WHERE P_BRAND >= 'MFGR#2221' AND P_BRAND <= 'MFGR#2228' AND S_REGION = 'ASIA' GROUP BY year, P_BRAND ORDER BY year, P_BRAND; -- Q2.2 +SELECT sum(LO_REVENUE), F_YEAR AS year, P_BRAND FROM lineorder_flat WHERE P_BRAND = 'MFGR#2239' AND S_REGION = 'EUROPE' GROUP BY year, P_BRAND ORDER BY year, P_BRAND; -- Q2.3 +SELECT C_NATION, S_NATION, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_REGION = 'ASIA' AND S_REGION = 'ASIA' AND year >= 1992 AND year <= 1997 GROUP BY C_NATION, S_NATION, year ORDER BY year ASC, revenue DESC; -- Q3.1 +SELECT C_CITY, S_CITY, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE C_NATION = 'UNITED STATES' AND S_NATION = 'UNITED STATES' AND year >= 1992 AND year <= 1997 GROUP BY C_CITY, S_CITY, year ORDER BY year ASC, revenue DESC; -- Q3.2 +SELECT C_CITY, S_CITY, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND year >= 1992 AND year <= 1997 GROUP BY C_CITY, S_CITY, year ORDER BY year ASC, revenue DESC; -- Q3.3 +SELECT C_CITY, S_CITY, F_YEAR AS year, sum(LO_REVENUE) AS revenue FROM lineorder_flat WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND toYYYYMM(LO_ORDERDATE) = 199712 GROUP BY C_CITY, S_CITY, year ORDER BY year ASC, revenue DESC; -- Q3.4 +SELECT F_YEAR AS year, C_NATION, sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year, C_NATION ORDER BY year ASC, C_NATION ASC; -- Q4.1 +SELECT F_YEAR AS year, S_NATION, P_CATEGORY, sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (year = 1997 OR year = 1998) AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') GROUP BY year, S_NATION, P_CATEGORY ORDER BY year ASC, S_NATION ASC, P_CATEGORY ASC; -- Q4.2 +SELECT F_YEAR AS year, S_CITY, P_BRAND, sum(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat WHERE S_NATION = 'UNITED STATES' AND (year = 1997 OR year = 1998) AND P_CATEGORY = 'MFGR#14' GROUP BY year, S_CITY, P_BRAND ORDER BY year ASC, S_CITY ASC, P_BRAND ASC; -- Q4. \ No newline at end of file