diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 2280b1a7305..ecafde9e4cb 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -67,7 +67,7 @@ jobs: if: ${{ !cancelled() }} run: | export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json" - cat >> "$WORKFLOW_RESULT_FILE" << 'EOF' + cat > "$WORKFLOW_RESULT_FILE" << 'EOF' ${{ toJson(needs) }} EOF python3 ./tests/ci/ci_buddy.py --check-wf-status diff --git a/CMakeLists.txt b/CMakeLists.txt index f796e6c4616..7b4e0484ab1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -428,12 +428,17 @@ if (NOT SANITIZE) set (CMAKE_POSITION_INDEPENDENT_CODE OFF) endif() -if (OS_LINUX AND NOT (ARCH_AARCH64 OR ARCH_S390X) AND NOT SANITIZE) - # Slightly more efficient code can be generated - # It's disabled for ARM because otherwise ClickHouse cannot run on Android. +if (NOT OS_ANDROID AND OS_LINUX AND NOT ARCH_S390X AND NOT SANITIZE) + # Using '-no-pie' builds executables with fixed addresses, resulting in slightly more efficient code + # and keeping binary addresses constant even with ASLR enabled. + # Disabled on Android as it requires PIE: https://source.android.com/docs/security/enhancements#android-5 + # Disabled on IBM S390X due to build issues with 'no-pie' + # Disabled with sanitizers to avoid issues with maximum relocation size: https://github.com/ClickHouse/ClickHouse/pull/49145 set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie") +else () + message (WARNING "ClickHouse is built as PIE, system.trace_log will contain invalid addresses after server restart.") endif () if (ENABLE_TESTS) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 977efda15ff..eb3afe0ccdf 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -71,7 +71,6 @@ add_contrib (zlib-ng-cmake zlib-ng) add_contrib (bzip2-cmake bzip2) add_contrib (minizip-ng-cmake minizip-ng) add_contrib (snappy-cmake snappy) -add_contrib (rocksdb-cmake rocksdb) add_contrib (thrift-cmake thrift) # parquet/arrow/orc add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion @@ -148,6 +147,7 @@ add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift, avro, arro add_contrib (cppkafka-cmake cppkafka) add_contrib (libpqxx-cmake libpqxx) add_contrib (libpq-cmake libpq) +add_contrib (rocksdb-cmake rocksdb) # requires: jemalloc, snappy, zlib, lz4, zstd, liburing add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) add_contrib (idna-cmake idna) diff --git a/contrib/librdkafka b/contrib/librdkafka index 2d2aab6f5b7..39d4ed49ccf 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 2d2aab6f5b79db1cfca15d7bf0dee75d00d82082 +Subproject commit 39d4ed49ccf3406e2bf825d5d7b0903b5a290782 diff --git a/contrib/rocksdb b/contrib/rocksdb index 49ce8a1064d..5f003e4a22d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 49ce8a1064dd1ad89117899839bf136365e49e79 +Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 57c056532c6..44aa7494607 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -5,36 +5,38 @@ if (NOT ENABLE_ROCKSDB OR NO_SSE3_OR_HIGHER) # assumes SSE4.2 and PCLMUL return() endif() -# not in original build system, otherwise xxHash.cc fails to compile with ClickHouse C++23 default -set (CMAKE_CXX_STANDARD 20) - -# Always disable jemalloc for rocksdb by default because it introduces non-standard jemalloc APIs -option(WITH_JEMALLOC "build with JeMalloc" OFF) - -option(WITH_LIBURING "build with liburing" OFF) # TODO could try to enable this conditionally, depending on ClickHouse's ENABLE_LIBURING - # ClickHouse cannot be compiled without snappy, lz4, zlib, zstd option(WITH_SNAPPY "build with SNAPPY" ON) option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) -if(WITH_SNAPPY) +if (ENABLE_JEMALLOC AND OS_LINUX) # gives compile errors with jemalloc enabled for rocksdb on non-Linux + add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE) + list (APPEND THIRDPARTY_LIBS ch_contrib::jemalloc) +endif () + +if (ENABLE_LIBURING) + add_definitions(-DROCKSDB_IOURING_PRESENT) + list (APPEND THIRDPARTY_LIBS ch_contrib::liburing) +endif () + +if (WITH_SNAPPY) add_definitions(-DSNAPPY) list(APPEND THIRDPARTY_LIBS ch_contrib::snappy) endif() -if(WITH_ZLIB) +if (WITH_ZLIB) add_definitions(-DZLIB) list(APPEND THIRDPARTY_LIBS ch_contrib::zlib) endif() -if(WITH_LZ4) +if (WITH_LZ4) add_definitions(-DLZ4) list(APPEND THIRDPARTY_LIBS ch_contrib::lz4) endif() -if(WITH_ZSTD) +if (WITH_ZSTD) add_definitions(-DZSTD) list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) endif() @@ -88,6 +90,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/tiered_secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc + ${ROCKSDB_SOURCE_DIR}/db/attribute_group_iterator_impl.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_contents.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_fetcher.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc @@ -104,6 +107,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/blob/prefetch_buffer_collection.cc ${ROCKSDB_SOURCE_DIR}/db/builder.cc ${ROCKSDB_SOURCE_DIR}/db/c.cc + ${ROCKSDB_SOURCE_DIR}/db/coalescing_iterator.cc ${ROCKSDB_SOURCE_DIR}/db/column_family.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction.cc ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_iterator.cc @@ -124,6 +128,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_write.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_compaction_flush.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_files.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_follower.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_open.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_debug.cc ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_experimental.cc @@ -181,6 +186,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc ${ROCKSDB_SOURCE_DIR}/env/file_system.cc ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc + ${ROCKSDB_SOURCE_DIR}/env/fs_on_demand.cc ${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc ${ROCKSDB_SOURCE_DIR}/env/mock_env.cc ${ROCKSDB_SOURCE_DIR}/env/unique_id_gen.cc @@ -368,6 +374,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/volatile_tier_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/cache_simulator.cc ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc + ${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_for_tiering_collector.cc ${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc ${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc ${ROCKSDB_SOURCE_DIR}/utilities/trace/replayer_impl.cc @@ -388,6 +395,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/types_util.cc ${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/wal_filter.cc ${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc @@ -418,14 +426,18 @@ if(HAS_ARMV8_CRC) endif(HAS_ARMV8_CRC) list(APPEND SOURCES - "${ROCKSDB_SOURCE_DIR}/port/port_posix.cc" - "${ROCKSDB_SOURCE_DIR}/env/env_posix.cc" - "${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc" - "${ROCKSDB_SOURCE_DIR}/env/io_posix.cc") + ${ROCKSDB_SOURCE_DIR}/port/port_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/env_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/io_posix.cc) add_library(_rocksdb ${SOURCES}) add_library(ch_contrib::rocksdb ALIAS _rocksdb) target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS}) +# Not in the native build system but useful anyways: +# Make all functions in xxHash.h inline. Beneficial for performance: https://github.com/Cyan4973/xxHash/tree/v0.8.2#build-modifiers +target_compile_definitions (_rocksdb PRIVATE XXH_INLINE_ALL) + # SYSTEM is required to overcome some issues target_include_directories(_rocksdb SYSTEM BEFORE INTERFACE "${ROCKSDB_SOURCE_DIR}/include") diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 94603763572..a44664259fb 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.2.13" +ARG VERSION="24.7.3.42" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index f40118c7b06..2565828c846 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.7.2.13" +ARG VERSION="24.7.3.42" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 032aa862e4a..5ac8a58afea 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.7.2.13" +ARG VERSION="24.7.3.42" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 8e2f1890f89..3a4f0d97993 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -232,15 +232,26 @@ function run_tests() set +e + TEST_ARGS=( + -j 2 + --testname + --shard + --zookeeper + --check-zookeeper-session + --no-stateless + --hung-check + --print-time + --capture-client-stacktrace + "${ADDITIONAL_OPTIONS[@]}" + "$SKIP_TESTS_OPTION" + ) if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then - clickhouse-test --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \ - --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \ - -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ - "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt - else - clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ - "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt + TEST_ARGS+=( + --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" + --no-parallel-replicas + ) fi + clickhouse-test "${TEST_ARGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt set -e } diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 788bddd811d..3f734bef6a8 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -264,11 +264,22 @@ function run_tests() TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800)) START_TIME=${SECONDS} set +e - timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s \ - clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ - | ts '%Y-%m-%d %H:%M:%S' \ - | tee -a test_output/test_result.txt + + TEST_ARGS=( + --testname + --shard + --zookeeper + --check-zookeeper-session + --hung-check + --print-time + --no-drop-if-fail + --capture-client-stacktrace + --test-runs "$NUM_TRIES" + "${ADDITIONAL_OPTIONS[@]}" + ) + timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s clickhouse-test "${TEST_ARGS[@]}" 2>&1 \ + | ts '%Y-%m-%d %H:%M:%S' \ + | tee -a test_output/test_result.txt set -e DURATION=$((SECONDS - START_TIME)) diff --git a/docs/changelogs/v24.7.3.42-stable.md b/docs/changelogs/v24.7.3.42-stable.md new file mode 100644 index 00000000000..48f6e301f3c --- /dev/null +++ b/docs/changelogs/v24.7.3.42-stable.md @@ -0,0 +1,37 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.3.42-stable (63730bc4293) FIXME as compared to v24.7.2.13-stable (6e41f601b2f) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67969](https://github.com/ClickHouse/ClickHouse/issues/67969): Fixed reading of subcolumns after `ALTER ADD COLUMN` query. [#66243](https://github.com/ClickHouse/ClickHouse/pull/66243) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#67637](https://github.com/ClickHouse/ClickHouse/issues/67637): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67820](https://github.com/ClickHouse/ClickHouse/issues/67820): Fix possible deadlock on query cancel with parallel replicas. [#66905](https://github.com/ClickHouse/ClickHouse/pull/66905) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67881](https://github.com/ClickHouse/ClickHouse/issues/67881): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67713](https://github.com/ClickHouse/ClickHouse/issues/67713): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67995](https://github.com/ClickHouse/ClickHouse/issues/67995): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#67818](https://github.com/ClickHouse/ClickHouse/issues/67818): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67766](https://github.com/ClickHouse/ClickHouse/issues/67766): Fix crash of `uniq` and `uniqTheta ` with `tuple()` argument. Closes [#67303](https://github.com/ClickHouse/ClickHouse/issues/67303). [#67306](https://github.com/ClickHouse/ClickHouse/pull/67306) ([flynn](https://github.com/ucasfl)). +* Backported in [#67854](https://github.com/ClickHouse/ClickHouse/issues/67854): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#67840](https://github.com/ClickHouse/ClickHouse/issues/67840): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67518](https://github.com/ClickHouse/ClickHouse/issues/67518): Split slow test 03036_dynamic_read_subcolumns. [#66954](https://github.com/ClickHouse/ClickHouse/pull/66954) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67516](https://github.com/ClickHouse/ClickHouse/issues/67516): Split 01508_partition_pruning_long. [#66983](https://github.com/ClickHouse/ClickHouse/pull/66983) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67529](https://github.com/ClickHouse/ClickHouse/issues/67529): Reduce max time of 00763_long_lock_buffer_alter_destination_table. [#67185](https://github.com/ClickHouse/ClickHouse/pull/67185) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67643](https://github.com/ClickHouse/ClickHouse/issues/67643): [Green CI] Fix potentially flaky test_mask_sensitive_info integration test. [#67506](https://github.com/ClickHouse/ClickHouse/pull/67506) ([Alexey Katsman](https://github.com/alexkats)). +* Backported in [#67609](https://github.com/ClickHouse/ClickHouse/issues/67609): Fix test_zookeeper_config_load_balancing after adding the xdist worker name to the instance. [#67590](https://github.com/ClickHouse/ClickHouse/pull/67590) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67871](https://github.com/ClickHouse/ClickHouse/issues/67871): Fix 02434_cancel_insert_when_client_dies. [#67600](https://github.com/ClickHouse/ClickHouse/pull/67600) ([vdimir](https://github.com/vdimir)). +* Backported in [#67704](https://github.com/ClickHouse/ClickHouse/issues/67704): Fix 02910_bad_logs_level_in_local in fast tests. [#67603](https://github.com/ClickHouse/ClickHouse/pull/67603) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67689](https://github.com/ClickHouse/ClickHouse/issues/67689): Fix 01605_adaptive_granularity_block_borders. [#67605](https://github.com/ClickHouse/ClickHouse/pull/67605) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67827](https://github.com/ClickHouse/ClickHouse/issues/67827): Try fix 03143_asof_join_ddb_long. [#67620](https://github.com/ClickHouse/ClickHouse/pull/67620) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67892](https://github.com/ClickHouse/ClickHouse/issues/67892): Revert "Merge pull request [#66510](https://github.com/ClickHouse/ClickHouse/issues/66510) from canhld94/fix_trivial_count_non_deterministic_func". [#67800](https://github.com/ClickHouse/ClickHouse/pull/67800) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index db3eabaecfc..c49492c1cb4 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -27,23 +27,23 @@ Avoid dumping copies of external code into the library directory. Instead create a Git submodule to pull third-party code from an external upstream repository. All submodules used by ClickHouse are listed in the `.gitmodule` file. -If the library can be used as-is (the default case), you can reference the upstream repository directly. -If the library needs patching, create a fork of the upstream repository in the [ClickHouse organization on GitHub](https://github.com/ClickHouse). +- If the library can be used as-is (the default case), you can reference the upstream repository directly. +- If the library needs patching, create a fork of the upstream repository in the [ClickHouse organization on GitHub](https://github.com/ClickHouse). In the latter case, we aim to isolate custom patches as much as possible from upstream commits. -To that end, create a branch with prefix `clickhouse/` from the branch or tag you want to integrate, e.g. `clickhouse/master` (for branch `master`) or `clickhouse/release/vX.Y.Z` (for tag `release/vX.Y.Z`). -This ensures that pulls from the upstream repository into the fork will leave custom `clickhouse/` branches unaffected. -Submodules in `contrib/` must only track `clickhouse/` branches of forked third-party repositories. +To that end, create a branch with prefix `ClickHouse/` from the branch or tag you want to integrate, e.g. `ClickHouse/2024_2` (for branch `2024_2`) or `ClickHouse/release/vX.Y.Z` (for tag `release/vX.Y.Z`). +Avoid following upstream development branches `master`/ `main` / `dev` (i.e., prefix branches `ClickHouse/master` / `ClickHouse/main` / `ClickHouse/dev` in the fork repository). +Such branches are moving targets which make proper versioning harder. +"Prefix branches" ensure that pulls from the upstream repository into the fork will leave custom `ClickHouse/` branches unaffected. +Submodules in `contrib/` must only track `ClickHouse/` branches of forked third-party repositories. -Patches are only applied against `clickhouse/` branches of external libraries. -For that, push the patch as a branch with `clickhouse/`, e.g. `clickhouse/fix-some-desaster`. -Then create a PR from the new branch against the custom tracking branch with `clickhouse/` prefix, (e.g. `clickhouse/master` or `clickhouse/release/vX.Y.Z`) and merge the patch. +Patches are only applied against `ClickHouse/` branches of external libraries. + +There are two ways to do that: +- you like to make a new fix against a `ClickHouse/`-prefix branch in the forked repository, e.g. a sanitizer fix. In that case, push the fix as a branch with `ClickHouse/` prefix, e.g. `ClickHouse/fix-sanitizer-disaster`. Then create a PR from the new branch against the custom tracking branch, e.g. `ClickHouse/2024_2 <-- ClickHouse/fix-sanitizer-disaster` and merge the PR. +- you update the submodule and need to re-apply earlier patches. In this case, re-creating old PRs is overkill. Instead, simply cherry-pick older commits into the new `ClickHouse/` branch (corresponding to the new version). Feel free to squash commits of PRs that had multiple commits. In the best case, we did contribute custom patches back to upstream and can omit patches in the new version. + +Once the submodule has been updated, bump the submodule in ClickHouse to point to the new hash in the fork. Create patches of third-party libraries with the official repository in mind and consider contributing the patch back to the upstream repository. This makes sure that others will also benefit from the patch and it will not be a maintenance burden for the ClickHouse team. - -To pull upstream changes into the submodule, you can use two methods: -- (less work but less clean): merge upstream `master` into the corresponding `clickhouse/` tracking branch in the forked repository. You will need to resolve merge conflicts with previous custom patches. This method can be used when the `clickhouse/` branch tracks an upstream development branch like `master`, `main`, `dev`, etc. -- (more work but cleaner): create a new branch with `clickhouse/` prefix from the upstream commit or tag you like to integrate. Then re-apply all existing patches using new PRs (or squash them into a single PR). This method can be used when the `clickhouse/` branch tracks a specific upstream version branch or tag. It is cleaner in the sense that custom patches and upstream changes are better isolated from each other. - -Once the submodule has been updated, bump the submodule in ClickHouse to point to the new hash in the fork. diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index 5e81eacc937..20c7c511aa9 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -61,6 +61,7 @@ Engines in the family: - [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) - [S3Queue](../../engines/table-engines/integrations/s3queue.md) +- [TimeSeries](../../engines/table-engines/integrations/time-series.md) ### Special Engines {#special-engines} diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 7bdc856c9fd..de6492e8ea7 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -251,6 +251,44 @@ The number of rows in one Kafka message depends on whether the format is row-bas - For row-based formats the number of rows in one Kafka message can be controlled by setting `kafka_max_rows_per_message`. - For block-based formats we cannot divide block into smaller parts, but the number of rows in one block can be controlled by general setting [max_block_size](../../../operations/settings/settings.md#setting-max_block_size). +## Experimental engine to store committed offsets in ClickHouse Keeper + +If `allow_experimental_kafka_offsets_storage_in_keeper` is enabled, then two more settings can be specified to the Kafka table engine: + - `kafka_keeper_path` specifies the path to the table in ClickHouse Keeper + - `kafka_replica_name` specifies the replica name in ClickHouse Keeper + +Either both of the settings must be specified or neither of them. When both of them are specified, then a new, experimental Kafka engine will be used. The new engine doesn't depend on storing the committed offsets in Kafka, but stores them in ClickHouse Keeper. It still tries to commit the offsets to Kafka, but it only depends on those offsets when the table is created. In any other circumstances (table is restarted, or recovered after some error) the offsets stored in ClickHouse Keeper will be used as an offset to continue consuming messages from. Apart from the committed offset, it also stores how many messages were consumed in the last batch, so if the insert fails, the same amount of messages will be consumed, thus enabling deduplication if necessary. + +Example: + +``` sql +CREATE TABLE experimental_kafka (key UInt64, value UInt64) +ENGINE = Kafka('localhost:19092', 'my-topic', 'my-consumer', 'JSONEachRow') +SETTINGS + kafka_keeper_path = '/clickhouse/{database}/experimental_kafka', + kafka_replica_name = 'r1' +SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1; +``` + +Or to utilize the `uuid` and `replica` macros similarly to ReplicatedMergeTree: + +``` sql +CREATE TABLE experimental_kafka (key UInt64, value UInt64) +ENGINE = Kafka('localhost:19092', 'my-topic', 'my-consumer', 'JSONEachRow') +SETTINGS + kafka_keeper_path = '/clickhouse/{database}/{uuid}', + kafka_replica_name = '{replica}' +SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1; +``` + +### Known limitations + +As the new engine is experimental, it is not production ready yet. There are few known limitations of the implementation: + - The biggest limitation is the engine doesn't support direct reading. Reading from the engine using materialized views and writing to the engine work, but direct reading doesn't. As a result, all direct `SELECT` queries will fail. + - Rapidly dropping and recreating the table or specifying the same ClickHouse Keeper path to different engines might cause issues. As best practice you can use the `{uuid}` in `kafka_keeper_path` to avoid clashing paths. + - To make repeatable reads, messages cannot be consumed from multiple partitions on a single thread. On the other hand, the Kafka consumers have to be polled regularly to keep them alive. As a result of these two objectives, we decided to only allow creating multiple consumers if `kafka_thread_per_consumer` is enabled, otherwise it is too complicated to avoid issues regarding polling consumers regularly. + - Consumers created by the new storage engine do not show up in [`system.kafka_consumers`](../../../operations/system-tables/kafka_consumers.md) table. + **See Also** - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/docs/en/engines/table-engines/integrations/time-series.md b/docs/en/engines/table-engines/integrations/time-series.md new file mode 100644 index 00000000000..4830fd61d27 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/time-series.md @@ -0,0 +1,295 @@ +--- +slug: /en/engines/table-engines/special/time_series +sidebar_position: 60 +sidebar_label: TimeSeries +--- + +# TimeSeries Engine [Experimental] + +A table engine storing time series, i.e. a set of values associated with timestamps and tags (or labels): + +``` +metric_name1[tag1=value1, tag2=value2, ...] = {timestamp1: value1, timestamp2: value2, ...} +metric_name2[...] = ... +``` + +:::info +This is an experimental feature that may change in backwards-incompatible ways in the future releases. +Enable usage of the TimeSeries table engine +with [allow_experimental_time_series_table](../../../operations/settings/settings.md#allow-experimental-time-series-table) setting. +Input the command `set allow_experimental_time_series_table = 1`. +::: + +## Syntax {#syntax} + +``` sql +CREATE TABLE name [(columns)] ENGINE=TimeSeries +[SETTINGS var1=value1, ...] +[DATA db.data_table_name | DATA ENGINE data_table_engine(arguments)] +[TAGS db.tags_table_name | TAGS ENGINE tags_table_engine(arguments)] +[METRICS db.metrics_table_name | METRICS ENGINE metrics_table_engine(arguments)] +``` + +## Usage {#usage} + +It's easier to start with everything set by default (it's allowed to create a `TimeSeries` table without specifying a list of columns): + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +``` + +Then this table can be used with the following protocols (a port must be assigned in the server configuration): +- [prometheus remote-write](../../../interfaces/prometheus.md#remote-write) +- [prometheus remote-read](../../../interfaces/prometheus.md#remote-read) + +## Target tables {#target-tables} + +A `TimeSeries` table doesn't have its own data, everything is stored in its target tables. +This is similar to how a [materialized view](../../../sql-reference/statements/create/view#materialized-view) works, +with the difference that a materialized view has one target table +whereas a `TimeSeries` table has three target tables named [data]{#data-table}, [tags]{#tags-table], and [metrics]{#metrics-table}. + +The target tables can be either specified explicitly in the `CREATE TABLE` query +or the `TimeSeries` table engine can generate inner target tables automatically. + +The target tables are the following: +1. The _data_ table {#data-table} contains time series associated with some identifier. +The _data_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `id` | [x] | `UUID` | any | Identifies a combination of a metric names and tags | +| `timestamp` | [x] | `DateTime64(3)` | `DateTime64(X)` | A time point | +| `value` | [x] | `Float64` | `Float32` or `Float64` | A value associated with the `timestamp` | + +2. The _tags_ table {#tags-table} contains identifiers calculated for each combination of a metric name and tags. +The _tags_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `id` | [x] | `UUID` | any (must match the type of `id` in the [data]{#data-table} table) | An `id` identifies a combination of a metric name and tags. The DEFAULT expression specifies how to calculate such an identifier | +| `metric_name` | [x] | `LowCardinality(String)` | `String` or `LowCardinality(String)` | The name of a metric | +| `` | [ ] | `String` | `String` or `LowCardinality(String)` or `LowCardinality(Nullable(String))` | The value of a specific tag, the tag's name and the name of a corresponding column are specified in the [tags_to_columns](#settings) setting | +| `tags` | [x] | `Map(LowCardinality(String), String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Map of tags excluding the tag `__name__` containing the name of a metric and excluding tags with names enumerated in the [tags_to_columns](#settings) setting | +| `all_tags` | [ ] | `Map(String, String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` | +| `min_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Minimum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | +| `max_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Maximum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` | + +3. The _metrics_ table {#metrics-table} contains some information about metrics been collected, the types of those metrics and their descriptions. +The _metrics_ table must have columns: + +| Name | Mandatory? | Default type | Possible types | Description | +|---|---|---|---|---| +| `metric_family_name` | [x] | `String` | `String` or `LowCardinality(String)` | The name of a metric family | +| `type` | [x] | `String` | `String` or `LowCardinality(String)` | The type of a metric family, one of "counter", "gauge", "summary", "stateset", "histogram", "gaugehistogram" | +| `unit` | [x] | `String` | `String` or `LowCardinality(String)` | The unit used in a metric | +| `help` | [x] | `String` | `String` or `LowCardinality(String)` | The description of a metric | + +Any row inserted into a `TimeSeries` table will be in fact stored in those three target tables. +A `TimeSeries` table contains all those columns from the [data]{#data-table}, [tags]{#tags-table}, [metrics]{#metrics-table} tables. + +## Creation {#creation} + +There are multiple ways to create a table with the `TimeSeries` table engine. +The simplest statement + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +``` + +will actually create the following table (you can see that by executing `SHOW CREATE TABLE my_table`): + +``` sql +CREATE TABLE my_table +( + `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), + `timestamp` DateTime64(3), + `value` Float64, + `metric_name` LowCardinality(String), + `tags` Map(LowCardinality(String), String), + `all_tags` Map(String, String), + `min_time` Nullable(DateTime64(3)), + `max_time` Nullable(DateTime64(3)), + `metric_family_name` String, + `type` String, + `unit` String, + `help` String +) +ENGINE = TimeSeries +DATA ENGINE = MergeTree ORDER BY (id, timestamp) +DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +TAGS ENGINE = AggregatingMergeTree PRIMARY KEY metric_name ORDER BY (metric_name, id) +TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +METRICS ENGINE = ReplacingMergeTree ORDER BY metric_family_name +METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +So the columns were generated automatically and also there are three inner UUIDs in this statement - +one per each inner target table that was created. +(Inner UUIDs are not shown normally until setting +[show_table_uuid_in_table_create_query_if_not_nil](../../../operations/settings/settings#show_table_uuid_in_table_create_query_if_not_nil) +is set.) + +Inner target tables have names like `.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, +`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, `.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +and each target table has columns which is a subset of the columns of the main `TimeSeries` table: + +``` sql +CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID, + `timestamp` DateTime64(3), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp) +``` + +``` sql +CREATE TABLE default.`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)), + `metric_name` LowCardinality(String), + `tags` Map(LowCardinality(String), String), + `all_tags` Map(String, String) EPHEMERAL, + `min_time` SimpleAggregateFunction(min, Nullable(DateTime64(3))), + `max_time` SimpleAggregateFunction(max, Nullable(DateTime64(3))) +) +ENGINE = AggregatingMergeTree +PRIMARY KEY metric_name +ORDER BY (metric_name, id) +``` + +``` sql +CREATE TABLE default.`.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `metric_family_name` String, + `type` String, + `unit` String, + `help` String +) +ENGINE = ReplacingMergeTree +ORDER BY metric_family_name +``` + +## Adjusting types of columns {#adjusting-column-types} + +You can adjust the types of almost any column of the inner target tables by specifying them explicitly +while defining the main table. For example, + +``` sql +CREATE TABLE my_table +( + timestamp DateTime64(6) +) ENGINE=TimeSeries +``` + +will make the inner [data]{#data-table} table store timestamp in microseconds instead of milliseconds: + +``` sql +CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx` +( + `id` UUID, + `timestamp` DateTime64(6), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp) +``` + +## The `id` column {#id-column} + +The `id` column contains identifiers, every identifier is calculated for a combination of a metric name and tags. +The DEFAULT expression for the `id` column is an expression which will be used to calculate such identifiers. +Both the type of the `id` column and that expression can be adjusted by specifying them explicitly: + +``` sql +CREATE TABLE my_table +( + id UInt64 DEFAULT sipHash64(metric_name, all_tags) +) ENGINE=TimeSeries +``` + +## The `tags` and `all_tags` columns {#tags-and-all-tags} + +There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different +if setting `tags_to_columns` is used. This setting allows to specify that a specific tag should be stored in a separate column instead of storing +in a map inside the `tags` column: + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} +``` + +This statement will add columns +``` + `instance` String, + `job` String +``` +to the definition of both `my_table` and its inner [tags]{#tags-table} target table. In this case the `tags` column will not contain tags `instance` and `job`, +but the `all_tags` column will contain them. The `all_tags` column is ephemeral and its only purpose to be used in the DEFAULT expression +for the `id` column. + +The types of columns can be adjusted by specifying them explicitly: + +``` sql +CREATE TABLE my_table (instance LowCardinality(String), job LowCardinality(Nullable(String))) +ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'} +``` + +## Table engines of inner target tables {#inner-table-engines} + +By default inner target tables use the following table engines: +- the [data]{#data-table} table uses [MergeTree](../mergetree-family/mergetree); +- the [tags]{#tags-table} table uses [AggregatingMergeTree](../mergetree-family/aggregatingmergetree) because the same data is often inserted multiple times to this table so we need a way +to remove duplicates, and also because it's required to do aggregation for columns `min_time` and `max_time`; +- the [metrics]{#metrics-table} table uses [ReplacingMergeTree](../mergetree-family/replacingmergetree) because the same data is often inserted multiple times to this table so we need a way +to remove duplicates. + +Other table engines also can be used for inner target tables if it's specified so: + +``` sql +CREATE TABLE my_table ENGINE=TimeSeries +DATA ENGINE=ReplicatedMergeTree +TAGS ENGINE=ReplicatedAggregatingMergeTree +METRICS ENGINE=ReplicatedReplacingMergeTree +``` + +## External target tables {#external-target-tables} + +It's possible to make a `TimeSeries` table use a manually created table: + +``` sql +CREATE TABLE data_for_my_table +( + `id` UUID, + `timestamp` DateTime64(3), + `value` Float64 +) +ENGINE = MergeTree +ORDER BY (id, timestamp); + +CREATE TABLE tags_for_my_table ... + +CREATE TABLE metrics_for_my_table ... + +CREATE TABLE my_table ENGINE=TimeSeries DATA data_for_my_table TAGS tags_for_my_table METRICS metrics_for_my_table; +``` + +## Settings {#settings} + +Here is a list of settings which can be specified while defining a `TimeSeries` table: + +| Name | Type | Default | Description | +|---|---|---|---| +| `tags_to_columns` | Map | {} | Map specifying which tags should be put to separate columns in the [tags]{#tags-table} table. Syntax: `{'tag1': 'column1', 'tag2' : column2, ...}` | +| `use_all_tags_column_to_generate_id` | Bool | true | When generating an expression to calculate an identifier of a time series, this flag enables using the `all_tags` column in that calculation | +| `store_min_time_and_max_time` | Bool | true | If set to true then the table will store `min_time` and `max_time` for each time series | +| `aggregate_min_time_and_max_time` | Bool | true | When creating an inner target `tags` table, this flag enables using `SimpleAggregateFunction(min, Nullable(DateTime64(3)))` instead of just `Nullable(DateTime64(3))` as the type of the `min_time` column, and the same for the `max_time` column | +| `filter_by_min_time_and_max_time` | Bool | true | If set to true then the table will use the `min_time` and `max_time` columns for filtering time series | + +# Functions {#functions} + +Here is a list of functions supporting a `TimeSeries` table as an argument: +- [timeSeriesData](../../../sql-reference/table-functions/timeSeriesData.md) +- [timeSeriesTags](../../../sql-reference/table-functions/timeSeriesTags.md) +- [timeSeriesMetrics](../../../sql-reference/table-functions/timeSeriesMetrics.md) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7ffbd9a5bae..183b94f4641 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1005,7 +1005,7 @@ They can be used for prewhere optimization only if we enable `set allow_statisti ## Column-level Settings {#column-level-settings} -Certain MergeTree settings can be override at column level: +Certain MergeTree settings can be overridden at column level: - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md new file mode 100644 index 00000000000..bf541901b34 --- /dev/null +++ b/docs/en/interfaces/prometheus.md @@ -0,0 +1,160 @@ +--- +slug: /en/interfaces/prometheus +sidebar_position: 19 +sidebar_label: Prometheus protocols +--- + +# Prometheus protocols + +## Exposing metrics {#expose} + +:::note +ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. +::: + +ClickHouse can expose its own metrics for scraping from Prometheus: + +```xml + + 9363 + /metrics + true + true + true + true + + +Section `` can be used to make more extended handlers. +This section is similar to [](/en/interfaces/http) but works for prometheus protocols: + +```xml + + 9363 + + + /metrics + + expose_metrics + true + true + true + true + + + + +``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the exposing metrics protocol. | +| `endpoint` | `/metrics` | HTTP endpoint for scraping metrics by prometheus server. Starts with `/`. Should not be used with the `` section. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](/en/interfaces/http) section. | +| `metrics` | true | Expose metrics from the [system.metrics](/en/operations/system-tables/metrics) table. | +| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](/en/operations/system-tables/asynchronous_metrics) table. | +| `events` | true | Expose metrics from the [system.events](/en/operations/system-tables/events) table. | +| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](/en/operations/system-tables/errors) as well. | + +Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): +```bash +curl 127.0.0.1:9363/metrics +``` + +## Remote-write protocol {#remote-write} + +ClickHouse supports the [remote-write](https://prometheus.io/docs/specs/remote_write_spec/) protocol. +Data are received by this protocol and written to a [TimeSeries](/en/engines/table-engines/special/time_series) table +(which should be created beforehand). + +```xml + + 9363 + + + /write + + remote_writedb_name + time_series_table
+
+
+
+
+``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the `remote-write` protocol. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](/en/interfaces/http) section. | +| `table` | none | The name of a [TimeSeries](/en/engines/table-engines/special/time_series) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. | +| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | + +## Remote-read protocol {#remote-read} + +ClickHouse supports the [remote-read](https://prometheus.io/docs/prometheus/latest/querying/remote_read_api/) protocol. +Data are read from a [TimeSeries](/en/engines/table-engines/special/time_series) table and sent via this protocol. + +```xml + + 9363 + + + /read + + remote_readdb_name + time_series_table
+
+
+
+
+``` + +Settings: + +| Name | Default | Description | +|---|---|---|---| +| `port` | none | Port for serving the `remote-read` protocol. | +| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [](/en/interfaces/http) section. | +| `table` | none | The name of a [TimeSeries](/en/engines/table-engines/special/time_series) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. | +| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. | + +## Configuration for multiple protocols {#multiple-protocols} + +Multiple protocols can be specified together in one place: + +```xml + + 9363 + + + /metrics + + expose_metrics + true + true + true + true + + + + /write + + remote_writedb_name.time_series_table + + + + /read + + remote_readdb_name.time_series_table + + + + +``` diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a1e3c292b04..68f61650e00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2112,48 +2112,6 @@ The trailing slash is mandatory. /var/lib/clickhouse/ ``` -## Prometheus {#prometheus} - -:::note -ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. -::: - -Exposing metrics data for scraping from [Prometheus](https://prometheus.io). - -Settings: - -- `endpoint` – HTTP endpoint for scraping metrics by prometheus server. Start from ‘/’. -- `port` – Port for `endpoint`. -- `metrics` – Expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table. -- `events` – Expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table. -- `asynchronous_metrics` – Expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table. -- `errors` - Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../../operations/system-tables/asynchronous_metrics.md#system_tables-errors) as well. - -**Example** - -``` xml - - 0.0.0.0 - 8123 - 9000 - - - /metrics - 9363 - true - true - true - true - - - -``` - -Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server): -```bash -curl 127.0.0.1:9363/metrics -``` - ## query_log {#query-log} Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 968481062e9..e432f4e038f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5626,3 +5626,14 @@ Default value: `False` Disable all insert and mutations (alter table update / alter table delete / alter table drop partition). Set to true, can make this node focus on reading queries. Default value: `false`. + +## allow_experimental_time_series_table {#allow-experimental-time-series-table} + +Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. + +Possible values: + +- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. +- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. + +Default value: `0`. diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index 5adc33de37f..a60de2a08d1 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/trace_log --- # trace_log -Contains stack traces collected by the sampling query profiler. +Contains stack traces collected by the [sampling query profiler](../../operations/optimizing-performance/sampling-query-profiler.md). ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also see settings: [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns), [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns), [memory_profiler_step](../../operations/settings/settings.md#memory_profiler_step), [memory_profiler_sample_probability](../../operations/settings/settings.md#memory_profiler_sample_probability), [trace_profile_events](../../operations/settings/settings.md#trace_profile_events). diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 778816f8934..1bb7817364a 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -9,6 +9,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget - [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it. - [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part. +- [DROP DETACHED PARTITION\|PART](#drop-detached-partitionpart) - Delete a part or all parts of a partition from `detached`. - [FORGET PARTITION](#forget-partition) — Deletes a partition metadata from zookeeper if it's empty. - [ATTACH PARTITION\|PART](#attach-partitionpart) — Adds a partition or part from the `detached` directory to the table. - [ATTACH PARTITION FROM](#attach-partition-from) — Copies the data partition from one table to another and adds. @@ -68,7 +69,7 @@ ALTER TABLE mt DROP PART 'all_4_4_0'; ## DROP DETACHED PARTITION\|PART ``` sql -ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partition_expr +ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART ALL|partition_expr ``` Removes the specified part or all parts of the specified partition from `detached`. diff --git a/docs/en/sql-reference/table-functions/fuzzQuery.md b/docs/en/sql-reference/table-functions/fuzzQuery.md new file mode 100644 index 00000000000..e15f8a40156 --- /dev/null +++ b/docs/en/sql-reference/table-functions/fuzzQuery.md @@ -0,0 +1,36 @@ +--- +slug: /en/sql-reference/table-functions/fuzzQuery +sidebar_position: 75 +sidebar_label: fuzzQuery +--- + +# fuzzQuery + +Perturbs the given query string with random variations. + +``` sql +fuzzQuery(query[, max_query_length[, random_seed]]) +``` + +**Arguments** + +- `query` (String) - The source query to perform the fuzzing on. +- `max_query_length` (UInt64) - A maximum length the query can get during the fuzzing process. +- `random_seed` (UInt64) - A random seed for producing stable results. + +**Returned Value** + +A table object with a single column containing perturbed query strings. + +## Usage Example + +``` sql +SELECT * FROM fuzzQuery('SELECT materialize(\'a\' AS key) GROUP BY key') LIMIT 2; +``` + +``` + ┌─query──────────────────────────────────────────────────────────┐ +1. │ SELECT 'a' AS key GROUP BY key │ +2. │ EXPLAIN PIPELINE compact = true SELECT 'a' AS key GROUP BY key │ + └────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 28cba5ccc6a..405ac477846 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -44,6 +44,7 @@ LIMIT 2 Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. - `*` — Represents arbitrarily many characters except `/` but including the empty string. +- `**` — Represents all files inside a folder recursively. - `?` — Represents an arbitrary single character. - `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. The strings can contain the `/` symbol. - `{N..M}` — Represents any number `>= N` and `<= M`. diff --git a/docs/en/sql-reference/table-functions/timeSeriesData.md b/docs/en/sql-reference/table-functions/timeSeriesData.md new file mode 100644 index 00000000000..aa7a9d30c2a --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesData.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesData +sidebar_position: 145 +sidebar_label: timeSeriesData +--- + +# timeSeriesData + +`timeSeriesData(db_name.time_series_table)` - Returns the [data](../../engines/table-engines/integrations/time-series.md#data-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA data_table +``` + +The function also works if the _data_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesData(db_name.time_series_table); +SELECT * FROM timeSeriesData('db_name.time_series_table'); +SELECT * FROM timeSeriesData('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesMetrics.md b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md new file mode 100644 index 00000000000..913f1185bca --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesMetrics.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesMetrics +sidebar_position: 145 +sidebar_label: timeSeriesMetrics +--- + +# timeSeriesMetrics + +`timeSeriesMetrics(db_name.time_series_table)` - Returns the [metrics](../../engines/table-engines/integrations/time-series.md#metrics-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS metrics_table +``` + +The function also works if the _metrics_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesMetrics(db_name.time_series_table); +SELECT * FROM timeSeriesMetrics('db_name.time_series_table'); +SELECT * FROM timeSeriesMetrics('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/table-functions/timeSeriesTags.md b/docs/en/sql-reference/table-functions/timeSeriesTags.md new file mode 100644 index 00000000000..663a7dc6ac8 --- /dev/null +++ b/docs/en/sql-reference/table-functions/timeSeriesTags.md @@ -0,0 +1,28 @@ +--- +slug: /en/sql-reference/table-functions/timeSeriesTags +sidebar_position: 145 +sidebar_label: timeSeriesTags +--- + +# timeSeriesTags + +`timeSeriesTags(db_name.time_series_table)` - Returns the [tags](../../engines/table-engines/integrations/time-series.md#tags-table) table +used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md): + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS tags_table +``` + +The function also works if the _tags_ table is inner: + +``` sql +CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef' +``` + +The following queries are equivalent: + +``` sql +SELECT * FROM timeSeriesTags(db_name.time_series_table); +SELECT * FROM timeSeriesTags('db_name.time_series_table'); +SELECT * FROM timeSeriesTags('db_name', 'time_series_table'); +``` diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index de6e9005baa..01bf809e76e 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/lagInFrame sidebar_label: lagInFrame -sidebar_position: 8 +sidebar_position: 9 --- # lagInFrame diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index 4a82c03f6e6..dae4353b582 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/leadInFrame sidebar_label: leadInFrame -sidebar_position: 9 +sidebar_position: 10 --- # leadInFrame diff --git a/docs/en/sql-reference/window-functions/percent_rank.md b/docs/en/sql-reference/window-functions/percent_rank.md new file mode 100644 index 00000000000..2e348f2a333 --- /dev/null +++ b/docs/en/sql-reference/window-functions/percent_rank.md @@ -0,0 +1,72 @@ +--- +slug: /en/sql-reference/window-functions/percent_rank +sidebar_label: percent_rank +sidebar_position: 8 +--- + +# percent_rank + +returns the relative rank (i.e. percentile) of rows within a window partition. + +**Syntax** + +Alias: `percentRank` (case-sensitive) + +```sql +percent_rank (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] | [window_name]) +FROM table_name +WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column] RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +``` + +The default and required window frame definition is `RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Example** + + +Query: + +```sql +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT player, salary, + percent_rank() OVER (ORDER BY salary DESC) AS percent_rank +FROM salaries; +``` + +Result: + +```response + + ┌─player──────────┬─salary─┬───────percent_rank─┐ +1. │ Gary Chen │ 195000 │ 0 │ +2. │ Robert George │ 195000 │ 0 │ +3. │ Charles Juarez │ 190000 │ 0.3333333333333333 │ +4. │ Michael Stanley │ 150000 │ 0.5 │ +5. │ Scott Harrison │ 150000 │ 0.5 │ +6. │ Douglas Benson │ 150000 │ 0.5 │ +7. │ James Henderson │ 140000 │ 1 │ + └─────────────────┴────────┴────────────────────┘ + +``` diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 546a674d41a..f3e8b0f46b7 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -81,6 +81,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U Обрабатываться будут те и только те файлы, которые существуют в файловой системе и удовлетворяют всему шаблону пути. - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. +- `**` — Заменяет любое количество любых символов, включая `/`, то есть осуществляет рекурсивный поиск по вложенным директориям. - `?` — заменяет ровно один любой символ. - `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. Эти строки также могут содержать символ `/`. - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). diff --git a/docs/ru/sql-reference/table-functions/hdfs.md b/docs/ru/sql-reference/table-functions/hdfs.md index 6dcb1a21791..13f1bdc43af 100644 --- a/docs/ru/sql-reference/table-functions/hdfs.md +++ b/docs/ru/sql-reference/table-functions/hdfs.md @@ -47,6 +47,7 @@ LIMIT 2 - `*` — Заменяет любое количество любых символов (кроме `/`), включая отсутствие символов. +- `**` — Заменяет любое количество любых символов, включая `/`, то есть осуществляет рекурсивный поиск по вложенным директориям. - `?` — Заменяет ровно один любой символ. - `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. Эти строки также могут содержать символ `/`. - `{N..M}` — Заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). diff --git a/programs/client/Client.h b/programs/client/Client.h index 7fdf77031ab..07a8e293b1a 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -11,7 +11,10 @@ class Client : public ClientApplicationBase public: using Arguments = ClientApplicationBase::Arguments; - Client() = default; + Client() + { + fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr); + } void initialize(Poco::Util::Application & self) override; diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 6bb9d6f9b41..a447a9e50f6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -38,7 +38,7 @@ #include #include #include -#include +#include #include #include "Core/Defines.h" @@ -509,14 +509,13 @@ try auto address = socketBindListen(socket, listen_host, port); socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); socket.setSendTimeout(my_http_context->getSendTimeout()); - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); servers->emplace_back( listen_host, port_name, "Prometheus: http://" + address.toString(), std::make_unique( std::move(my_http_context), - createPrometheusMainHandlerFactory(*this, config_getter(), metrics_writer, "PrometheusHandler-factory"), + createKeeperPrometheusHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0d731ed0e14..6b0b8fc5b50 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -51,6 +50,7 @@ #include #include #include +#include #include #include #include @@ -216,12 +216,12 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str return system_database; } -static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context) +static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) { - auto overlay = std::make_shared(name_, context); - overlay->registerNextDatabase(std::make_shared(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context)); - overlay->registerNextDatabase(std::make_shared(name_, "", context)); - return overlay; + auto databaseCombiner = std::make_shared(name_, context_); + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); + databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); + return databaseCombiner; } /// If path is specified and not empty, will try to setup server environment and load existing metadata @@ -367,7 +367,7 @@ std::string LocalServer::getInitialCreateTableQuery() else table_structure = "(" + table_structure + ")"; - return fmt::format("CREATE TEMPORARY TABLE {} {} ENGINE = File({}, {});", + return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});", table_name, table_structure, data_format, table_file); } @@ -761,12 +761,7 @@ void LocalServer::processConfig() DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); std::string default_database = server_settings.default_database; - { - DatabasePtr database = createClickHouseLocalDatabaseOverlay(default_database, global_context); - if (UUID uuid = database->getUUID(); uuid != UUIDHelpers::Nil) - DatabaseCatalog::instance().addUUIDMapping(uuid); - DatabaseCatalog::instance().attachDatabase(default_database, database); - } + DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context)); global_context->setCurrentDatabase(default_database); if (getClientConfiguration().has("path")) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a362e18b7c2..51fcd494111 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -814,10 +814,11 @@ try const size_t physical_server_memory = getMemoryAmount(); - LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", + LOG_INFO(log, "Available RAM: {}; logical cores: {}; used cores: {}.", formatReadableSizeWithBinarySuffix(physical_server_memory), - getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores - std::thread::hardware_concurrency()); + std::thread::hardware_concurrency(), + getNumberOfPhysicalCPUCores() // on ARM processors it can show only enabled at current moment cores + ); #if defined(__x86_64__) String cpu_info; @@ -1623,7 +1624,7 @@ try concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num; if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0) { - auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency(); + auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * getNumberOfPhysicalCPUCores(); if (value > 0 && value < concurrent_threads_soft_limit) concurrent_threads_soft_limit = value; } diff --git a/src/AggregateFunctions/WindowFunction.h b/src/AggregateFunctions/WindowFunction.h new file mode 100644 index 00000000000..f7fbd7389ea --- /dev/null +++ b/src/AggregateFunctions/WindowFunction.h @@ -0,0 +1,117 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} +class WindowTransform; + + +// Interface for true window functions. It's not much of an interface, they just +// accept the guts of WindowTransform and do 'something'. Given a small number of +// true window functions, and the fact that the WindowTransform internals are +// pretty much well-defined in domain terms (e.g. frame boundaries), this is +// somewhat acceptable. +class IWindowFunction +{ +public: + virtual ~IWindowFunction() = default; + + // Must insert the result for current_row. + virtual void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const = 0; + + virtual std::optional getDefaultFrame() const { return {}; } + + virtual ColumnPtr castColumn(const Columns &, const std::vector &) { return nullptr; } + + /// Is the frame type supported by this function. + virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } +}; + +// Runtime data for computing one window function. +struct WindowFunctionWorkspace +{ + AggregateFunctionPtr aggregate_function; + + // Cached value of aggregate function isState virtual method + bool is_aggregate_function_state = false; + + // This field is set for pure window functions. When set, we ignore the + // window_function.aggregate_function, and work through this interface + // instead. + IWindowFunction * window_function_impl = nullptr; + + std::vector argument_column_indices; + + // Will not be initialized for a pure window function. + mutable AlignedBuffer aggregate_function_state; + + // Argument columns. Be careful, this is a per-block cache. + std::vector argument_columns; + UInt64 cached_block_number = std::numeric_limits::max(); +}; + +// A basic implementation for a true window function. It pretends to be an +// aggregate function, but refuses to work as such. +struct WindowFunction : public IAggregateFunctionHelper, public IWindowFunction +{ + std::string name; + + WindowFunction( + const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) + : IAggregateFunctionHelper(argument_types_, parameters_, result_type_), name(name_) + { + } + + bool isOnlyWindowFunction() const override { return true; } + + [[noreturn]] void fail() const + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The function '{}' can only be used as a window function, not as an aggregate function", getName()); + } + + String getName() const override { return name; } + void create(AggregateDataPtr __restrict) const override { } + void destroy(AggregateDataPtr __restrict) const noexcept override { } + bool hasTrivialDestructor() const override { return true; } + size_t sizeOfData() const override { return 0; } + size_t alignOfData() const override { return 1; } + void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { fail(); } + void merge(AggregateDataPtr __restrict, ConstAggregateDataPtr, Arena *) const override { fail(); } + void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { fail(); } + void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional, Arena *) const override { fail(); } + void insertResultInto(AggregateDataPtr __restrict, IColumn &, Arena *) const override { fail(); } +}; + +template +struct StatefulWindowFunction : public WindowFunction +{ + StatefulWindowFunction( + const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) + : WindowFunction(name_, argument_types_, parameters_, result_type_) + { + } + + size_t sizeOfData() const override { return sizeof(State); } + size_t alignOfData() const override { return 1; } + + void create(AggregateDataPtr __restrict place) const override { new (place) State(); } + + void destroy(AggregateDataPtr __restrict place) const noexcept override { reinterpret_cast(place)->~State(); } + + bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v; } + + State & getState(const WindowFunctionWorkspace & workspace) const + { + return *reinterpret_cast(workspace.aggregate_function_state.data()); + } +}; + +} diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 8c133971785..98dd0601a1b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -78,6 +78,7 @@ add_headers_and_sources(clickhouse_common_io Common/Scheduler) add_headers_and_sources(clickhouse_common_io Common/Scheduler/Nodes) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) +add_headers_and_sources(clickhouse_common_io IO/Protobuf) add_headers_and_sources(clickhouse_common_io IO/S3) add_headers_and_sources(clickhouse_common_io IO/AzureBlobStorage) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) @@ -225,6 +226,7 @@ add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) add_object_library(clickhouse_storages_s3queue Storages/ObjectStorageQueue) add_object_library(clickhouse_storages_materializedview Storages/MaterializedView) +add_object_library(clickhouse_storages_time_series Storages/TimeSeries) add_object_library(clickhouse_client Client) # Always compile this file with the highest possible level of optimizations, even in Debug builds. # https://github.com/ClickHouse/ClickHouse/issues/65745 @@ -469,6 +471,7 @@ dbms_target_link_libraries (PUBLIC ch_contrib::sparsehash) if (TARGET ch_contrib::protobuf) dbms_target_link_libraries (PRIVATE ch_contrib::protobuf) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::protobuf) endif () if (TARGET clickhouse_grpc_protos) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 1a23b6b1363..45251aea28a 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include #include diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 072184e0a66..7595a29912b 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -365,7 +365,7 @@ bool LocalConnection::poll(size_t) { while (pollImpl()) { - LOG_DEBUG(&Poco::Logger::get("LocalConnection"), "Executor timeout encountered, will retry"); + LOG_TEST(&Poco::Logger::get("LocalConnection"), "Executor timeout encountered, will retry"); if (needSendProgressOrMetrics()) return true; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b1b8e2367a4..44a1cd071cb 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -604,6 +604,10 @@ M(723, PARQUET_EXCEPTION) \ M(724, TOO_MANY_TABLES) \ M(725, TOO_MANY_DATABASES) \ + M(726, UNEXPECTED_HTTP_HEADERS) \ + M(727, UNEXPECTED_TABLE_ENGINE) \ + M(728, UNEXPECTED_DATA_TYPE) \ + M(729, ILLEGAL_TIME_SERIES_TAGS) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index f5ec8cf0356..0b1ec552d43 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -60,6 +60,7 @@ static struct InitFiu ONCE(receive_timeout_on_table_status_response) \ REGULAR(keepermap_fail_drop_data) \ REGULAR(lazy_pipe_fds_fail_close) \ + PAUSEABLE(infinite_sleep) \ namespace FailPoints diff --git a/src/Client/QueryFuzzer.cpp b/src/Common/QueryFuzzer.cpp similarity index 97% rename from src/Client/QueryFuzzer.cpp rename to src/Common/QueryFuzzer.cpp index f5b700ea529..161c38f20e0 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Common/QueryFuzzer.cpp @@ -68,22 +68,21 @@ Field QueryFuzzer::getRandomField(int type) { case 0: { - return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) - / sizeof(*bad_int64_values))]; + return bad_int64_values[fuzz_rand() % std::size(bad_int64_values)]; } case 1: { static constexpr double values[] = {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999, 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20, - FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; + FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % std::size(values)]; } case 2: { static constexpr UInt64 scales[] = {0, 1, 2, 10}; return DecimalField( - bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))], - static_cast(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]) + bad_int64_values[fuzz_rand() % std::size(bad_int64_values)], + static_cast(scales[fuzz_rand() % std::size(scales)]) ); } default: @@ -165,7 +164,8 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -174,12 +174,14 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -197,7 +199,9 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -206,12 +210,16 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -344,7 +352,8 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } } @@ -378,7 +387,8 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) if (col) impl->children.insert(pos, col); else - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by @@ -1361,11 +1371,15 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) collectFuzzInfoMain(ast); fuzz(ast); - std::cout << std::endl; - WriteBufferFromOStream ast_buf(std::cout, 4096); - formatAST(*ast, ast_buf, false /*highlight*/); - ast_buf.finalize(); - std::cout << std::endl << std::endl; + if (out_stream) + { + *out_stream << std::endl; + + WriteBufferFromOStream ast_buf(*out_stream, 4096); + formatAST(*ast, ast_buf, false /*highlight*/); + ast_buf.finalize(); + *out_stream << std::endl << std::endl; + } } } diff --git a/src/Client/QueryFuzzer.h b/src/Common/QueryFuzzer.h similarity index 91% rename from src/Client/QueryFuzzer.h rename to src/Common/QueryFuzzer.h index 6165e589cae..35d088809f2 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -35,9 +35,31 @@ struct ASTWindowDefinition; * queries, so you want to feed it a lot of queries to get some interesting mix * of them. Normally we feed SQL regression tests to it. */ -struct QueryFuzzer +class QueryFuzzer { - pcg64 fuzz_rand{randomSeed()}; +public: + explicit QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) + : fuzz_rand(fuzz_rand_) + , out_stream(out_stream_) + , debug_stream(debug_stream_) + { + } + + // This is the only function you have to call -- it will modify the passed + // ASTPtr to point to new AST with some random changes. + void fuzzMain(ASTPtr & ast); + + ASTs getInsertQueriesForFuzzedTables(const String & full_query); + ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); + void notifyQueryFailed(ASTPtr ast); + + static bool isSuitableForFuzzing(const ASTCreateQuery & create); + +private: + pcg64 fuzz_rand; + + std::ostream * out_stream = nullptr; + std::ostream * debug_stream = nullptr; // We add elements to expression lists with fixed probability. Some elements // are so large, that the expected number of elements we add to them is @@ -66,10 +88,6 @@ struct QueryFuzzer std::unordered_map index_of_fuzzed_table; std::set created_tables_hashes; - // This is the only function you have to call -- it will modify the passed - // ASTPtr to point to new AST with some random changes. - void fuzzMain(ASTPtr & ast); - // Various helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); @@ -77,9 +95,6 @@ struct QueryFuzzer ASTPtr getRandomExpressionList(); DataTypePtr fuzzDataType(DataTypePtr type); DataTypePtr getRandomType(); - ASTs getInsertQueriesForFuzzedTables(const String & full_query); - ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); - void notifyQueryFailed(ASTPtr ast); void replaceWithColumnLike(ASTPtr & ast); void replaceWithTableLike(ASTPtr & ast); void fuzzOrderByElement(ASTOrderByElement * elem); @@ -102,8 +117,6 @@ struct QueryFuzzer void addTableLike(ASTPtr ast); void addColumnLike(ASTPtr ast); void collectFuzzInfoRecurse(ASTPtr ast); - - static bool isSuitableForFuzzing(const ASTCreateQuery & create); }; } diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index e025e49e0a3..c4358da2453 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -629,6 +629,7 @@ void HandledSignals::setupTerminateHandler() void HandledSignals::setupCommonDeadlySignalHandlers() { /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. + /// NOTE: that it is also used by clickhouse-test wrapper addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, true); #if defined(SANITIZER) diff --git a/src/Common/getNumberOfPhysicalCPUCores.h b/src/Common/getNumberOfPhysicalCPUCores.h index 827e95e1bea..9e3412fdcba 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.h +++ b/src/Common/getNumberOfPhysicalCPUCores.h @@ -1,4 +1,5 @@ #pragma once /// Get number of CPU cores without hyper-threading. +/// The calculation respects possible cgroups limits. unsigned getNumberOfPhysicalCPUCores(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b3db27f8aef..f04c696645a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -690,6 +690,7 @@ class IColumn; M(UInt64, max_size_to_preallocate_for_joins, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before join", 0) \ \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ + M(Bool, allow_experimental_kafka_offsets_storage_in_keeper, false, "Allow experimental feature to store Kafka related offsets in ClickHouse Keeper. When enabled a ClickHouse Keeper path and replica name can be specified to the Kafka table engine. As a result instead of the regular Kafka engine, a new type of storage engine will be used that stores the committed offsets primarily in ClickHouse Keeper", 0) \ M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ M(Bool, allow_aggregate_partitions_independently, false, "Enable independent aggregation of partitions on separate threads when partition key suits group by key. Beneficial when number of partitions close to number of cores and partitions have roughly the same size", 0) \ M(Bool, force_aggregate_partitions_independently, false, "Force the use of optimization when it is applicable, but heuristics decided not to use it", 0) \ @@ -905,6 +906,7 @@ class IColumn; M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ + M(Bool, allow_experimental_time_series_table, false, "Allows experimental TimeSeries table engine", 0) \ M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 41d5180ead0..41e4ac2e154 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,7 +79,9 @@ static std::initializer_listgetPath()) / "metadata"); + fs::create_directories(path_to_table_symlinks); + tryCreateMetadataSymlink(); } DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_) @@ -60,16 +63,6 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, C { } -void DatabaseAtomic::createDirectories() -{ - if (database_atomic_directories_created.test_and_set()) - return; - DatabaseOnDisk::createDirectories(); - fs::create_directories(fs::path(getContext()->getPath()) / "metadata"); - fs::create_directories(path_to_table_symlinks); - tryCreateMetadataSymlink(); -} - String DatabaseAtomic::getTableDataPath(const String & table_name) const { std::lock_guard lock(mutex); @@ -106,7 +99,6 @@ void DatabaseAtomic::drop(ContextPtr) void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, const StoragePtr & table, const String & relative_table_path) { assert(relative_table_path != data_path && !relative_table_path.empty()); - createDirectories(); DetachedTables not_in_use; std::lock_guard lock(mutex); not_in_use = cleanupDetachedTables(); @@ -208,15 +200,11 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ if (exchange && !supportsAtomicRename()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported"); - createDirectories(); waitDatabaseStarted(); auto & other_db = dynamic_cast(to_database); bool inside_database = this == &other_db; - if (!inside_database) - other_db.createDirectories(); - String old_metadata_path = getObjectMetadataPath(table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); @@ -337,7 +325,6 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) { - createDirectories(); DetachedTables not_in_use; auto table_data_path = getTableDataPath(query); try @@ -474,9 +461,6 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin if (mode < LoadingStrictnessLevel::FORCE_RESTORE) return; - if (!fs::exists(path_to_table_symlinks)) - return; - /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken for (const auto & table_path : fs::directory_iterator(path_to_table_symlinks)) { @@ -604,7 +588,6 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new { /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard - createDirectories(); waitDatabaseStarted(); bool check_ref_deps = query_context->getSettingsRef().check_referential_table_dependencies; @@ -696,5 +679,4 @@ void registerDatabaseAtomic(DatabaseFactory & factory) }; factory.registerDatabase("Atomic", create_fn); } - } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index ca24494f600..4a4ccfa2573 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -76,9 +76,6 @@ protected: using DetachedTables = std::unordered_map; [[nodiscard]] DetachedTables cleanupDetachedTables() TSA_REQUIRES(mutex); - std::atomic_flag database_atomic_directories_created = ATOMIC_FLAG_INIT; - void createDirectories(); - void tryCreateMetadataSymlink(); virtual bool allowMoveTableToOtherDatabaseEngine(IDatabase & /*to_database*/) const { return false; } diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index e43adfc5d37..3fb6d30fcb8 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -47,13 +47,12 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, : DatabaseOnDisk(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseLazy (" + name_ + ")", context_) , expiration_time(expiration_time_) { - createDirectories(); } void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/) { - iterateMetadataFiles([this, &local_context](const String & file_name) + iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) { const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index aeac130594f..41cfb751141 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -12,7 +12,7 @@ class DatabaseLazyIterator; class Context; /** Lazy engine of databases. - * Works like DatabaseOrdinary, but stores only recently accessed tables in memory. + * Works like DatabaseOrdinary, but stores in memory only the cache. * Can be used only with *Log engines. */ class DatabaseLazy final : public DatabaseOnDisk diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 82a81b0b32d..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -172,14 +172,7 @@ DatabaseOnDisk::DatabaseOnDisk( , metadata_path(metadata_path_) , data_path(data_path_) { -} - - -void DatabaseOnDisk::createDirectories() -{ - if (directories_created.test_and_set()) - return; - fs::create_directories(std::filesystem::path(getContext()->getPath()) / data_path); + fs::create_directories(local_context->getPath() + data_path); fs::create_directories(metadata_path); } @@ -197,8 +190,6 @@ void DatabaseOnDisk::createTable( const StoragePtr & table, const ASTPtr & query) { - createDirectories(); - const auto & settings = local_context->getSettingsRef(); const auto & create = query->as(); assert(table_name == create.getTable()); @@ -266,6 +257,7 @@ void DatabaseOnDisk::createTable( } commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context); + removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false); } @@ -293,8 +285,6 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora { try { - createDirectories(); - /// Add a table to the map of known tables. attachTable(query_context, query.getTable(), table, getTableDataPath(query)); @@ -430,7 +420,6 @@ void DatabaseOnDisk::renameTable( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases of different engines is not supported"); } - createDirectories(); waitDatabaseStarted(); auto table_data_relative_path = getTableDataPath(table_name); @@ -579,14 +568,14 @@ void DatabaseOnDisk::drop(ContextPtr local_context) assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty()); if (local_context->getSettingsRef().force_remove_data_recursively_on_drop) { - (void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path); + (void)fs::remove_all(local_context->getPath() + getDataPath()); (void)fs::remove_all(getMetadataPath()); } else { try { - (void)fs::remove(std::filesystem::path(getContext()->getPath()) / data_path); + (void)fs::remove(local_context->getPath() + getDataPath()); (void)fs::remove(getMetadataPath()); } catch (const fs::filesystem_error & e) @@ -624,18 +613,15 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n } } -void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const +void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const { - if (!fs::exists(metadata_path)) - return; - auto process_tmp_drop_metadata_file = [&](const String & file_name) { assert(getUUID() == UUIDHelpers::Nil); static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); - if (fs::exists(std::filesystem::path(getContext()->getPath()) / data_path / object_name)) + if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name)) { fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name)); @@ -652,7 +638,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_meta std::vector> metadata_files; fs::directory_iterator dir_end; - for (fs::directory_iterator dir_it(metadata_path); dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) { String file_name = dir_it->path().filename(); /// For '.svn', '.gitignore' directory and similar. diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 0c0ecf76a26..12656068643 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -64,7 +64,7 @@ public: time_t getObjectMetadataModificationTime(const String & object_name) const override; String getDataPath() const override { return data_path; } - String getTableDataPath(const String & table_name) const override { return std::filesystem::path(data_path) / escapeForFileName(table_name) / ""; } + String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; } String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); } String getMetadataPath() const override { return metadata_path; } @@ -83,7 +83,7 @@ protected: using IteratingFunction = std::function; - void iterateMetadataFiles(const IteratingFunction & process_metadata_file) const; + void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const; ASTPtr getCreateTableQueryImpl( const String & table_name, @@ -99,9 +99,6 @@ protected: virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach); virtual void setDetachedTableNotInUseForce(const UUID & /*uuid*/) {} - std::atomic_flag directories_created = ATOMIC_FLAG_INIT; - void createDirectories(); - const String metadata_path; const String data_path; }; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index dd8a3f42ea8..8808261654f 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -55,7 +55,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; static constexpr const char * const CONVERT_TO_REPLICATED_FLAG_NAME = "convert_to_replicated"; DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_) - : DatabaseOrdinary(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseOrdinary (" + name_ + ")", context_) + : DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) { } @@ -265,7 +265,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables } }; - iterateMetadataFiles(process_metadata); + iterateMetadataFiles(local_context, process_metadata); size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count; size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f127ccbc224..fe00c1c60aa 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -338,9 +339,12 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const return std::make_shared(getContext()->getSettingsRef(), shards, params); } -std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr & cluster_) const +ReplicasInfo DatabaseReplicated::tryGetReplicasInfo(const ClusterPtr & cluster_) const { Strings paths; + + paths.emplace_back(fs::path(zookeeper_path) / "max_log_ptr"); + const auto & addresses_with_failover = cluster_->getShardsAddresses(); const auto & shards_info = cluster_->getShardsInfo(); for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) @@ -349,22 +353,50 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "log_ptr"); } } try { auto current_zookeeper = getZooKeeper(); - auto res = current_zookeeper->exists(paths); + auto zk_res = current_zookeeper->tryGet(paths); - std::vector statuses; - statuses.resize(paths.size()); + auto max_log_ptr_zk = zk_res[0]; + if (max_log_ptr_zk.error != Coordination::Error::ZOK) + throw Coordination::Exception(max_log_ptr_zk.error); - for (size_t i = 0; i < res.size(); ++i) - if (res[i].error == Coordination::Error::ZOK) - statuses[i] = 1; + UInt32 max_log_ptr = parse(max_log_ptr_zk.data); - return statuses; + ReplicasInfo replicas_info; + replicas_info.resize((zk_res.size() - 1) / 2); + + size_t global_replica_index = 0; + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) + { + for (const auto & replica : addresses_with_failover[shard_index]) + { + auto replica_active = zk_res[2 * global_replica_index + 1]; + auto replica_log_ptr = zk_res[2 * global_replica_index + 2]; + + UInt64 recovery_time = 0; + { + std::lock_guard lock(ddl_worker_mutex); + if (replica.is_local && ddl_worker) + recovery_time = ddl_worker->getCurrentInitializationDurationMs(); + } + + replicas_info[global_replica_index] = ReplicaInfo{ + .is_active = replica_active.error == Coordination::Error::ZOK, + .replication_lag = replica_log_ptr.error != Coordination::Error::ZNONODE ? std::optional(max_log_ptr - parse(replica_log_ptr.data)) : std::nullopt, + .recovery_time = recovery_time, + }; + + ++global_replica_index; + } + } + + return replicas_info; } catch (...) { @@ -373,7 +405,6 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr } } - void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco::Util::AbstractConfiguration & config_ref) { const auto & config_prefix = fmt::format("named_collections.{}", collection_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 27ab262d1f1..db683be8f36 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -17,6 +19,14 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; +struct ReplicaInfo +{ + bool is_active; + std::optional replication_lag; + UInt64 recovery_time; +}; +using ReplicasInfo = std::vector; + class DatabaseReplicated : public DatabaseAtomic { public: @@ -84,7 +94,7 @@ public: static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, bool throw_if_noop); - std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; + ReplicasInfo tryGetReplicasInfo(const ClusterPtr & cluster_) const; void renameDatabase(ContextPtr query_context, const String & new_name) override; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 1ef88dc03bc..4e7408aa96e 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -32,6 +32,12 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db bool DatabaseReplicatedDDLWorker::initializeMainThread() { + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.emplace(); + initialization_duration_timer->start(); + } + while (!stop_flag) { try @@ -69,6 +75,10 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() initializeReplication(); initialized = true; + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } return true; } catch (...) @@ -78,6 +88,11 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread() } } + { + std::lock_guard lock(initialization_duration_timer_mutex); + initialization_duration_timer.reset(); + } + return false; } @@ -459,4 +474,10 @@ UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const return max_id.load(); } +UInt64 DatabaseReplicatedDDLWorker::getCurrentInitializationDurationMs() const +{ + std::lock_guard lock(initialization_duration_timer_mutex); + return initialization_duration_timer ? initialization_duration_timer->elapsedMilliseconds() : 0; +} + } diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 41edf2221b8..2309c831839 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -36,6 +36,8 @@ public: DatabaseReplicated * const database, bool committed = false); /// NOLINT UInt32 getLogPointer() const; + + UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; void initializeReplication(); @@ -56,6 +58,9 @@ private: ZooKeeperPtr active_node_holder_zookeeper; /// It will remove "active" node when database is detached zkutil::EphemeralNodeHolderPtr active_node_holder; + + std::optional initialization_duration_timer; + mutable std::mutex initialization_duration_timer_mutex; }; } diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 495733e15fd..801356b3dd7 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -14,8 +14,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int CANNOT_GET_CREATE_TABLE_QUERY; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TABLE; } DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_) @@ -126,39 +124,6 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab getEngineName()); } -void DatabasesOverlay::renameTable( - ContextPtr current_context, - const String & name, - IDatabase & to_database, - const String & to_name, - bool exchange, - bool dictionary) -{ - for (auto & db : databases) - { - if (db->isTableExist(name, current_context)) - { - if (DatabasesOverlay * to_overlay_database = typeid_cast(&to_database)) - { - /// Renaming from Overlay database inside itself or into another Overlay database. - /// Just use the first database in the overlay as a destination. - if (to_overlay_database->databases.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The destination Overlay database {} does not have any members", to_database.getDatabaseName()); - - db->renameTable(current_context, name, *to_overlay_database->databases[0], to_name, exchange, dictionary); - } - else - { - /// Renaming into a different type of database. E.g. from Overlay on top of Atomic database into just Atomic database. - db->renameTable(current_context, name, to_database, to_name, exchange, dictionary); - } - - return; - } - } - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(getDatabaseName()), backQuote(name)); -} - ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const { ASTPtr result = nullptr; @@ -213,18 +178,6 @@ String DatabasesOverlay::getTableDataPath(const ASTCreateQuery & query) const return result; } -UUID DatabasesOverlay::getUUID() const -{ - UUID result = UUIDHelpers::Nil; - for (const auto & db : databases) - { - result = db->getUUID(); - if (result != UUIDHelpers::Nil) - break; - } - return result; -} - UUID DatabasesOverlay::tryGetTableUUID(const String & table_name) const { UUID result = UUIDHelpers::Nil; diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index 40c653e5cb5..b0c7e7e4032 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -35,21 +35,12 @@ public: StoragePtr detachTable(ContextPtr context, const String & table_name) override; - void renameTable( - ContextPtr current_context, - const String & name, - IDatabase & to_database, - const String & to_name, - bool exchange, - bool dictionary) override; - ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; - UUID getUUID() const override; UUID tryGetTableUUID(const String & table_name) const override; void drop(ContextPtr context) override; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 02418abb2b0..f94326d220e 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -416,7 +416,6 @@ public: std::lock_guard lock{mutex}; return database_name; } - /// Get UUID of database. virtual UUID getUUID() const { return UUIDHelpers::Nil; } diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 8b3850c4e0c..2f5477a6b9d 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -46,7 +46,6 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( , settings(std::move(settings_)) , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get()) { - createDirectories(); } void DatabaseMaterializedMySQL::rethrowExceptionIfNeeded() const diff --git a/src/Functions/seriesPeriodDetectFFT.cpp b/src/Functions/seriesPeriodDetectFFT.cpp index 471354235d5..ecf8398bbd5 100644 --- a/src/Functions/seriesPeriodDetectFFT.cpp +++ b/src/Functions/seriesPeriodDetectFFT.cpp @@ -153,12 +153,8 @@ public: return true; } - std::vector xfreq(spec_len); double step = 0.5 / (spec_len - 1); - for (size_t i = 0; i < spec_len; ++i) - xfreq[i] = i * step; - - auto freq = xfreq[idx]; + auto freq = idx * step; period = std::round(1 / freq); return true; diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 62ee19fa904..b6e4b36ee64 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +33,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace FailPoints +{ + extern const char infinite_sleep[]; +} + /** sleep(seconds) - the specified number of seconds sleeps each columns. */ @@ -107,6 +113,8 @@ public: { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. UInt64 microseconds = static_cast(seconds * 1e6); + FailPointInjection::pauseFailPoint(FailPoints::infinite_sleep); + if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {} microseconds", max_microseconds, microseconds); diff --git a/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp new file mode 100644 index 00000000000..86b7eb4d7f7 --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.cpp @@ -0,0 +1,56 @@ +#include "config.h" + +#if USE_PROTOBUF +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ProtobufZeroCopyInputStreamFromReadBuffer::ProtobufZeroCopyInputStreamFromReadBuffer(std::unique_ptr in_) : in(std::move(in_)) +{ +} + +ProtobufZeroCopyInputStreamFromReadBuffer::~ProtobufZeroCopyInputStreamFromReadBuffer() = default; + +bool ProtobufZeroCopyInputStreamFromReadBuffer::Next(const void ** data, int * size) +{ + if (in->eof()) + return false; + *data = in->position(); + *size = static_cast(in->available()); + in->position() += *size; + return true; +} + +void ProtobufZeroCopyInputStreamFromReadBuffer::BackUp(int count) +{ + if (static_cast(in->offset()) < count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "ProtobufZeroCopyInputStreamFromReadBuffer::BackUp() cannot back up {} bytes (max = {} bytes)", + count, + in->offset()); + + in->position() -= count; +} + +bool ProtobufZeroCopyInputStreamFromReadBuffer::Skip(int count) +{ + return static_cast(in->tryIgnore(count)) == count; +} + +int64_t ProtobufZeroCopyInputStreamFromReadBuffer::ByteCount() const +{ + return in->count(); +} + +} + +#endif diff --git a/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h new file mode 100644 index 00000000000..3f86815ef3f --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyInputStreamFromReadBuffer.h @@ -0,0 +1,38 @@ +#pragma once + +#include "config.h" +#if USE_PROTOBUF + +#include + + +namespace DB +{ +class ReadBuffer; + +class ProtobufZeroCopyInputStreamFromReadBuffer : public google::protobuf::io::ZeroCopyInputStream +{ +public: + explicit ProtobufZeroCopyInputStreamFromReadBuffer(std::unique_ptr in_); + ~ProtobufZeroCopyInputStreamFromReadBuffer() override; + + // Obtains a chunk of data from the stream. + bool Next(const void ** data, int * size) override; + + // Backs up a number of bytes, so that the next call to Next() returns + // data again that was already returned by the last call to Next(). + void BackUp(int count) override; + + // Skips a number of bytes. + bool Skip(int count) override; + + // Returns the total number of bytes read since this object was created. + int64_t ByteCount() const override; + +private: + std::unique_ptr in; +}; + +} + +#endif diff --git a/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp new file mode 100644 index 00000000000..d1e02b436f3 --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.cpp @@ -0,0 +1,60 @@ +#include "config.h" + +#if USE_PROTOBUF +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ProtobufZeroCopyOutputStreamFromWriteBuffer::ProtobufZeroCopyOutputStreamFromWriteBuffer(WriteBuffer & out_) : out(&out_) +{ +} + +ProtobufZeroCopyOutputStreamFromWriteBuffer::ProtobufZeroCopyOutputStreamFromWriteBuffer(std::unique_ptr out_) + : ProtobufZeroCopyOutputStreamFromWriteBuffer(*out_) +{ + out_holder = std::move(out_); +} + +ProtobufZeroCopyOutputStreamFromWriteBuffer::~ProtobufZeroCopyOutputStreamFromWriteBuffer() = default; + +bool ProtobufZeroCopyOutputStreamFromWriteBuffer::Next(void ** data, int * size) +{ + *data = out->position(); + *size = static_cast(out->available()); + out->position() += *size; + return true; +} + +void ProtobufZeroCopyOutputStreamFromWriteBuffer::BackUp(int count) +{ + if (static_cast(out->offset()) < count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "ProtobufZeroCopyOutputStreamFromWriteBuffer::BackUp() cannot back up {} bytes (max = {} bytes)", + count, + out->offset()); + + out->position() -= count; +} + +int64_t ProtobufZeroCopyOutputStreamFromWriteBuffer::ByteCount() const +{ + return out->count(); +} + +void ProtobufZeroCopyOutputStreamFromWriteBuffer::finalize() +{ + out->finalize(); +} + +} + +#endif diff --git a/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h new file mode 100644 index 00000000000..c47cef9ff4d --- /dev/null +++ b/src/IO/Protobuf/ProtobufZeroCopyOutputStreamFromWriteBuffer.h @@ -0,0 +1,40 @@ +#pragma once + +#include "config.h" +#if USE_PROTOBUF + +#include + + +namespace DB +{ +class WriteBuffer; + +class ProtobufZeroCopyOutputStreamFromWriteBuffer : public google::protobuf::io::ZeroCopyOutputStream +{ +public: + explicit ProtobufZeroCopyOutputStreamFromWriteBuffer(WriteBuffer & out_); + explicit ProtobufZeroCopyOutputStreamFromWriteBuffer(std::unique_ptr out_); + + ~ProtobufZeroCopyOutputStreamFromWriteBuffer() override; + + // Obtains a buffer into which data can be written. + bool Next(void ** data, int * size) override; + + // Backs up a number of bytes, so that the end of the last buffer returned + // by Next() is not actually written. + void BackUp(int count) override; + + // Returns the total number of bytes written since this object was created. + int64_t ByteCount() const override; + + void finalize(); + +private: + WriteBuffer * out; + std::unique_ptr out_holder; +}; + +} + +#endif diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index ca40d0656d1..0e02b48e1e0 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -16,7 +16,13 @@ namespace ErrorCodes } SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t buf_size, char * existing_memory, size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) + : SnappyWriteBuffer(*out_, buf_size, existing_memory, alignment) +{ + out_holder = std::move(out_); +} + +SnappyWriteBuffer::SnappyWriteBuffer(WriteBuffer & out_, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(&out_) { } diff --git a/src/IO/SnappyWriteBuffer.h b/src/IO/SnappyWriteBuffer.h index 2ff86fb64ef..b7a084d0f80 100644 --- a/src/IO/SnappyWriteBuffer.h +++ b/src/IO/SnappyWriteBuffer.h @@ -18,6 +18,12 @@ public: char * existing_memory = nullptr, size_t alignment = 0); + explicit SnappyWriteBuffer( + WriteBuffer & out_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + ~SnappyWriteBuffer() override; void finalizeImpl() override { finish(); } @@ -28,7 +34,9 @@ private: void finishImpl(); void finish(); - std::unique_ptr out; + WriteBuffer * out; + std::unique_ptr out_holder; + bool finished = false; String uncompress_buffer; diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 356bffa75e9..a28c7c1bff3 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -100,6 +101,7 @@ private: const String database_name; std::set external_tables; + mutable std::unordered_set with_aliases; bool only_replace_current_database_function = false; bool only_replace_in_join = false; @@ -117,6 +119,10 @@ private: void visit(ASTSelectQuery & select, ASTPtr &) const { + if (select.recursive_with) + for (const auto & child : select.with()->children) + with_aliases.insert(child->as()->name); + if (select.tables()) tryVisit(select.refTables()); @@ -165,6 +171,9 @@ private: /// There is temporary table with such name, should not be rewritten. if (external_tables.contains(identifier.shortName())) return; + /// This is WITH RECURSIVE alias. + if (with_aliases.contains(identifier.name())) + return; auto qualified_identifier = std::make_shared(database_name, identifier.name()); if (!identifier.alias.empty()) diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 39fdef23baa..daf1e300046 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -254,6 +254,8 @@ String toString(ClientInfo::Interface interface) return "LOCAL"; case ClientInfo::Interface::TCP_INTERSERVER: return "TCP_INTERSERVER"; + case ClientInfo::Interface::PROMETHEUS: + return "PROMETHEUS"; } return std::format("Unknown server interface ({}).", static_cast(interface)); diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index ca32b4c5cfa..48dea3cc3ea 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -38,6 +38,7 @@ public: POSTGRESQL = 5, LOCAL = 6, TCP_INTERSERVER = 7, + PROMETHEUS = 8, }; enum class HTTPMethod : uint8_t diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c767aeb2ec2..7063b2162a0 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -38,6 +38,7 @@ #include #include +#include #include #include @@ -590,6 +591,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAG & actions, Aggrega void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, const WindowDescriptions & existing_descriptions, + AggregateFunctionPtr aggregate_function, WindowDescription & desc, const IAST * ast) { const auto & definition = ast->as(); @@ -698,7 +700,21 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, ast->formatForErrorMessage()); } + const auto * window_function = aggregate_function ? dynamic_cast(aggregate_function.get()) : nullptr; desc.frame.is_default = definition.frame_is_default; + if (desc.frame.is_default && window_function) + { + auto default_window_frame_opt = window_function->getDefaultFrame(); + if (default_window_frame_opt) + { + desc.frame = *default_window_frame_opt; + /// Append the default frame description to window_name, make sure it will be put into + /// a proper window description. + desc.window_name += " " + desc.frame.toString(); + return; + } + } + desc.frame.type = definition.frame_type; desc.frame.begin_type = definition.frame_begin_type; desc.frame.begin_preceding = definition.frame_begin_preceding; @@ -734,7 +750,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) WindowDescription desc; desc.window_name = elem.name; makeWindowDescriptionFromAST(*current_context, window_descriptions, - desc, elem.definition.get()); + nullptr, desc, elem.definition.get()); auto [it, inserted] = window_descriptions.insert( {elem.name, std::move(desc)}); @@ -821,12 +837,12 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) WindowDescription desc; desc.window_name = default_window_name; makeWindowDescriptionFromAST(*current_context, window_descriptions, - desc, &definition); + window_function.aggregate_function, desc, &definition); auto full_sort_description = desc.full_sort_description; auto [it, inserted] = window_descriptions.insert( - {default_window_name, std::move(desc)}); + {desc.window_name, std::move(desc)}); if (!inserted) { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 0c00247df85..dc038e10594 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -135,7 +135,12 @@ public: /// A list of windows for window functions. const WindowDescriptions & windowDescriptions() const { return window_descriptions; } - void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast); + void makeWindowDescriptionFromAST( + const Context & context, + const WindowDescriptions & existing_descriptions, + AggregateFunctionPtr aggregate_function, + WindowDescription & desc, + const IAST * ast); void makeWindowDescriptions(ActionsDAG & actions); /** Checks if subquery is not a plain StorageSet. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3f0fe51b0e2..a101e5e8f09 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -751,6 +752,10 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti if (create.storage && create.storage->engine) getContext()->checkAccess(AccessType::TABLE_ENGINE, create.storage->engine->name); + /// If this is a TimeSeries table then we need to normalize list of columns (add missing columns and reorder), and also set inner table engines. + if (create.is_time_series_table && (mode < LoadingStrictnessLevel::ATTACH)) + StorageTimeSeries::normalizeTableDefinition(create, getContext()); + TableProperties properties; TableLockHolder as_storage_lock; @@ -1093,6 +1098,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const else if (as_create.storage) { storage_def = typeid_cast>(as_create.storage->ptr()); + create.is_time_series_table = as_create.is_time_series_table; } else { diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0615a2a1d62..866f5ba8c0a 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -105,9 +105,10 @@ ColumnsDescription SessionLogElement::getColumnsDescription() {"MySQL", static_cast(Interface::MYSQL)}, {"PostgreSQL", static_cast(Interface::POSTGRESQL)}, {"Local", static_cast(Interface::LOCAL)}, - {"TCP_Interserver", static_cast(Interface::TCP_INTERSERVER)} + {"TCP_Interserver", static_cast(Interface::TCP_INTERSERVER)}, + {"Prometheus", static_cast(Interface::PROMETHEUS)}, }); - static_assert(magic_enum::enum_count() == 7); + static_assert(magic_enum::enum_count() == 8); auto lc_string_datatype = std::make_shared(std::make_shared()); diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index ad55d16e284..f9afbc7b98d 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -27,6 +27,7 @@ class ASTQueryWithTableAndOutput; class ASTTableIdentifier; class Context; +// TODO(ilezhankin): refactor and merge |ASTTableIdentifier| struct StorageID { String database_name; diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 1a7c166c6a5..bee4b494958 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -214,6 +214,10 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { + /// Bool is not represented in which_type, so we need to type it separately + if (isInt64OrUInt64orBoolFieldType(src.getType()) && type.getName() == "Bool") + return bool(src.safeGet()); + if (which_type.isUInt8()) return convertNumericType(src, type); if (which_type.isUInt16()) return convertNumericType(src, type); if (which_type.isUInt32()) return convertNumericType(src, type); diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index f0f782c0a63..359e93ab269 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -483,6 +483,13 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (auto to_storage = getTargetInnerEngine(ViewTarget::To)) to_storage->formatImpl(settings, state, frame); + if (targets) + { + targets->formatTarget(ViewTarget::Data, settings, state, frame); + targets->formatTarget(ViewTarget::Tags, settings, state, frame); + targets->formatTarget(ViewTarget::Metrics, settings, state, frame); + } + if (dictionary) dictionary->formatImpl(settings, state, frame); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index a95010aea31..6be0fa78903 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -97,6 +97,7 @@ public: bool is_materialized_view{false}; bool is_live_view{false}; bool is_window_view{false}; + bool is_time_series_table{false}; /// CREATE TABLE ... ENGINE=TimeSeries() ... bool is_populate{false}; bool is_create_empty{false}; /// CREATE TABLE ... EMPTY AS SELECT ... bool replace_view{false}; /// CREATE OR REPLACE VIEW diff --git a/src/Parsers/ASTViewTargets.cpp b/src/Parsers/ASTViewTargets.cpp index 8ee98e704df..ffd746cc38a 100644 --- a/src/Parsers/ASTViewTargets.cpp +++ b/src/Parsers/ASTViewTargets.cpp @@ -21,6 +21,9 @@ std::string_view toString(ViewTarget::Kind kind) { case ViewTarget::To: return "to"; case ViewTarget::Inner: return "inner"; + case ViewTarget::Data: return "data"; + case ViewTarget::Tags: return "tags"; + case ViewTarget::Metrics: return "metrics"; } throw Exception(ErrorCodes::LOGICAL_ERROR, "{} doesn't support kind {}", __FUNCTION__, kind); } @@ -254,6 +257,9 @@ std::optional ASTViewTargets::getKeywordForTableID(ViewTarget::Kind kin { case ViewTarget::To: return Keyword::TO; /// TO mydb.mydata case ViewTarget::Inner: return std::nullopt; + case ViewTarget::Data: return Keyword::DATA; /// DATA mydb.mydata + case ViewTarget::Tags: return Keyword::TAGS; /// TAGS mydb.mytags + case ViewTarget::Metrics: return Keyword::METRICS; /// METRICS mydb.mymetrics } UNREACHABLE(); } @@ -264,6 +270,9 @@ std::optional ASTViewTargets::getKeywordForInnerStorage(ViewTarget::Kin { case ViewTarget::To: return std::nullopt; /// ENGINE = MergeTree() case ViewTarget::Inner: return Keyword::INNER; /// INNER ENGINE = MergeTree() + case ViewTarget::Data: return Keyword::DATA; /// DATA ENGINE = MergeTree() + case ViewTarget::Tags: return Keyword::TAGS; /// TAGS ENGINE = MergeTree() + case ViewTarget::Metrics: return Keyword::METRICS; /// METRICS ENGINE = MergeTree() } UNREACHABLE(); } @@ -274,6 +283,9 @@ std::optional ASTViewTargets::getKeywordForInnerUUID(ViewTarget::Kind k { case ViewTarget::To: return Keyword::TO_INNER_UUID; /// TO INNER UUID 'XXX' case ViewTarget::Inner: return std::nullopt; + case ViewTarget::Data: return Keyword::DATA_INNER_UUID; /// DATA INNER UUID 'XXX' + case ViewTarget::Tags: return Keyword::TAGS_INNER_UUID; /// TAGS INNER UUID 'XXX' + case ViewTarget::Metrics: return Keyword::METRICS_INNER_UUID; /// METRICS INNER UUID 'XXX' } UNREACHABLE(); } diff --git a/src/Parsers/ASTViewTargets.h b/src/Parsers/ASTViewTargets.h index 12182919f0e..7814dd5249c 100644 --- a/src/Parsers/ASTViewTargets.h +++ b/src/Parsers/ASTViewTargets.h @@ -9,7 +9,7 @@ namespace DB class ASTStorage; enum class Keyword : size_t; -/// Information about target tables (external or inner) of a materialized view or a window view. +/// Information about target tables (external or inner) of a materialized view or a window view or a TimeSeries table. /// See ASTViewTargets for more details. struct ViewTarget { @@ -24,6 +24,15 @@ struct ViewTarget /// If `kind == ViewTarget::Inner` then `ViewTarget` contains information about the "INNER" table of a window view: /// CREATE WINDOW VIEW db.wv_name {INNER ENGINE inner_engine} AS SELECT ... Inner, + + /// The "data" table for a TimeSeries table, contains time series. + Data, + + /// The "tags" table for a TimeSeries table, contains identifiers for each combination of a metric name and tags (labels). + Tags, + + /// The "metrics" table for a TimeSeries table, contains general information (metadata) about metrics. + Metrics, }; Kind kind = To; diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 0ae9ee4833c..34df4b135bb 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -116,6 +116,8 @@ namespace DB MR_MACROS(CURRENT_TRANSACTION, "CURRENT TRANSACTION") \ MR_MACROS(CURRENTUSER, "CURRENTUSER") \ MR_MACROS(D, "D") \ + MR_MACROS(DATA, "DATA") \ + MR_MACROS(DATA_INNER_UUID, "DATA INNER UUID") \ MR_MACROS(DATABASE, "DATABASE") \ MR_MACROS(DATABASES, "DATABASES") \ MR_MACROS(DATE, "DATE") \ @@ -288,6 +290,8 @@ namespace DB MR_MACROS(MCS, "MCS") \ MR_MACROS(MEMORY, "MEMORY") \ MR_MACROS(MERGES, "MERGES") \ + MR_MACROS(METRICS, "METRICS") \ + MR_MACROS(METRICS_INNER_UUID, "METRICS INNER UUID") \ MR_MACROS(MI, "MI") \ MR_MACROS(MICROSECOND, "MICROSECOND") \ MR_MACROS(MICROSECONDS, "MICROSECONDS") \ @@ -464,6 +468,8 @@ namespace DB MR_MACROS(TABLE_OVERRIDE, "TABLE OVERRIDE") \ MR_MACROS(TABLE, "TABLE") \ MR_MACROS(TABLES, "TABLES") \ + MR_MACROS(TAGS, "TAGS") \ + MR_MACROS(TAGS_INNER_UUID, "TAGS INNER UUID") \ MR_MACROS(TEMPORARY_TABLE, "TEMPORARY TABLE") \ MR_MACROS(TEMPORARY, "TEMPORARY") \ MR_MACROS(TEST, "TEST") \ diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index 4dfee67b537..fbdc6161408 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -45,6 +45,13 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// then MV will create inner table. We should generate UUID of inner table here. if (query.is_materialized_view) generate_target_uuid(ViewTarget::To); + + if (query.is_time_series_table) + { + generate_target_uuid(ViewTarget::Data); + generate_target_uuid(ViewTarget::Tags); + generate_target_uuid(ViewTarget::Metrics); + } } } } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index a592975613b..66965903ab0 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -696,6 +696,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ASTPtr table; ASTPtr columns_list; std::shared_ptr storage; + bool is_time_series_table = false; ASTPtr targets; ASTPtr as_database; ASTPtr as_table; @@ -784,6 +785,13 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; storage = typeid_cast>(ast); + + if (storage && storage->engine && (storage->engine->name == "TimeSeries")) + { + is_time_series_table = true; + ParserViewTargets({ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}).parse(pos, targets, expected); + } + return true; }; @@ -873,6 +881,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->create_or_replace = or_replace; query->if_not_exists = if_not_exists; query->temporary = is_temporary; + query->is_time_series_table = is_time_series_table; query->database = table_id->getDatabase(); query->table = table_id->getTable(); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 57457493844..43177fc73c0 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -237,7 +238,7 @@ public: if (function_node.isWindowFunction()) { buffer << " OVER ("; - buffer << calculateWindowNodeActionName(function_node.getWindowNode()); + buffer << calculateWindowNodeActionName(node, function_node.getWindowNode()); buffer << ')'; } @@ -298,21 +299,22 @@ public: return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); } - String calculateWindowNodeActionName(const QueryTreeNodePtr & node) + String calculateWindowNodeActionName(const QueryTreeNodePtr & function_nodew_node_, const QueryTreeNodePtr & window_node_) { - auto & window_node = node->as(); + const auto & function_node = function_nodew_node_->as(); + const auto & window_node = window_node_->as(); WriteBufferFromOwnString buffer; if (window_node.hasPartitionBy()) { buffer << "PARTITION BY "; - auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); + const auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); size_t partition_by_nodes_size = partition_by_nodes.size(); for (size_t i = 0; i < partition_by_nodes_size; ++i) { - auto & partition_by_node = partition_by_nodes[i]; + const auto & partition_by_node = partition_by_nodes[i]; buffer << calculateActionNodeName(partition_by_node); if (i + 1 != partition_by_nodes_size) buffer << ", "; @@ -326,7 +328,7 @@ public: buffer << "ORDER BY "; - auto & order_by_nodes = window_node.getOrderBy().getNodes(); + const auto & order_by_nodes = window_node.getOrderBy().getNodes(); size_t order_by_nodes_size = order_by_nodes.size(); for (size_t i = 0; i < order_by_nodes_size; ++i) @@ -364,44 +366,14 @@ public: } } - auto & window_frame = window_node.getWindowFrame(); - if (!window_frame.is_default) + auto window_frame_opt = extractWindowFrame(function_node); + if (window_frame_opt) { + auto & window_frame = *window_frame_opt; if (window_node.hasPartitionBy() || window_node.hasOrderBy()) buffer << ' '; - buffer << window_frame.type << " BETWEEN "; - if (window_frame.begin_type == WindowFrame::BoundaryType::Current) - { - buffer << "CURRENT ROW"; - } - else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) - { - buffer << "UNBOUNDED"; - buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); - } - else - { - buffer << calculateActionNodeName(window_node.getFrameBeginOffsetNode()); - buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); - } - - buffer << " AND "; - - if (window_frame.end_type == WindowFrame::BoundaryType::Current) - { - buffer << "CURRENT ROW"; - } - else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) - { - buffer << "UNBOUNDED"; - buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); - } - else - { - buffer << calculateActionNodeName(window_node.getFrameEndOffsetNode()); - buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); - } + window_frame.toString(buffer); } return buffer.str(); @@ -1056,20 +1028,11 @@ String calculateConstantActionNodeName(const Field & constant_literal) return ActionNodeNameHelper::calculateConstantActionNodeName(constant_literal); } -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, - const PlannerContext & planner_context, - QueryTreeNodeToName & node_to_name, - bool use_column_identifier_as_action_node_name) -{ - ActionNodeNameHelper helper(node_to_name, planner_context, use_column_identifier_as_action_node_name); - return helper.calculateWindowNodeActionName(node); -} - -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) +String calculateWindowNodeActionName(const QueryTreeNodePtr & function_node, const QueryTreeNodePtr & window_node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name) { QueryTreeNodeToName empty_map; ActionNodeNameHelper helper(empty_map, planner_context, use_column_identifier_as_action_node_name); - return helper.calculateWindowNodeActionName(node); + return helper.calculateWindowNodeActionName(function_node, window_node); } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 6bb32047327..4f608ad3f7b 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -8,6 +9,7 @@ #include #include +#include namespace DB { @@ -73,16 +75,8 @@ String calculateConstantActionNodeName(const Field & constant_literal); * Window node action name can only be part of window function action name. * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, - const PlannerContext & planner_context, - QueryTreeNodeToName & node_to_name, - bool use_column_identifier_as_action_node_name = true); - -/** Calculate action node name for window node. - * Window node action name can only be part of window function action name. - * For column node column node identifier from planner context is used, if use_column_identifier_as_action_node_name = true. - */ -String calculateWindowNodeActionName(const QueryTreeNodePtr & node, +String calculateWindowNodeActionName(const QueryTreeNodePtr & function_node, + const QueryTreeNodePtr & window_node, const PlannerContext & planner_context, bool use_column_identifier_as_action_node_name = true); diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 225852de5a7..f91cf644cf0 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -1,5 +1,7 @@ +#include #include +#include #include #include #include @@ -8,8 +10,9 @@ #include -#include #include +#include +#include namespace DB { @@ -22,27 +25,33 @@ namespace ErrorCodes namespace { -WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & func_node_, const PlannerContext & planner_context) { + const auto & func_node = func_node_->as(); + auto node = func_node.getWindowNode(); auto & window_node = node->as(); WindowDescription window_description; - window_description.window_name = calculateWindowNodeActionName(node, planner_context); + window_description.window_name = calculateWindowNodeActionName(func_node_, node, planner_context); for (const auto & partition_by_node : window_node.getPartitionBy().getNodes()) { auto partition_by_node_action_name = calculateActionNodeName(partition_by_node, planner_context); - auto partition_by_sort_column_description = SortColumnDescription(partition_by_node_action_name, 1 /* direction */, 1 /* nulls_direction */); + auto partition_by_sort_column_description + = SortColumnDescription(partition_by_node_action_name, 1 /* direction */, 1 /* nulls_direction */); window_description.partition_by.push_back(std::move(partition_by_sort_column_description)); } window_description.order_by = extractSortDescription(window_node.getOrderByNode(), planner_context); window_description.full_sort_description = window_description.partition_by; - window_description.full_sort_description.insert(window_description.full_sort_description.end(), window_description.order_by.begin(), window_description.order_by.end()); + window_description.full_sort_description.insert( + window_description.full_sort_description.end(), window_description.order_by.begin(), window_description.order_by.end()); /// WINDOW frame is validated during query analysis stage - window_description.frame = window_node.getWindowFrame(); + auto window_frame = extractWindowFrame(func_node); + window_description.frame = window_frame ? *window_frame : window_node.getWindowFrame(); + auto node_frame = window_node.getWindowFrame(); const auto & query_context = planner_context.getQueryContext(); const auto & query_context_settings = query_context->getSettingsRef(); @@ -64,7 +73,8 @@ WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr } -std::vector extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context) +std::vector +extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context) { std::unordered_map window_name_to_description; @@ -72,7 +82,7 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & { auto & window_function_node_typed = window_function_node->as(); - auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node_typed.getWindowNode(), planner_context); + auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node, planner_context); auto frame_type = function_window_description.frame.type; if (frame_type != WindowFrame::FrameType::ROWS && frame_type != WindowFrame::FrameType::RANGE) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index a6e94a124e6..822a3e9465e 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -22,6 +22,8 @@ #include +#include + #include #include #include @@ -34,6 +36,7 @@ #include #include #include +#include #include @@ -507,4 +510,20 @@ void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets) } } +std::optional extractWindowFrame(const FunctionNode & node) +{ + if (!node.isWindowFunction()) + return {}; + auto & window_node = node.getWindowNode()->as(); + const auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + return window_frame; + auto aggregate_function = node.getAggregateFunction(); + if (const auto * win_func = dynamic_cast(aggregate_function.get())) + { + return win_func->getDefaultFrame(); + } + return {}; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index ae60976a8d6..254b8f4eae1 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -19,6 +19,8 @@ #include +#include + namespace DB { @@ -91,4 +93,9 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings); using UsefulSets = std::unordered_set; void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets); +/// If the window frame is not set in sql, try to use the default frame from window function +/// if it have any one. Otherwise return empty. +/// If the window frame is set in sql, use it anyway. +std::optional extractWindowFrame(const FunctionNode & node); + } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 8de248a9c95..c26cd7cc8c3 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -65,28 +65,6 @@ namespace ErrorCodes extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } -// Interface for true window functions. It's not much of an interface, they just -// accept the guts of WindowTransform and do 'something'. Given a small number of -// true window functions, and the fact that the WindowTransform internals are -// pretty much well-defined in domain terms (e.g. frame boundaries), this is -// somewhat acceptable. -class IWindowFunction -{ -public: - virtual ~IWindowFunction() = default; - - // Must insert the result for current_row. - virtual void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) const = 0; - - virtual std::optional getDefaultFrame() const { return {}; } - - virtual ColumnPtr castColumn(const Columns &, const std::vector &) { return nullptr; } - - /// Is the frame type supported by this function. - virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } -}; - // Compares ORDER BY column values at given rows to find the boundaries of frame: // [compared] with [reference] +/- offset. Return value is -1/0/+1, like in // sorting predicates -- -1 means [compared] is less than [reference] +/- offset. @@ -1523,41 +1501,6 @@ void WindowTransform::work() } } -// A basic implementation for a true window function. It pretends to be an -// aggregate function, but refuses to work as such. -struct WindowFunction - : public IAggregateFunctionHelper - , public IWindowFunction -{ - std::string name; - - WindowFunction(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) - : IAggregateFunctionHelper(argument_types_, parameters_, result_type_) - , name(name_) - {} - - bool isOnlyWindowFunction() const override { return true; } - - [[noreturn]] void fail() const - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The function '{}' can only be used as a window function, not as an aggregate function", - getName()); - } - - String getName() const override { return name; } - void create(AggregateDataPtr __restrict) const override {} - void destroy(AggregateDataPtr __restrict) const noexcept override {} - bool hasTrivialDestructor() const override { return true; } - size_t sizeOfData() const override { return 0; } - size_t alignOfData() const override { return 1; } - void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { fail(); } - void merge(AggregateDataPtr __restrict, ConstAggregateDataPtr, Arena *) const override { fail(); } - void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { fail(); } - void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional, Arena *) const override { fail(); } - void insertResultInto(AggregateDataPtr __restrict, IColumn &, Arena *) const override { fail(); } -}; - struct WindowFunctionRank final : public WindowFunction { WindowFunctionRank(const std::string & name_, @@ -1669,36 +1612,6 @@ struct WindowFunctionHelpers } }; -template -struct StatefulWindowFunction : public WindowFunction -{ - StatefulWindowFunction(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) - : WindowFunction(name_, argument_types_, parameters_, result_type_) - { - } - - size_t sizeOfData() const override { return sizeof(State); } - size_t alignOfData() const override { return 1; } - - void create(AggregateDataPtr __restrict place) const override - { - new (place) State(); - } - - void destroy(AggregateDataPtr __restrict place) const noexcept override - { - reinterpret_cast(place)->~State(); - } - - bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v; } - - State & getState(const WindowFunctionWorkspace & workspace) const - { - return *reinterpret_cast(workspace.aggregate_function_state.data()); - } -}; - struct ExponentialTimeDecayedSumState { Float64 previous_time; @@ -2278,14 +2191,13 @@ public: bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame.type != WindowFrame::FrameType::RANGE - || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded - || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Current) - { - LOG_ERROR( - getLogger("WindowFunctionPercentRank"), - "Window frame for function 'percent_rank' should be 'RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT'"); - return false; + auto default_window_frame = getDefaultFrame(); + if (transform->window_description.frame != default_window_frame) + { + LOG_ERROR( + getLogger("WindowFunctionPercentRank"), + "Window frame for function 'percent_rank' should be '{}'", default_window_frame->toString()); + return false; } return true; } @@ -2295,7 +2207,7 @@ public: WindowFrame frame; frame.type = WindowFrame::FrameType::RANGE; frame.begin_type = WindowFrame::BoundaryType::Unbounded; - frame.end_type = WindowFrame::BoundaryType::Current; + frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } @@ -2860,5 +2772,4 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) name, argument_types, parameters); }, properties}); } - } diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index fe4f79e997c..cb672ad6841 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -21,30 +22,6 @@ using ExpressionActionsPtr = std::shared_ptr; class Arena; -class IWindowFunction; - -// Runtime data for computing one window function. -struct WindowFunctionWorkspace -{ - AggregateFunctionPtr aggregate_function; - - // Cached value of aggregate function isState virtual method - bool is_aggregate_function_state = false; - - // This field is set for pure window functions. When set, we ignore the - // window_function.aggregate_function, and work through this interface - // instead. - IWindowFunction * window_function_impl = nullptr; - - std::vector argument_column_indices; - - // Will not be initialized for a pure window function. - mutable AlignedBuffer aggregate_function_state; - - // Argument columns. Be careful, this is a per-block cache. - std::vector argument_columns; - UInt64 cached_block_number = std::numeric_limits::max(); -}; struct WindowTransformBlock { diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index 047db014560..39e066005b9 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -97,6 +98,21 @@ void HTTPServerConnection::run() { sendErrorResponse(session, Poco::Net::HTTPResponse::HTTP_BAD_REQUEST); } + catch (const Poco::Net::NetException & e) + { + /// Do not spam logs with messages related to connection reset by peer. + if (e.code() == POCO_ENOTCONN) + { + LOG_DEBUG(LogFrequencyLimiter(getLogger("HTTPServerConnection"), 10), "Connection reset by peer while processing HTTP request: {}", e.message()); + break; + } + + if (session.networkException()) + session.networkException()->rethrow(); + else + throw; + } + catch (const Poco::Exception &) { if (session.networkException()) diff --git a/src/Server/HTTP/checkHTTPHeader.cpp b/src/Server/HTTP/checkHTTPHeader.cpp new file mode 100644 index 00000000000..812adde022a --- /dev/null +++ b/src/Server/HTTP/checkHTTPHeader.cpp @@ -0,0 +1,22 @@ +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNEXPECTED_HTTP_HEADERS; +} + +void checkHTTPHeader(const HTTPRequest & request, const String & header_name, const String & expected_value) +{ + if (!request.has(header_name)) + throw Exception(ErrorCodes::UNEXPECTED_HTTP_HEADERS, "No HTTP header {}", header_name); + if (request.get(header_name) != expected_value) + throw Exception(ErrorCodes::UNEXPECTED_HTTP_HEADERS, "HTTP header {} has unexpected value '{}' (instead of '{}')", header_name, request.get(header_name), expected_value); +} + +} diff --git a/src/Server/HTTP/checkHTTPHeader.h b/src/Server/HTTP/checkHTTPHeader.h new file mode 100644 index 00000000000..956599ae66b --- /dev/null +++ b/src/Server/HTTP/checkHTTPHeader.h @@ -0,0 +1,13 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Checks that the HTTP request has a specified header with a specified value. +void checkHTTPHeader(const HTTPRequest & request, const String & header_name, const String & expected_value); + +} diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.cpp b/src/Server/HTTP/sendExceptionToHTTPClient.cpp index 022a763a9a2..321c4866e8f 100644 --- a/src/Server/HTTP/sendExceptionToHTTPClient.cpp +++ b/src/Server/HTTP/sendExceptionToHTTPClient.cpp @@ -43,7 +43,6 @@ void sendExceptionToHTTPClient( out->position() = out->buffer().begin(); out->writeln(exception_message); - out->finalize(); } } diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 5344b2d024b..2e7f16f59d7 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -1,18 +1,16 @@ -#include #include #include +#include +#include #include -#include #include #include "HTTPHandler.h" -#include "Server/PrometheusMetricsWriter.h" #include "StaticRequestHandler.h" #include "ReplicasStatusHandler.h" #include "InterserverIOHTTPHandler.h" -#include "PrometheusRequestHandler.h" #include "WebUIRequestHandler.h" @@ -124,7 +122,7 @@ static inline auto createHandlersFactoryFromConfig( } else if (handler_type == "prometheus") { - main_handler_factory->addHandler(createPrometheusHandlerFactory(server, config, async_metrics, prefix + "." + key)); + main_handler_factory->addHandler(createPrometheusHandlerFactoryForHTTPRule(server, config, prefix + "." + key, async_metrics)); } else if (handler_type == "replicas_status") { @@ -201,10 +199,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); else if (name == "PrometheusHandler-factory") - { - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); - } + return createPrometheusHandlerFactory(server, config, async_metrics, name); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); } @@ -291,20 +286,9 @@ void addDefaultHandlersFactory( ); factory.addHandler(query_handler); - /// We check that prometheus handler will be served on current (default) port. - /// Otherwise it will be created separately, see createHandlerFactory(...). - if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) - { - auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; - auto prometheus_handler = std::make_shared>(std::move(creator)); - prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - prometheus_handler->allowGetAndHeadRequest(); + /// createPrometheusHandlerFactoryForHTTPRuleDefaults() can return nullptr if prometheus protocols must not be served on http port. + if (auto prometheus_handler = createPrometheusHandlerFactoryForHTTPRuleDefaults(server, config, async_metrics)) factory.addHandler(prometheus_handler); - } } } diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index b4c32366463..db4bb73cbc4 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -1,15 +1,12 @@ #pragma once -#include -#include #include #include #include #include -#include - #include + namespace DB { @@ -19,6 +16,7 @@ namespace ErrorCodes } class IServer; +class AsynchronousMetrics; template class HandlingRuleHTTPHandlerFactory : public HTTPRequestHandlerFactory @@ -126,18 +124,6 @@ HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); -HTTPRequestHandlerFactoryPtr -createPrometheusHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - AsynchronousMetrics & async_metrics, - const std::string & config_prefix); - -HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, - const Poco::Util::AbstractConfiguration & config, - PrometheusMetricsWriterPtr metrics_writer, - const std::string & name); - /// @param server - used in handlers to check IServer::isCancelled() /// @param config - not the same as server.config(), since it can be newer /// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true) diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index 85eafbe4808..43370116015 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -1,13 +1,27 @@ #include "PrometheusMetricsWriter.h" -#include +#include +#include #include #include - -#include +#include #include "config.h" + +#if USE_NURAFT +namespace ProfileEvents +{ + extern const std::vector keeper_profile_events; +} + +namespace CurrentMetrics +{ + extern const std::vector keeper_metrics; +} +#endif + + namespace { @@ -107,100 +121,84 @@ void writeAsyncMetrics(DB::WriteBuffer & wb, const DB::AsynchronousMetricValues } -#if USE_NURAFT -namespace ProfileEvents -{ - extern const std::vector keeper_profile_events; -} - -namespace CurrentMetrics -{ - extern const std::vector keeper_metrics; -} -#endif - namespace DB { -PrometheusMetricsWriter::PrometheusMetricsWriter( - const Poco::Util::AbstractConfiguration & config, const std::string & config_name, - const AsynchronousMetrics & async_metrics_) - : async_metrics(async_metrics_) - , send_events(config.getBool(config_name + ".events", true)) - , send_metrics(config.getBool(config_name + ".metrics", true)) - , send_asynchronous_metrics(config.getBool(config_name + ".asynchronous_metrics", true)) - , send_errors(config.getBool(config_name + ".errors", true)) +void PrometheusMetricsWriter::writeEvents(WriteBuffer & wb) const { + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) + writeEvent(wb, i); } -void PrometheusMetricsWriter::write(WriteBuffer & wb) const +void PrometheusMetricsWriter::writeMetrics(WriteBuffer & wb) const { - if (send_events) + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + writeMetric(wb, i); +} + +void PrometheusMetricsWriter::writeAsynchronousMetrics(WriteBuffer & wb, const AsynchronousMetrics & async_metrics) const +{ + writeAsyncMetrics(wb, async_metrics.getValues()); +} + +void PrometheusMetricsWriter::writeErrors(WriteBuffer & wb) const +{ + size_t total_count = 0; + + for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) { - for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) - writeEvent(wb, i); - } + const auto & error = ErrorCodes::values[i].get(); + std::string_view name = ErrorCodes::getName(static_cast(i)); - if (send_metrics) - { - for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) - writeMetric(wb, i); - } + if (name.empty()) + continue; - if (send_asynchronous_metrics) - writeAsyncMetrics(wb, async_metrics.getValues()); + std::string key{error_metrics_prefix + toString(name)}; + std::string help = fmt::format("The number of {} errors since last server restart", name); - if (send_errors) - { - size_t total_count = 0; - - for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i) - { - const auto & error = ErrorCodes::values[i].get(); - std::string_view name = ErrorCodes::getName(static_cast(i)); - - if (name.empty()) - continue; - - std::string key{error_metrics_prefix + toString(name)}; - std::string help = fmt::format("The number of {} errors since last server restart", name); - - writeOutLine(wb, "# HELP", key, help); - writeOutLine(wb, "# TYPE", key, "counter"); - /// We are interested in errors which are happened only on this server. - writeOutLine(wb, key, error.local.count); - - total_count += error.local.count; - } - - /// Write the total number of errors as a separate metric - std::string key{error_metrics_prefix + toString("ALL")}; - writeOutLine(wb, "# HELP", key, "The total number of errors since last server restart"); + writeOutLine(wb, "# HELP", key, help); writeOutLine(wb, "# TYPE", key, "counter"); - writeOutLine(wb, key, total_count); + /// We are interested in errors which are happened only on this server. + writeOutLine(wb, key, error.local.count); + + total_count += error.local.count; } + /// Write the total number of errors as a separate metric + std::string key{error_metrics_prefix + toString("ALL")}; + writeOutLine(wb, "# HELP", key, "The total number of errors since last server restart"); + writeOutLine(wb, "# TYPE", key, "counter"); + writeOutLine(wb, key, total_count); } -void KeeperPrometheusMetricsWriter::write([[maybe_unused]] WriteBuffer & wb) const + +void KeeperPrometheusMetricsWriter::writeEvents([[maybe_unused]] WriteBuffer & wb) const { #if USE_NURAFT - if (send_events) - { - for (auto event : ProfileEvents::keeper_profile_events) - writeEvent(wb, event); - } - - if (send_metrics) - { - for (auto metric : CurrentMetrics::keeper_metrics) - writeMetric(wb, metric); - } - - if (send_asynchronous_metrics) - writeAsyncMetrics(wb, async_metrics.getValues()); + for (auto event : ProfileEvents::keeper_profile_events) + writeEvent(wb, event); #endif } +void KeeperPrometheusMetricsWriter::writeMetrics([[maybe_unused]] WriteBuffer & wb) const +{ +#if USE_NURAFT + for (auto metric : CurrentMetrics::keeper_metrics) + writeMetric(wb, metric); +#endif +} + +void KeeperPrometheusMetricsWriter::writeAsynchronousMetrics([[maybe_unused]] WriteBuffer & wb, + [[maybe_unused]] const AsynchronousMetrics & async_metrics) const +{ +#if USE_NURAFT + writeAsyncMetrics(wb, async_metrics.getValues()); +#endif +} + +void KeeperPrometheusMetricsWriter::writeErrors(WriteBuffer &) const +{ +} + } diff --git a/src/Server/PrometheusMetricsWriter.h b/src/Server/PrometheusMetricsWriter.h index 933ad909ee0..cf2587d80b8 100644 --- a/src/Server/PrometheusMetricsWriter.h +++ b/src/Server/PrometheusMetricsWriter.h @@ -1,44 +1,33 @@ #pragma once -#include - -#include -#include -#include - -#include +#include namespace DB { +class AsynchronousMetrics; +class WriteBuffer; /// Write metrics in Prometheus format class PrometheusMetricsWriter { public: - PrometheusMetricsWriter( - const Poco::Util::AbstractConfiguration & config, const std::string & config_name, - const AsynchronousMetrics & async_metrics_); - - virtual void write(WriteBuffer & wb) const; - virtual ~PrometheusMetricsWriter() = default; -protected: - const AsynchronousMetrics & async_metrics; - const bool send_events; - const bool send_metrics; - const bool send_asynchronous_metrics; - const bool send_errors; + virtual void writeMetrics(WriteBuffer & wb) const; + virtual void writeAsynchronousMetrics(WriteBuffer & wb, const AsynchronousMetrics & async_metrics) const; + virtual void writeEvents(WriteBuffer & wb) const; + virtual void writeErrors(WriteBuffer & wb) const; }; + class KeeperPrometheusMetricsWriter : public PrometheusMetricsWriter { - using PrometheusMetricsWriter::PrometheusMetricsWriter; - - void write(WriteBuffer & wb) const override; +public: + void writeMetrics(WriteBuffer & wb) const override; + void writeAsynchronousMetrics(WriteBuffer & wb, const AsynchronousMetrics & async_metrics) const override; + void writeEvents(WriteBuffer & wb) const override; + void writeErrors(WriteBuffer & wb) const override; }; -using PrometheusMetricsWriterPtr = std::shared_ptr; - } diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 87c106c3fc0..bbd9a978c91 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -1,74 +1,447 @@ #include +#include +#include #include #include -#include +#include #include -#include -#include -#include -#include "Server/PrometheusMetricsWriter.h" +#include +#include "config.h" -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { -void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) + +namespace ErrorCodes { + extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; + extern const int LOGICAL_ERROR; +} + +/// Base implementation of a prometheus protocol. +class PrometheusRequestHandler::Impl +{ +public: + explicit Impl(PrometheusRequestHandler & parent) : parent_ref(parent) {} + virtual ~Impl() = default; + virtual void beforeHandlingRequest(HTTPServerRequest & /* request */) {} + virtual void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) = 0; + virtual void onException() {} + +protected: + PrometheusRequestHandler & parent() { return parent_ref; } + IServer & server() { return parent().server; } + const PrometheusRequestHandlerConfig & config() { return parent().config; } + PrometheusMetricsWriter & metrics_writer() { return *parent().metrics_writer; } + LoggerPtr log() { return parent().log; } + WriteBuffer & getOutputStream(HTTPServerResponse & response) { return parent().getOutputStream(response); } + +private: + PrometheusRequestHandler & parent_ref; +}; + + +/// Implementation of the exposing metrics protocol. +class PrometheusRequestHandler::ExposeMetricsImpl : public Impl +{ +public: + explicit ExposeMetricsImpl(PrometheusRequestHandler & parent) : Impl(parent) {} + + void beforeHandlingRequest(HTTPServerRequest & request) override + { + LOG_INFO(log(), "Handling metrics request from {}", request.get("User-Agent")); + chassert(config().type == PrometheusRequestHandlerConfig::Type::ExposeMetrics); + } + + void handleRequest(HTTPServerRequest & /* request */, HTTPServerResponse & response) override + { + response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); + auto & out = getOutputStream(response); + + if (config().expose_events) + metrics_writer().writeEvents(out); + + if (config().expose_metrics) + metrics_writer().writeMetrics(out); + + if (config().expose_asynchronous_metrics) + metrics_writer().writeAsynchronousMetrics(out, parent().async_metrics); + + if (config().expose_errors) + metrics_writer().writeErrors(out); + } +}; + + +/// Base implementation of a protocol with Context and authentication. +class PrometheusRequestHandler::ImplWithContext : public Impl +{ +public: + explicit ImplWithContext(PrometheusRequestHandler & parent) : Impl(parent), default_settings(parent.server.context()->getSettingsRef()) { } + + virtual void handlingRequestWithContext(HTTPServerRequest & request, HTTPServerResponse & response) = 0; + +protected: + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override + { + SCOPE_EXIT({ + request_credentials.reset(); + context.reset(); + session.reset(); + params.reset(); + }); + + params = std::make_unique(default_settings, request); + parent().send_stacktrace = config().is_stacktrace_enabled && params->getParsed("stacktrace", false); + + if (!authenticateUserAndMakeContext(request, response)) + return; /// The user is not authenticated yet, and the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header, + /// and `request_credentials` must be preserved until the next request or until any exception. + + /// Initialize query scope. + std::optional query_scope; + if (context) + query_scope.emplace(context); + + handlingRequestWithContext(request, response); + } + + bool authenticateUserAndMakeContext(HTTPServerRequest & request, HTTPServerResponse & response) + { + session = std::make_unique(server().context(), ClientInfo::Interface::PROMETHEUS, request.isSecure()); + + if (!authenticateUser(request, response)) + return false; + + makeContext(request); + return true; + } + + bool authenticateUser(HTTPServerRequest & request, HTTPServerResponse & response) + { + return authenticateUserByHTTP(request, *params, response, *session, request_credentials, server().context(), log()); + } + + void makeContext(HTTPServerRequest & request) + { + context = session->makeQueryContext(); + + /// Anything else beside HTTP POST should be readonly queries. + setReadOnlyIfHTTPMethodIdempotent(context, request.getMethod()); + + auto roles = params->getAll("role"); + if (!roles.empty()) + context->setCurrentRoles(roles); + + /// Settings can be overridden in the URL query. + auto is_setting_like_parameter = [&] (const String & name) + { + /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. + if (name.empty()) + return false; + + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + static const NameSet reserved_param_names{"user", "password", "quota_key", "stacktrace", "role", "query_id"}; + return !reserved_param_names.contains(name); + }; + + SettingsChanges settings_changes; + for (const auto & [key, value] : *params) + { + if (is_setting_like_parameter(key)) + { + /// This query parameter should be considered as a ClickHouse setting. + settings_changes.push_back({key, value}); + } + } + + context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); + context->applySettingsChanges(settings_changes); + + /// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields. + context->setCurrentQueryId(params->get("query_id", request.get("X-ClickHouse-Query-Id", ""))); + } + + void onException() override + { + // So that the next requests on the connection have to always start afresh in case of exceptions. + request_credentials.reset(); + } + + const Settings & default_settings; + std::unique_ptr params; + std::unique_ptr session; + std::unique_ptr request_credentials; + ContextMutablePtr context; +}; + + +/// Implementation of the remote-write protocol. +class PrometheusRequestHandler::RemoteWriteImpl : public ImplWithContext +{ +public: + using ImplWithContext::ImplWithContext; + + void beforeHandlingRequest(HTTPServerRequest & request) override + { + LOG_INFO(log(), "Handling remote write request from {}", request.get("User-Agent", "")); + chassert(config().type == PrometheusRequestHandlerConfig::Type::RemoteWrite); + } + + void handlingRequestWithContext([[maybe_unused]] HTTPServerRequest & request, [[maybe_unused]] HTTPServerResponse & response) override + { +#if USE_PROMETHEUS_PROTOBUFS + checkHTTPHeader(request, "Content-Type", "application/x-protobuf"); + checkHTTPHeader(request, "Content-Encoding", "snappy"); + + ProtobufZeroCopyInputStreamFromReadBuffer zero_copy_input_stream{ + std::make_unique(wrapReadBufferReference(request.getStream()))}; + + prometheus::WriteRequest write_request; + if (!write_request.ParsePartialFromZeroCopyStream(&zero_copy_input_stream)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse WriteRequest"); + + auto table = DatabaseCatalog::instance().getTable(StorageID{config().time_series_table_name}, context); + PrometheusRemoteWriteProtocol protocol{table, context}; + + if (write_request.timeseries_size()) + protocol.writeTimeSeries(write_request.timeseries()); + + if (write_request.metadata_size()) + protocol.writeMetricsMetadata(write_request.metadata()); + + response.setContentType("text/plain; charset=UTF-8"); + response.send(); + +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Prometheus remote write protocol is disabled"); +#endif + } +}; + +/// Implementation of the remote-read protocol. +class PrometheusRequestHandler::RemoteReadImpl : public ImplWithContext +{ +public: + using ImplWithContext::ImplWithContext; + + void beforeHandlingRequest(HTTPServerRequest & request) override + { + LOG_INFO(log(), "Handling remote read request from {}", request.get("User-Agent", "")); + chassert(config().type == PrometheusRequestHandlerConfig::Type::RemoteRead); + } + + void handlingRequestWithContext([[maybe_unused]] HTTPServerRequest & request, [[maybe_unused]] HTTPServerResponse & response) override + { +#if USE_PROMETHEUS_PROTOBUFS + checkHTTPHeader(request, "Content-Type", "application/x-protobuf"); + checkHTTPHeader(request, "Content-Encoding", "snappy"); + + auto table = DatabaseCatalog::instance().getTable(StorageID{config().time_series_table_name}, context); + PrometheusRemoteReadProtocol protocol{table, context}; + + ProtobufZeroCopyInputStreamFromReadBuffer zero_copy_input_stream{ + std::make_unique(wrapReadBufferReference(request.getStream()))}; + + prometheus::ReadRequest read_request; + if (!read_request.ParseFromZeroCopyStream(&zero_copy_input_stream)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse ReadRequest"); + + prometheus::ReadResponse read_response; + + size_t num_queries = read_request.queries_size(); + for (size_t i = 0; i != num_queries; ++i) + { + const auto & query = read_request.queries(static_cast(i)); + auto & new_query_result = *read_response.add_results(); + protocol.readTimeSeries( + *new_query_result.mutable_timeseries(), + query.start_timestamp_ms(), + query.end_timestamp_ms(), + query.matchers(), + query.hints()); + } + +# if 0 + LOG_DEBUG(log, "ReadResponse = {}", read_response.DebugString()); +# endif + + response.setContentType("application/x-protobuf"); + response.set("Content-Encoding", "snappy"); + + ProtobufZeroCopyOutputStreamFromWriteBuffer zero_copy_output_stream{std::make_unique(getOutputStream(response))}; + read_response.SerializeToZeroCopyStream(&zero_copy_output_stream); + zero_copy_output_stream.finalize(); + +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Prometheus remote read protocol is disabled"); +#endif + } +}; + + +PrometheusRequestHandler::PrometheusRequestHandler( + IServer & server_, + const PrometheusRequestHandlerConfig & config_, + const AsynchronousMetrics & async_metrics_, + std::shared_ptr metrics_writer_) + : server(server_) + , config(config_) + , async_metrics(async_metrics_) + , metrics_writer(metrics_writer_) + , log(getLogger("PrometheusRequestHandler")) +{ + createImpl(); +} + +PrometheusRequestHandler::~PrometheusRequestHandler() = default; + +void PrometheusRequestHandler::createImpl() +{ + switch (config.type) + { + case PrometheusRequestHandlerConfig::Type::ExposeMetrics: + { + impl = std::make_unique(*this); + return; + } + case PrometheusRequestHandlerConfig::Type::RemoteWrite: + { + impl = std::make_unique(*this); + return; + } + case PrometheusRequestHandlerConfig::Type::RemoteRead: + { + impl = std::make_unique(*this); + return; + } + } + UNREACHABLE(); +} + +void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) +{ + setThreadName("PrometheusHndlr"); + try { - const auto & config = server.config(); - unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); + response_finalized = false; + write_event = write_event_; + http_method = request.getMethod(); + chassert(!write_buffer_from_response); /// Nothing is written to the response yet. - /// In order to make keep-alive works. + /// Make keep-alive works. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response, config.keep_alive_timeout); - response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); + impl->beforeHandlingRequest(request); + impl->handleRequest(request, response); - WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); - metrics_writer->write(wb); - wb.finalize(); + finalizeResponse(response); } catch (...) { - tryLogCurrentException("PrometheusRequestHandler"); + tryLogCurrentException(log); + + ExecutionStatus status = ExecutionStatus::fromCurrentException("", send_stacktrace); + trySendExceptionToClient(status.message, status.code, request, response); + tryFinalizeResponse(response); + + tryCallOnException(); } } -HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( - IServer & server, - const Poco::Util::AbstractConfiguration & config, - AsynchronousMetrics & async_metrics, - const std::string & config_prefix) +WriteBufferFromHTTPServerResponse & PrometheusRequestHandler::getOutputStream(HTTPServerResponse & response) { - auto writer = std::make_shared(config, config_prefix + ".handler", async_metrics); - auto creator = [&server, writer]() -> std::unique_ptr - { - return std::make_unique(server, writer); - }; - - auto factory = std::make_shared>(std::move(creator)); - factory->addFiltersFromConfig(config, config_prefix); - return factory; + if (response_finalized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "PrometheusRequestHandler: Response already sent"); + if (write_buffer_from_response) + return *write_buffer_from_response; + write_buffer_from_response = std::make_unique( + response, http_method == HTTPRequest::HTTP_HEAD, config.keep_alive_timeout, write_event); + return *write_buffer_from_response; } -HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name) +void PrometheusRequestHandler::finalizeResponse(HTTPServerResponse & response) { - auto factory = std::make_shared(name); - auto creator = [&server, metrics_writer] + if (response_finalized) { - return std::make_unique(server, metrics_writer); - }; + /// Response is already finalized or at least tried to. We don't need the write buffer anymore in either case. + write_buffer_from_response = nullptr; + } + else + { + /// We set `response_finalized = true` before actually calling `write_buffer_from_response->finalize()` + /// because we shouldn't call finalize() again even if finalize() throws an exception. + response_finalized = true; - auto handler = std::make_shared>(std::move(creator)); - handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - handler->allowGetAndHeadRequest(); - factory->addHandler(handler); - return factory; + if (write_buffer_from_response) + std::exchange(write_buffer_from_response, {})->finalize(); + else + WriteBufferFromHTTPServerResponse{response, http_method == HTTPRequest::HTTP_HEAD, config.keep_alive_timeout, write_event}.finalize(); + } + chassert(response_finalized && !write_buffer_from_response); } + +void PrometheusRequestHandler::trySendExceptionToClient(const String & exception_message, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response) +{ + if (response_finalized) + return; /// Response is already finalized (or tried to). We can't write the error message to the response in either case. + + try + { + sendExceptionToHTTPClient(exception_message, exception_code, request, response, &getOutputStream(response), log); + } + catch (...) + { + tryLogCurrentException(log, "Couldn't send exception to client"); + } +} + +void PrometheusRequestHandler::tryFinalizeResponse(HTTPServerResponse & response) +{ + try + { + finalizeResponse(response); + } + catch (...) + { + tryLogCurrentException(log, "Cannot flush data to client (after sending exception)"); + } +} + +void PrometheusRequestHandler::tryCallOnException() +{ + try + { + if (impl) + impl->onException(); + } + catch (...) + { + tryLogCurrentException(log, "onException"); + } +} + } diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index d120752c8c5..3a2dac0b7ad 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -1,28 +1,61 @@ #pragma once #include +#include -#include "PrometheusMetricsWriter.h" namespace DB { - +class AsynchronousMetrics; class IServer; +class PrometheusMetricsWriter; +class WriteBufferFromHTTPServerResponse; +/// Handles requests for prometheus protocols (expose_metrics, remote_write, remote-read). class PrometheusRequestHandler : public HTTPRequestHandler { -private: - IServer & server; - PrometheusMetricsWriterPtr metrics_writer; - public: - PrometheusRequestHandler(IServer & server_, PrometheusMetricsWriterPtr metrics_writer_) - : server(server_) - , metrics_writer(std::move(metrics_writer_)) - { - } + PrometheusRequestHandler(IServer & server_, const PrometheusRequestHandlerConfig & config_, + const AsynchronousMetrics & async_metrics_, std::shared_ptr metrics_writer_); + ~PrometheusRequestHandler() override; - void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) override; + +private: + /// Creates an internal implementation based on which PrometheusRequestHandlerConfig::Type is used. + void createImpl(); + + /// Returns the write buffer used for the current HTTP response. + WriteBufferFromHTTPServerResponse & getOutputStream(HTTPServerResponse & response); + + /// Finalizes the output stream and sends the response to the client. + void finalizeResponse(HTTPServerResponse & response); + void tryFinalizeResponse(HTTPServerResponse & response); + + /// Writes the current exception to the response. + void trySendExceptionToClient(const String & exception_message, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response); + + /// Calls onException() in a try-catch block. + void tryCallOnException(); + + IServer & server; + const PrometheusRequestHandlerConfig config; + const AsynchronousMetrics & async_metrics; + const std::shared_ptr metrics_writer; + const LoggerPtr log; + + class Impl; + class ImplWithContext; + class ExposeMetricsImpl; + class RemoteWriteImpl; + class RemoteReadImpl; + std::unique_ptr impl; + + String http_method; + bool send_stacktrace = false; + std::unique_ptr write_buffer_from_response; + bool response_finalized = false; + ProfileEvents::Event write_event; }; } diff --git a/src/Server/PrometheusRequestHandlerConfig.h b/src/Server/PrometheusRequestHandlerConfig.h new file mode 100644 index 00000000000..d01d28f702c --- /dev/null +++ b/src/Server/PrometheusRequestHandlerConfig.h @@ -0,0 +1,39 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Configuration of a Prometheus protocol handler after it's parsed from a configuration file. +struct PrometheusRequestHandlerConfig +{ + enum class Type + { + /// Exposes ClickHouse metrics for scraping by Prometheus. + ExposeMetrics, + + /// Handles Prometheus remote-write protocol. + RemoteWrite, + + /// Handles Prometheus remote-read protocol. + RemoteRead, + }; + + Type type = Type::ExposeMetrics; + + /// Settings for type ExposeMetrics: + bool expose_metrics = false; + bool expose_asynchronous_metrics = false; + bool expose_events = false; + bool expose_errors = false; + + /// Settings for types RemoteWrite, RemoteRead: + QualifiedTableName time_series_table_name; + + size_t keep_alive_timeout = 0; + bool is_stacktrace_enabled = true; +}; + +} diff --git a/src/Server/PrometheusRequestHandlerFactory.cpp b/src/Server/PrometheusRequestHandlerFactory.cpp new file mode 100644 index 00000000000..52f1d3b64c1 --- /dev/null +++ b/src/Server/PrometheusRequestHandlerFactory.cpp @@ -0,0 +1,242 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + +namespace +{ + /// Parses common configuration which is attached to any other configuration. The common configuration looks like this: + /// + /// true + /// + /// 30 + void parseCommonConfig(const Poco::Util::AbstractConfiguration & config, PrometheusRequestHandlerConfig & res) + { + res.is_stacktrace_enabled = config.getBool("prometheus.enable_stacktrace", true); + res.keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); + } + + /// Parses a configuration like this: + /// + /// true + /// true + /// true + /// true + PrometheusRequestHandlerConfig parseExposeMetricsConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + PrometheusRequestHandlerConfig res; + res.type = PrometheusRequestHandlerConfig::Type::ExposeMetrics; + res.expose_metrics = config.getBool(config_prefix + ".metrics", true); + res.expose_asynchronous_metrics = config.getBool(config_prefix + ".asynchronous_metrics", true); + res.expose_events = config.getBool(config_prefix + ".events", true); + res.expose_errors = config.getBool(config_prefix + ".errors", true); + parseCommonConfig(config, res); + return res; + } + + /// Extracts a qualified table name from the config. It can be set either as + /// mydb.prometheus
+ /// or + /// mydb + /// prometheus
+ QualifiedTableName parseTableNameFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + QualifiedTableName res; + res.table = config.getString(config_prefix + ".table", "prometheus"); + res.database = config.getString(config_prefix + ".database", ""); + if (res.database.empty()) + res = QualifiedTableName::parseFromString(res.table); + if (res.database.empty()) + res.database = "default"; + return res; + } + + /// Parses a configuration like this: + /// + /// db.time_series_table_name
+ PrometheusRequestHandlerConfig parseRemoteWriteConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + PrometheusRequestHandlerConfig res; + res.type = PrometheusRequestHandlerConfig::Type::RemoteWrite; + res.time_series_table_name = parseTableNameFromConfig(config, config_prefix); + parseCommonConfig(config, res); + return res; + } + + /// Parses a configuration like this: + /// + /// db.time_series_table_name
+ PrometheusRequestHandlerConfig parseRemoteReadConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + PrometheusRequestHandlerConfig res; + res.type = PrometheusRequestHandlerConfig::Type::RemoteRead; + res.time_series_table_name = parseTableNameFromConfig(config, config_prefix); + parseCommonConfig(config, res); + return res; + } + + /// Parses a configuration like this: + /// expose_metrics + /// true + /// true + /// true + /// true + /// -OR- + /// remote_write + /// db.time_series_table_name
+ PrometheusRequestHandlerConfig parseHandlerConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + String type = config.getString(config_prefix + ".type"); + + if (type == "expose_metrics") + return parseExposeMetricsConfig(config, config_prefix); + else if (type == "remote_write") + return parseRemoteWriteConfig(config, config_prefix); + else if (type == "remote_read") + return parseRemoteReadConfig(config, config_prefix); + else + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown type {} is specified in the configuration for a prometheus protocol", type); + } + + /// Returns true if the protocol represented by a passed config can be handled. + bool canBeHandled(const PrometheusRequestHandlerConfig & config, bool for_keeper) + { + /// The standalone ClickHouse Keeper can only expose its metrics. + /// It can't handle other Prometheus protocols. + return !for_keeper || (config.type == PrometheusRequestHandlerConfig::Type::ExposeMetrics); + } + + /// Creates a writer which serializes exposing metrics. + std::shared_ptr createPrometheusMetricWriter(bool for_keeper) + { + if (for_keeper) + return std::make_unique(); + else + return std::make_unique(); + } + + /// Base function for making a factory for PrometheusRequestHandler. This function can return nullptr. + std::shared_ptr> createPrometheusHandlerFactoryFromConfig( + IServer & server, + const AsynchronousMetrics & async_metrics, + const PrometheusRequestHandlerConfig & config, + bool for_keeper) + { + if (!canBeHandled(config, for_keeper)) + return nullptr; + auto metric_writer = createPrometheusMetricWriter(for_keeper); + auto creator = [&server, &async_metrics, config, metric_writer]() -> std::unique_ptr + { + return std::make_unique(server, config, async_metrics, metric_writer); + }; + return std::make_shared>(std::move(creator)); + } + + /// Generic function for createPrometheusHandlerFactory() and createKeeperPrometheusHandlerFactory(). + HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryImpl( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name, + bool for_keeper) + { + auto factory = std::make_shared(name); + + if (config.has("prometheus.handlers")) + { + Strings keys; + config.keys("prometheus.handlers", keys); + for (const String & key : keys) + { + String prefix = "prometheus.handlers." + key; + auto parsed_config = parseHandlerConfig(config, prefix + ".handler"); + if (auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, for_keeper)) + { + handler->addFiltersFromConfig(config, prefix); + factory->addHandler(handler); + } + } + } + else + { + auto parsed_config = parseExposeMetricsConfig(config, "prometheus"); + if (auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, for_keeper)) + { + String endpoint = config.getString("prometheus.endpoint", "/metrics"); + handler->attachStrictPath(endpoint); + handler->allowGetAndHeadRequest(); + factory->addHandler(handler); + } + } + + return factory; + } + +} + + +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name) +{ + return createPrometheusHandlerFactoryImpl(server, config, asynchronous_metrics, name, /* for_keeper= */ false); +} + + +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRule( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const AsynchronousMetrics & asynchronous_metrics) +{ + auto parsed_config = parseExposeMetricsConfig(config, config_prefix + ".handler"); + auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, /* for_keeper= */ false); + chassert(handler); /// `handler` can't be nullptr here because `for_keeper` is false. + handler->addFiltersFromConfig(config, config_prefix); + return handler; +} + + +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRuleDefaults( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics) +{ + /// The "defaults" HTTP handler should serve the prometheus exposing metrics protocol on the http port + /// only if it isn't already served on its own port and if there is no section. + if (!config.has("prometheus") || config.getInt("prometheus.port", 0) || config.has("prometheus.handlers")) + return nullptr; + + auto parsed_config = parseExposeMetricsConfig(config, "prometheus"); + String endpoint = config.getString("prometheus.endpoint", "/metrics"); + auto handler = createPrometheusHandlerFactoryFromConfig(server, asynchronous_metrics, parsed_config, /* for_keeper= */ false); + chassert(handler); /// `handler` can't be nullptr here because `for_keeper` is false. + handler->attachStrictPath(endpoint); + handler->allowGetAndHeadRequest(); + return handler; +} + + +HTTPRequestHandlerFactoryPtr createKeeperPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name) +{ + return createPrometheusHandlerFactoryImpl(server, config, asynchronous_metrics, name, /* for_keeper= */ true); +} + +} diff --git a/src/Server/PrometheusRequestHandlerFactory.h b/src/Server/PrometheusRequestHandlerFactory.h new file mode 100644 index 00000000000..c52395ca93f --- /dev/null +++ b/src/Server/PrometheusRequestHandlerFactory.h @@ -0,0 +1,130 @@ +#pragma once + +#include +#include + + +namespace Poco::Util { class AbstractConfiguration; } + +namespace DB +{ + +class IServer; +class HTTPRequestHandlerFactory; +using HTTPRequestHandlerFactoryPtr = std::shared_ptr; +class AsynchronousMetrics; + +/// Makes a handler factory to handle prometheus protocols. +/// Expects a configuration like this: +/// +/// +/// 1234 +/// /metric +/// true +/// true +/// true +/// true +/// +/// +/// More prometheus protocols can be supported with using a different configuration +/// (which is similar to the section): +/// +/// +/// 1234 +/// +/// +/// /metrics +/// +/// expose_metrics +/// true +/// true +/// true +/// true +/// +/// +/// +/// +/// +/// An alternative port to serve prometheus protocols can be specified in the section: +/// +/// +/// +/// 4321 +/// prometheus +/// +/// +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name); + +/// Makes a HTTP handler factory to handle requests for prometheus metrics for a HTTP rule in the section. +/// Expects a configuration like this: +/// +/// 8123 +/// +/// +/// /metrics +/// +/// prometheus +/// true +/// true +/// true +/// true +/// +/// +/// +/// /write +/// +/// remote_write +/// db.time_series_table_name
+///
+///
+/// +/// /read +/// +/// remote_read +/// db.time_series_table_name
+///
+///
+///
+HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRule( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, /// path to "http_handlers.my_handler_1" + const AsynchronousMetrics & asynchronous_metrics); + +/// Makes a HTTP Handler factory to handle requests for prometheus metrics as a part of the default HTTP rule in the section. +/// Expects a configuration like this: +/// +/// 8123 +/// +/// +/// +/// +/// /metric +/// true +/// true +/// true +/// true +/// +/// +/// The "defaults" HTTP handler should serve the prometheus exposing metrics protocol on the http port +/// only if it isn't already served on its own port , +/// and also if there is no section in the configuration +/// (because if that section exists then it must be in charge of how prometheus protocols are handled). +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactoryForHTTPRuleDefaults( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics); + +/// Makes a handler factory to handle prometheus protocols. +/// Supports the "expose_metrics" protocol only. +HTTPRequestHandlerFactoryPtr createKeeperPrometheusHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + const AsynchronousMetrics & asynchronous_metrics, + const String & name); + +} diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp new file mode 100644 index 00000000000..df6ccec4b7f --- /dev/null +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -0,0 +1,480 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ +extern const Metric KafkaLibrdkafkaThreads; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +template +struct KafkaInterceptors +{ + static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx); + + static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx); + + static rd_kafka_resp_err_t + rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/); + + static rd_kafka_resp_err_t rdKafkaOnConfDup( + rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx); +}; + +template +rd_kafka_resp_err_t +KafkaInterceptors::rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1); + + const auto & storage_id = self->getStorageID(); + const auto & table = storage_id.getTableName(); + + switch (thread_type) + { + case RD_KAFKA_THREAD_MAIN: + setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); + break; + case RD_KAFKA_THREAD_BACKGROUND: + setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); + break; + case RD_KAFKA_THREAD_BROKER: + setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); + break; + } + + /// Create ThreadStatus to track memory allocations from librdkafka threads. + // + /// And store them in a separate list (thread_statuses) to make sure that they will be destroyed, + /// regardless how librdkafka calls the hooks. + /// But this can trigger use-after-free if librdkafka will not destroy threads after rd_kafka_wait_destroyed() + auto thread_status = std::make_shared(); + std::lock_guard lock(self->thread_statuses_mutex); + self->thread_statuses.emplace_back(std::move(thread_status)); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +template +rd_kafka_resp_err_t KafkaInterceptors::rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1); + + std::lock_guard lock(self->thread_statuses_mutex); + const auto it = std::find_if( + self->thread_statuses.begin(), + self->thread_statuses.end(), + [](const auto & thread_status_ptr) { return thread_status_ptr.get() == current_thread; }); + if (it == self->thread_statuses.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread status for this librdkafka thread."); + + self->thread_statuses.erase(it); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + +template +rd_kafka_resp_err_t KafkaInterceptors::rdKafkaOnNew( + rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) +{ + TStorageKafka * self = reinterpret_cast(ctx); + rd_kafka_resp_err_t status; + + status = rd_kafka_interceptor_add_on_thread_start(rk, "init-thread", rdKafkaOnThreadStart, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + { + LOG_ERROR(self->log, "Cannot set on thread start interceptor due to {} error", status); + return status; + } + + status = rd_kafka_interceptor_add_on_thread_exit(rk, "exit-thread", rdKafkaOnThreadExit, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(self->log, "Cannot set on thread exit interceptor due to {} error", status); + + return status; +} + +template +rd_kafka_resp_err_t KafkaInterceptors::rdKafkaOnConfDup( + rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) +{ + TStorageKafka * self = reinterpret_cast(ctx); + rd_kafka_resp_err_t status; + + // cppkafka copies configuration multiple times + status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "init", rdKafkaOnConfDup, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + { + LOG_ERROR(self->log, "Cannot set on conf dup interceptor due to {} error", status); + return status; + } + + status = rd_kafka_conf_interceptor_add_on_new(new_conf, "init", rdKafkaOnNew, ctx); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(self->log, "Cannot set on conf new interceptor due to {} error", status); + + return status; +} + +template struct KafkaInterceptors; +template struct KafkaInterceptors; + +namespace +{ + +void setKafkaConfigValue(cppkafka::Configuration & kafka_config, const String & key, const String & value) +{ + /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. + /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md + const String setting_name_in_kafka_config = (key == "log_level") ? key : boost::replace_all_copy(key, "_", "."); + kafka_config.set(setting_name_in_kafka_config, value); +} + +void loadConfigProperty( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const String & tag) +{ + const String property_path = config_prefix + "." + tag; + const String property_value = config.getString(property_path); + + setKafkaConfigValue(kafka_config, tag, property_value); +} + +void loadNamedCollectionConfig(cppkafka::Configuration & kafka_config, const String & collection_name, const String & config_prefix) +{ + const auto & collection = NamedCollectionFactory::instance().get(collection_name); + for (const auto & key : collection->getKeys(-1, config_prefix)) + { + // Cut prefix with '.' before actual config tag. + const auto param_name = key.substr(config_prefix.size() + 1); + setKafkaConfigValue(kafka_config, param_name, collection->get(key)); + } +} + +void loadLegacyTopicConfig( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const String & config_prefix) +{ + if (!collection_name.empty()) + { + loadNamedCollectionConfig(kafka_config, collection_name, config_prefix); + return; + } + + Poco::Util::AbstractConfiguration::Keys tags; + config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + loadConfigProperty(kafka_config, config, config_prefix, tag); + } +} + +/// Read server configuration into cppkafa configuration, used by new per-topic configuration +void loadTopicConfig( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const String & config_prefix, + const String & topic) +{ + if (!collection_name.empty()) + { + const auto topic_prefix = fmt::format("{}.{}", config_prefix, KafkaConfigLoader::CONFIG_KAFKA_TOPIC_TAG); + const auto & collection = NamedCollectionFactory::instance().get(collection_name); + for (const auto & key : collection->getKeys(1, config_prefix)) + { + /// Only consider key . Multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. + if (!key.starts_with(topic_prefix)) + continue; + + const String kafka_topic_path = config_prefix + "." + key; + const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; + if (topic == collection->get(kafka_topic_name_path)) + /// Found it! Now read the per-topic configuration into cppkafka. + loadNamedCollectionConfig(kafka_config, collection_name, kafka_topic_path); + } + } + else + { + /// Read all tags one level below + Poco::Util::AbstractConfiguration::Keys tags; + config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + if (tag == KafkaConfigLoader::CONFIG_NAME_TAG) + continue; // ignore , it is used to match topic configurations + loadConfigProperty(kafka_config, config, config_prefix, tag); + } + } +} + +/// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration +void loadFromConfig( + cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params, const String & config_prefix) +{ + if (!params.collection_name.empty()) + { + loadNamedCollectionConfig(kafka_config, params.collection_name, config_prefix); + return; + } + + /// Read all tags one level below + Poco::Util::AbstractConfiguration::Keys tags; + params.config.keys(config_prefix, tags); + + for (const auto & tag : tags) + { + if (tag == KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG || tag == KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG) + /// Do not load consumer/producer properties, since they should be separated by different configuration objects. + continue; + + if (tag.starts_with( + KafkaConfigLoader::CONFIG_KAFKA_TOPIC_TAG)) /// multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. + { + // Update consumer topic-specific configuration (new syntax). Example with topics "football" and "baseball": + // + // + // football + // 250 + // 5000 + // + // + // baseball + // 300 + // 2000 + // + // + // Advantages: The period restriction no longer applies (e.g. sports.football will work), everything + // Kafka-related is below . + for (const auto & topic : params.topics) + { + /// Read topic name between ... + const String kafka_topic_path = config_prefix + "." + tag; + const String kafka_topic_name_path = kafka_topic_path + "." + KafkaConfigLoader::CONFIG_NAME_TAG; + const String topic_name = params.config.getString(kafka_topic_name_path); + + if (topic_name != topic) + continue; + loadTopicConfig(kafka_config, params.config, params.collection_name, kafka_topic_path, topic); + } + continue; + } + if (tag.starts_with(KafkaConfigLoader::CONFIG_KAFKA_TAG)) + /// skip legacy configuration per topic e.g. . + /// it will be processed is a separate function + continue; + // Update configuration from the configuration. Example: + // + // 250 + // 100000 + // + loadConfigProperty(kafka_config, params.config, config_prefix, tag); + } +} + +void loadLegacyConfigSyntax( + cppkafka::Configuration & kafka_config, + const Poco::Util::AbstractConfiguration & config, + const String & collection_name, + const Names & topics) +{ + for (const auto & topic : topics) + { + const String kafka_topic_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_TAG + "_" + topic; + loadLegacyTopicConfig(kafka_config, config, collection_name, kafka_topic_path); + } +} + +void loadConsumerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) +{ + const String consumer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_CONSUMER_TAG; + loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); + // A new syntax has higher priority + loadFromConfig(kafka_config, params, consumer_path); +} + +void loadProducerConfig(cppkafka::Configuration & kafka_config, const KafkaConfigLoader::LoadConfigParams & params) +{ + const String producer_path = KafkaConfigLoader::CONFIG_KAFKA_TAG + "." + KafkaConfigLoader::CONFIG_KAFKA_PRODUCER_TAG; + loadLegacyConfigSyntax(kafka_config, params.config, params.collection_name, params.topics); + // A new syntax has higher priority + loadFromConfig(kafka_config, params, producer_path); +} + +template +void updateGlobalConfiguration( + cppkafka::Configuration & kafka_config, TKafkaStorage & storage, const KafkaConfigLoader::LoadConfigParams & params) +{ + loadFromConfig(kafka_config, params, KafkaConfigLoader::CONFIG_KAFKA_TAG); + +#if USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) + LOG_WARNING(params.log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); + + kafka_config.set("sasl.kerberos.kinit.cmd", ""); + kafka_config.set("sasl.kerberos.min.time.before.relogin", "0"); + + if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) + { + String keytab = kafka_config.get("sasl.kerberos.keytab"); + String principal = kafka_config.get("sasl.kerberos.principal"); + LOG_DEBUG(params.log, "Running KerberosInit"); + try + { + kerberosInit(keytab, principal); + } + catch (const Exception & e) + { + LOG_ERROR(params.log, "KerberosInit failure: {}", getExceptionMessage(e, false)); + } + LOG_DEBUG(params.log, "Finished KerberosInit"); + } +#else // USE_KRB5 + if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) + LOG_WARNING(params.log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); +#endif // USE_KRB5 + // No need to add any prefix, messages can be distinguished + kafka_config.set_log_callback( + [log = params.log](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) + { + auto [poco_level, client_logs_level] = parseSyslogLevel(level); + const auto & kafka_object_config = handle.get_configuration(); + const std::string client_id_key{"client.id"}; + chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); + LOG_IMPL( + log, + client_logs_level, + poco_level, + "[client.id:{}] [rdk:{}] {}", + kafka_object_config.get(client_id_key), + facility, + message); + }); + + /// NOTE: statistics should be consumed, otherwise it creates too much + /// entries in the queue, that leads to memory leak and slow shutdown. + if (!kafka_config.has_property("statistics.interval.ms")) + { + // every 3 seconds by default. set to 0 to disable. + kafka_config.set("statistics.interval.ms", "3000"); + } + // Configure interceptor to change thread name + // + // TODO: add interceptors support into the cppkafka. + // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibility overrides it to noop. + { + // This should be safe, since we wait the rdkafka object anyway. + void * self = static_cast(&storage); + + int status; + + status + = rd_kafka_conf_interceptor_add_on_new(kafka_config.get_handle(), "init", KafkaInterceptors::rdKafkaOnNew, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(params.log, "Cannot set new interceptor due to {} error", status); + + // cppkafka always copy the configuration + status = rd_kafka_conf_interceptor_add_on_conf_dup( + kafka_config.get_handle(), "init", KafkaInterceptors::rdKafkaOnConfDup, self); + if (status != RD_KAFKA_RESP_ERR_NO_ERROR) + LOG_ERROR(params.log, "Cannot set dup conf interceptor due to {} error", status); + } +} + +} + +template +cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params) +{ + cppkafka::Configuration conf; + + conf.set("metadata.broker.list", params.brokers); + conf.set("group.id", params.group); + if (params.multiple_consumers) + conf.set("client.id", fmt::format("{}-{}", params.client_id, params.consumer_number)); + else + conf.set("client.id", params.client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start + + // that allows to prevent fast draining of the librdkafka queue + // during building of single insert block. Improves performance + // significantly, but may lead to bigger memory consumption. + size_t default_queued_min_messages = 100000; // must be greater than or equal to default + size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value + conf.set( + "queued.min.messages", std::min(std::max(params.max_block_size, default_queued_min_messages), max_allowed_queued_min_messages)); + + updateGlobalConfiguration(conf, storage, params); + loadConsumerConfig(conf, params); + + // those settings should not be changed by users. + conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished + conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. + conf.set("enable.partition.eof", "false"); // Ignore EOF messages + + for (auto & property : conf.get_all()) + { + LOG_TRACE(params.log, "Consumer set property {}:{}", property.first, property.second); + } + + return conf; +} + +template cppkafka::Configuration +KafkaConfigLoader::getConsumerConfiguration(StorageKafka & storage, const ConsumerConfigParams & params); +template cppkafka::Configuration +KafkaConfigLoader::getConsumerConfiguration(StorageKafka2 & storage, const ConsumerConfigParams & params); + +template +cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params) +{ + cppkafka::Configuration conf; + conf.set("metadata.broker.list", params.brokers); + conf.set("client.id", params.client_id); + conf.set("client.software.name", VERSION_NAME); + conf.set("client.software.version", VERSION_DESCRIBE); + + updateGlobalConfiguration(conf, storage, params); + loadProducerConfig(conf, params); + + for (auto & property : conf.get_all()) + { + LOG_TRACE(params.log, "Producer set property {}:{}", property.first, property.second); + } + + return conf; +} + +template cppkafka::Configuration +KafkaConfigLoader::getProducerConfiguration(StorageKafka & storage, const ProducerConfigParams & params); +template cppkafka::Configuration +KafkaConfigLoader::getProducerConfiguration(StorageKafka2 & storage, const ProducerConfigParams & params); + +} diff --git a/src/Storages/Kafka/KafkaConfigLoader.h b/src/Storages/Kafka/KafkaConfigLoader.h new file mode 100644 index 00000000000..f18683c17f0 --- /dev/null +++ b/src/Storages/Kafka/KafkaConfigLoader.h @@ -0,0 +1,54 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ +struct KafkaSettings; +class VirtualColumnsDescription; + +struct KafkaConfigLoader +{ + static inline const String CONFIG_KAFKA_TAG = "kafka"; + static inline const String CONFIG_KAFKA_TOPIC_TAG = "kafka_topic"; + static inline const String CONFIG_NAME_TAG = "name"; + static inline const String CONFIG_KAFKA_CONSUMER_TAG = "consumer"; + static inline const String CONFIG_KAFKA_PRODUCER_TAG = "producer"; + using LogCallback = cppkafka::Configuration::LogCallback; + + + struct LoadConfigParams + { + const Poco::Util::AbstractConfiguration & config; + String & collection_name; + const Names & topics; + LoggerPtr & log; + }; + + struct ConsumerConfigParams : public LoadConfigParams + { + String brokers; + String group; + bool multiple_consumers; + size_t consumer_number; + String client_id; + size_t max_block_size; + }; + + struct ProducerConfigParams : public LoadConfigParams + { + String brokers; + String client_id; + }; + + template + static cppkafka::Configuration getConsumerConfiguration(TKafkaStorage & storage, const ConsumerConfigParams & params); + + template + static cppkafka::Configuration getProducerConfiguration(TKafkaStorage & storage, const ProducerConfigParams & params); +}; +} diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 9ba42b9875e..d9256cf39ce 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -20,13 +21,12 @@ namespace CurrentMetrics namespace ProfileEvents { - extern const Event KafkaRebalanceRevocations; - extern const Event KafkaRebalanceAssignments; - extern const Event KafkaRebalanceErrors; - extern const Event KafkaMessagesPolled; - extern const Event KafkaCommitFailures; - extern const Event KafkaCommits; - extern const Event KafkaConsumerErrors; +extern const Event KafkaRebalanceRevocations; +extern const Event KafkaRebalanceAssignments; +extern const Event KafkaRebalanceErrors; +extern const Event KafkaMessagesPolled; +extern const Event KafkaCommitFailures; +extern const Event KafkaCommits; } namespace DB @@ -199,44 +199,9 @@ KafkaConsumer::~KafkaConsumer() // https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. void KafkaConsumer::drain() { - auto start_time = std::chrono::steady_clock::now(); - cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); - - while (true) - { - auto msg = consumer->poll(100ms); - if (!msg) - break; - - auto error = msg.get_error(); - - if (error) - { - if (msg.is_eof() || error == last_error) - { - break; - } - else - { - LOG_ERROR(log, "Error during draining: {}", error); - setExceptionInfo(error); - } - } - - // i don't stop draining on first error, - // only if it repeats once again sequentially - last_error = error; - - auto ts = std::chrono::steady_clock::now(); - if (std::chrono::duration_cast(ts-start_time) > DRAIN_TIMEOUT_MS) - { - LOG_ERROR(log, "Timeout during draining."); - break; - } - } + StorageKafkaUtils::drainConsumer(*consumer, DRAIN_TIMEOUT_MS, log, [this](const cppkafka::Error & err) { setExceptionInfo(err); }); } - void KafkaConsumer::commit() { auto print_offsets = [this] (const char * prefix, const cppkafka::TopicPartitionList & offsets) @@ -409,7 +374,7 @@ void KafkaConsumer::resetToLastCommitted(const char * msg) { if (!assignment.has_value() || assignment->empty()) { - LOG_TRACE(log, "Not assignned. Can't reset to last committed position."); + LOG_TRACE(log, "Not assigned. Can't reset to last committed position."); return; } auto committed_offset = consumer->get_offsets_committed(consumer->get_assignment()); @@ -473,7 +438,7 @@ ReadBufferPtr KafkaConsumer::consume() // If we're doing a manual select then it's better to get something after a wait, then immediate nothing. if (!assignment.has_value()) { - waited_for_assignment += poll_timeout; // slightly innaccurate, but rough calculation is ok. + waited_for_assignment += poll_timeout; // slightly inaccurate, but rough calculation is ok. if (waited_for_assignment < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) { continue; @@ -535,26 +500,12 @@ ReadBufferPtr KafkaConsumer::getNextMessage() return getNextMessage(); } -size_t KafkaConsumer::filterMessageErrors() +void KafkaConsumer::filterMessageErrors() { assert(current == messages.begin()); - size_t skipped = std::erase_if(messages, [this](auto & message) - { - if (auto error = message.get_error()) - { - ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); - LOG_ERROR(log, "Consumer error: {}", error); - setExceptionInfo(error); - return true; - } - return false; - }); - - if (skipped) - LOG_ERROR(log, "There were {} messages with an error", skipped); - - return skipped; + StorageKafkaUtils::eraseMessageErrors(messages, log, [this](const cppkafka::Error & err) { setExceptionInfo(err); }); + current = messages.begin(); } void KafkaConsumer::resetIfStopped() diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 4daf8652c3b..285f3680213 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,14 +1,12 @@ #pragma once #include -#include #include #include #include #include -#include #include namespace CurrentMetrics @@ -193,12 +191,8 @@ private: void drain(); void cleanUnprocessed(); void resetIfStopped(); - /// Return number of messages with an error. - size_t filterMessageErrors(); + void filterMessageErrors(); ReadBufferPtr getNextMessage(); }; } - -template <> struct fmt::formatter : fmt::ostream_formatter {}; -template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Storages/Kafka/KafkaConsumer2.cpp b/src/Storages/Kafka/KafkaConsumer2.cpp new file mode 100644 index 00000000000..60626dfa402 --- /dev/null +++ b/src/Storages/Kafka/KafkaConsumer2.cpp @@ -0,0 +1,384 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include + + +namespace CurrentMetrics +{ +extern const Metric KafkaAssignedPartitions; +extern const Metric KafkaConsumersWithAssignment; +} + +namespace ProfileEvents +{ +extern const Event KafkaRebalanceRevocations; +extern const Event KafkaRebalanceAssignments; +extern const Event KafkaRebalanceErrors; +extern const Event KafkaMessagesPolled; +extern const Event KafkaCommitFailures; +extern const Event KafkaCommits; +} + +namespace DB +{ + +using namespace std::chrono_literals; +static constexpr auto EVENT_POLL_TIMEOUT = 50ms; +static constexpr auto DRAIN_TIMEOUT_MS = 5000ms; + + +bool KafkaConsumer2::TopicPartition::operator<(const TopicPartition & other) const +{ + return std::tie(topic, partition_id, offset) < std::tie(other.topic, other.partition_id, other.offset); +} + + +KafkaConsumer2::KafkaConsumer2( + ConsumerPtr consumer_, + LoggerPtr log_, + size_t max_batch_size, + size_t poll_timeout_, + const std::atomic & stopped_, + const Names & topics_) + : consumer(consumer_) + , log(log_) + , batch_size(max_batch_size) + , poll_timeout(poll_timeout_) + , stopped(stopped_) + , current(messages.begin()) + , topics(topics_) +{ + // called (synchronously, during poll) when we enter the consumer group + consumer->set_assignment_callback( + [this](const cppkafka::TopicPartitionList & topic_partitions) + { + CurrentMetrics::add(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceAssignments); + + if (topic_partitions.empty()) + { + LOG_INFO(log, "Got empty assignment: Not enough partitions in the topic for all consumers?"); + } + else + { + LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); + CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } + + chassert(!assignment.has_value()); + + assignment.emplace(); + assignment->reserve(topic_partitions.size()); + needs_offset_update = true; + for (const auto & topic_partition : topic_partitions) + { + assignment->push_back( + TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), topic_partition.get_offset()}); + } + + // We need to initialize the queues here in order to detach them from the consumer queue. Otherwise `pollEvents` might eventually poll actual messages also. + initializeQueues(topic_partitions); + }); + + // called (synchronously, during poll) when we leave the consumer group + consumer->set_revocation_callback( + [this](const cppkafka::TopicPartitionList & topic_partitions) + { + CurrentMetrics::sub(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size()); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceRevocations); + + // Rebalance is happening now, and now we have a chance to finish the work + // with topics/partitions we were working with before rebalance + LOG_TRACE(log, "Rebalance initiated. Revoking partitions: {}", topic_partitions); + + if (!topic_partitions.empty()) + { + CurrentMetrics::sub(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } + + assignment.reset(); + queues.clear(); + needs_offset_update = true; + }); + + consumer->set_rebalance_error_callback( + [this](cppkafka::Error err) + { + LOG_ERROR(log, "Rebalance error: {}", err); + ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors); + }); +} + +KafkaConsumer2::~KafkaConsumer2() +{ + try + { + if (!consumer->get_subscription().empty()) + { + try + { + consumer->unsubscribe(); + } + catch (const cppkafka::HandleException & e) + { + LOG_ERROR(log, "Error during unsubscribe: {}", e.what()); + } + drainConsumerQueue(); + } + } + catch (const cppkafka::HandleException & e) + { + LOG_ERROR(log, "Error while destructing consumer: {}", e.what()); + } +} + +// Needed to drain rest of the messages / queued callback calls from the consumer after unsubscribe, otherwise consumer +// will hang on destruction. Partition queues doesn't have to be attached as events are not handled by those queues. +// see https://github.com/edenhill/librdkafka/issues/2077 +// https://github.com/confluentinc/confluent-kafka-go/issues/189 etc. +void KafkaConsumer2::drainConsumerQueue() +{ + StorageKafkaUtils::drainConsumer(*consumer, DRAIN_TIMEOUT_MS, log); +} + +void KafkaConsumer2::pollEvents() +{ + static constexpr auto max_tries = 5; + for (auto i = 0; i < max_tries; ++i) + { + auto msg = consumer->poll(EVENT_POLL_TIMEOUT); + if (!msg) + return; + // All the partition queues are detached, so the consumer shouldn't be able to poll any real messages + const auto err = msg.get_error(); + chassert(RD_KAFKA_RESP_ERR_NO_ERROR != err.get_error() && "Consumer returned a message when it was not expected"); + LOG_ERROR(log, "Consumer received error while polling events, code {}, error '{}'", err.get_error(), err.to_string()); + } +}; + +bool KafkaConsumer2::polledDataUnusable(const TopicPartition & topic_partition) const +{ + const auto different_topic_partition = current == messages.end() + ? false + : (current->get_topic() != topic_partition.topic || current->get_partition() != topic_partition.partition_id); + return different_topic_partition; +} + +KafkaConsumer2::TopicPartitions const * KafkaConsumer2::getKafkaAssignment() const +{ + if (assignment.has_value()) + { + return &*assignment; + } + + return nullptr; +} + +void KafkaConsumer2::updateOffsets(const TopicPartitions & topic_partitions) +{ + cppkafka::TopicPartitionList original_topic_partitions; + original_topic_partitions.reserve(topic_partitions.size()); + std::transform( + topic_partitions.begin(), + topic_partitions.end(), + std::back_inserter(original_topic_partitions), + [](const TopicPartition & tp) + { + return cppkafka::TopicPartition{tp.topic, tp.partition_id, tp.offset}; + }); + initializeQueues(original_topic_partitions); + needs_offset_update = false; + stalled_status = StalledStatus::NOT_STALLED; +} + +void KafkaConsumer2::initializeQueues(const cppkafka::TopicPartitionList & topic_partitions) +{ + queues.clear(); + messages.clear(); + current = messages.end(); + // cppkafka itself calls assign(), but in order to detach the queues here we have to do the assignment manually. Later on we have to reassign the topic partitions with correct offsets. + consumer->assign(topic_partitions); + for (const auto & topic_partition : topic_partitions) + // This will also detach the partition queues from the consumer, thus the messages won't be forwarded without attaching them manually + queues.emplace( + TopicPartition{topic_partition.get_topic(), topic_partition.get_partition(), topic_partition.get_offset()}, + consumer->get_partition_queue(topic_partition)); +} + +// it do the poll when needed +ReadBufferPtr KafkaConsumer2::consume(const TopicPartition & topic_partition, const std::optional & message_count) +{ + resetIfStopped(); + + if (polledDataUnusable(topic_partition)) + return nullptr; + + if (hasMorePolledMessages()) + { + if (auto next_message = getNextMessage(); next_message) + return next_message; + } + + while (true) + { + stalled_status = StalledStatus::NO_MESSAGES_RETURNED; + + auto & queue_to_poll_from = queues.at(topic_partition); + LOG_TRACE(log, "Batch size {}, offset {}", batch_size, topic_partition.offset); + const auto messages_to_pull = message_count.value_or(batch_size); + /// Don't drop old messages immediately, since we may need them for virtual columns. + auto new_messages = queue_to_poll_from.consume_batch(messages_to_pull, std::chrono::milliseconds(poll_timeout)); + + resetIfStopped(); + if (stalled_status == StalledStatus::CONSUMER_STOPPED) + { + return nullptr; + } + + if (new_messages.empty()) + { + LOG_TRACE(log, "Stalled"); + return nullptr; + } + else + { + messages = std::move(new_messages); + current = messages.begin(); + LOG_TRACE( + log, + "Polled batch of {} messages. Offsets position: {}", + messages.size(), + consumer->get_offsets_position(consumer->get_assignment())); + break; + } + } + + filterMessageErrors(); + if (current == messages.end()) + { + LOG_ERROR(log, "Only errors left"); + stalled_status = StalledStatus::ERRORS_RETURNED; + return nullptr; + } + + ProfileEvents::increment(ProfileEvents::KafkaMessagesPolled, messages.size()); + + stalled_status = StalledStatus::NOT_STALLED; + return getNextMessage(); +} + +void KafkaConsumer2::commit(const TopicPartition & topic_partition) +{ + static constexpr auto max_retries = 5; + bool committed = false; + + LOG_TEST( + log, + "Trying to commit offset {} to Kafka for topic-partition [{}:{}]", + topic_partition.offset, + topic_partition.topic, + topic_partition.partition_id); + + const auto topic_partition_list = std::vector{cppkafka::TopicPartition{ + topic_partition.topic, + topic_partition.partition_id, + topic_partition.offset, + }}; + for (auto try_count = 0; try_count < max_retries && !committed; ++try_count) + { + try + { + // See https://github.com/edenhill/librdkafka/issues/1470 + // broker may reject commit if during offsets.commit.timeout.ms (5000 by default), + // there were not enough replicas available for the __consumer_offsets topic. + // also some other temporary issues like client-server connectivity problems are possible + + consumer->commit(topic_partition_list); + committed = true; + LOG_INFO( + log, + "Committed offset {} to Kafka for topic-partition [{}:{}]", + topic_partition.offset, + topic_partition.topic, + topic_partition.partition_id); + } + catch (const cppkafka::HandleException & e) + { + // If there were actually no offsets to commit, return. Retrying won't solve + // anything here + if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) + committed = true; + else + LOG_ERROR(log, "Exception during attempt to commit to Kafka: {}", e.what()); + } + } + + if (!committed) + { + // The failure is not the biggest issue, it only counts when a table is dropped and recreated, otherwise the offsets are taken from keeper. + ProfileEvents::increment(ProfileEvents::KafkaCommitFailures); + LOG_ERROR(log, "All commit attempts failed"); + } + else + { + ProfileEvents::increment(ProfileEvents::KafkaCommits); + } +} + +void KafkaConsumer2::subscribeIfNotSubscribedYet() +{ + if (likely(is_subscribed)) + return; + + consumer->subscribe(topics); + is_subscribed = true; + LOG_DEBUG(log, "Subscribed."); +} + +ReadBufferPtr KafkaConsumer2::getNextMessage() +{ + while (current != messages.end()) + { + const auto * data = current->get_payload().get_data(); + size_t size = current->get_payload().get_size(); + ++current; + + // `data` can be nullptr on case of the Kafka message has empty payload + if (data) + return std::make_shared(data, size); + } + + return nullptr; +} + +void KafkaConsumer2::filterMessageErrors() +{ + assert(current == messages.begin()); + + StorageKafkaUtils::eraseMessageErrors(messages, log); + current = messages.begin(); +} + +void KafkaConsumer2::resetIfStopped() +{ + if (stopped) + { + stalled_status = StalledStatus::CONSUMER_STOPPED; + } +} +} diff --git a/src/Storages/Kafka/KafkaConsumer2.h b/src/Storages/Kafka/KafkaConsumer2.h new file mode 100644 index 00000000000..dd2cfe87aa0 --- /dev/null +++ b/src/Storages/Kafka/KafkaConsumer2.h @@ -0,0 +1,162 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace CurrentMetrics +{ +extern const Metric KafkaConsumers; +} + +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +using ConsumerPtr = std::shared_ptr; + +class KafkaConsumer2 +{ +public: + static inline constexpr int INVALID_OFFSET = RD_KAFKA_OFFSET_INVALID; + static inline constexpr int BEGINNING_OFFSET = RD_KAFKA_OFFSET_BEGINNING; + static inline constexpr int END_OFFSET = RD_KAFKA_OFFSET_END; + + struct TopicPartition + { + String topic; + int32_t partition_id; + int64_t offset{INVALID_OFFSET}; + + bool operator==(const TopicPartition &) const = default; + bool operator<(const TopicPartition & other) const; + }; + + using TopicPartitions = std::vector; + + struct OnlyTopicNameAndPartitionIdHash + { + std::size_t operator()(const TopicPartition & tp) const + { + SipHash s; + s.update(tp.topic); + s.update(tp.partition_id); + return s.get64(); + } + }; + + struct OnlyTopicNameAndPartitionIdEquality + { + bool operator()(const TopicPartition & lhs, const TopicPartition & rhs) const + { + return lhs.topic == rhs.topic && lhs.partition_id == rhs.partition_id; + } + }; + + struct TopicPartitionCount + { + String topic; + size_t partition_count; + }; + + using TopicPartitionCounts = std::vector; + + KafkaConsumer2( + ConsumerPtr consumer_, + LoggerPtr log_, + size_t max_batch_size, + size_t poll_timeout_, + const std::atomic & stopped_, + const Names & topics_); + + ~KafkaConsumer2(); + + // Poll only the main consumer queue without any topic-partition queues. This is useful to get notified about events, such as rebalance, + // new assignment, etc.. + void pollEvents(); + + auto pollTimeout() const { return poll_timeout; } + + inline bool hasMorePolledMessages() const { return (stalled_status == StalledStatus::NOT_STALLED) && (current != messages.end()); } + + inline bool isStalled() const { return stalled_status != StalledStatus::NOT_STALLED; } + + // Returns the topic partitions that the consumer got from rebalancing the consumer group. If the consumer received + // no topic partitions or all of them were revoked, it returns a null pointer. + TopicPartitions const * getKafkaAssignment() const; + + // As the main source of offsets is not Kafka, the offsets needs to be pushed to the consumer from outside + // Returns true if it received new assignment and internal state should be updated by updateOffsets + bool needsOffsetUpdate() const { return needs_offset_update; } + void updateOffsets(const TopicPartitions & topic_partitions); + + /// Polls batch of messages from the given topic-partition and returns read buffer containing the next message or + /// nullptr when there are no messages to process. + ReadBufferPtr consume(const TopicPartition & topic_partition, const std::optional & message_count); + + void commit(const TopicPartition & topic_partition); + + // Return values for the message that's being read. + String currentTopic() const { return current[-1].get_topic(); } + String currentKey() const { return current[-1].get_key(); } + auto currentOffset() const { return current[-1].get_offset(); } + auto currentPartition() const { return current[-1].get_partition(); } + auto currentTimestamp() const { return current[-1].get_timestamp(); } + const auto & currentHeaderList() const { return current[-1].get_header_list(); } + String currentPayload() const { return current[-1].get_payload(); } + + void subscribeIfNotSubscribedYet(); + +private: + using Messages = std::vector; + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; + + enum class StalledStatus + { + NOT_STALLED, + NO_MESSAGES_RETURNED, + CONSUMER_STOPPED, + NO_ASSIGNMENT, + ERRORS_RETURNED + }; + + ConsumerPtr consumer; + LoggerPtr log; + const size_t batch_size = 1; + const size_t poll_timeout = 0; + + StalledStatus stalled_status = StalledStatus::NO_MESSAGES_RETURNED; + + const std::atomic & stopped; + bool is_subscribed = false; + + // order is important, need to be destructed before consumer + Messages messages; + Messages::const_iterator current; + + // order is important, need to be destructed before consumer + std::optional assignment; + bool needs_offset_update{false}; + std::unordered_map queues; + const Names topics; + + bool polledDataUnusable(const TopicPartition & topic_partition) const; + void drainConsumerQueue(); + void resetIfStopped(); + void filterMessageErrors(); + ReadBufferPtr getNextMessage(); + + void initializeQueues(const cppkafka::TopicPartitionList & topic_partitions); +}; + +} diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 0addaf9e3b3..9ca5e189f0e 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -38,6 +38,8 @@ const auto KAFKA_CONSUMERS_POOL_TTL_MS_MAX = 600'000; M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ + M(String, kafka_keeper_path, "", "The path to the table in ClickHouse Keeper", 0) \ + M(String, kafka_replica_name, "", "The replica name in ClickHouse Keeper", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 9c68107872e..3ddd0d1be8c 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -262,7 +262,7 @@ Chunk KafkaSource::generateImpl() // they are not needed here: // and it's misleading to use them here, // as columns 'materialized' that way stays 'ephemeral' - // i.e. will not be stored anythere + // i.e. will not be stored anywhere // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 3aad64a0cfb..f4f641d1c68 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1,13 +1,5 @@ #include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -21,18 +13,19 @@ #include #include #include +#include +#include #include #include #include #include #include #include +#include #include #include -#include #include #include -#include #include #include #include @@ -41,10 +34,10 @@ #include #include #include +#include #include #include #include -#include #include #include #include @@ -55,13 +48,8 @@ #include #include -#if USE_KRB5 -#include -#endif // USE_KRB5 - namespace CurrentMetrics { - extern const Metric KafkaLibrdkafkaThreads; extern const Metric KafkaBackgroundReads; extern const Metric KafkaConsumersInUse; extern const Metric KafkaWrites; @@ -82,104 +70,10 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int QUERY_NOT_ALLOWED; extern const int ABORTED; } -struct StorageKafkaInterceptors -{ - static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx) - { - StorageKafka * self = reinterpret_cast(ctx); - CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1); - - const auto & storage_id = self->getStorageID(); - const auto & table = storage_id.getTableName(); - - switch (thread_type) - { - case RD_KAFKA_THREAD_MAIN: - setThreadName(("rdk:m/" + table.substr(0, 9)).c_str()); - break; - case RD_KAFKA_THREAD_BACKGROUND: - setThreadName(("rdk:bg/" + table.substr(0, 8)).c_str()); - break; - case RD_KAFKA_THREAD_BROKER: - setThreadName(("rdk:b/" + table.substr(0, 9)).c_str()); - break; - } - - /// Create ThreadStatus to track memory allocations from librdkafka threads. - // - /// And store them in a separate list (thread_statuses) to make sure that they will be destroyed, - /// regardless how librdkafka calls the hooks. - /// But this can trigger use-after-free if librdkafka will not destroy threads after rd_kafka_wait_destroyed() - auto thread_status = std::make_shared(); - std::lock_guard lock(self->thread_statuses_mutex); - self->thread_statuses.emplace_back(std::move(thread_status)); - - return RD_KAFKA_RESP_ERR_NO_ERROR; - } - static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx) - { - StorageKafka * self = reinterpret_cast(ctx); - CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1); - - std::lock_guard lock(self->thread_statuses_mutex); - const auto it = std::find_if(self->thread_statuses.begin(), self->thread_statuses.end(), [](const auto & thread_status_ptr) - { - return thread_status_ptr.get() == current_thread; - }); - if (it == self->thread_statuses.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No thread status for this librdkafka thread."); - - self->thread_statuses.erase(it); - - return RD_KAFKA_RESP_ERR_NO_ERROR; - } - - static rd_kafka_resp_err_t rdKafkaOnNew(rd_kafka_t * rk, const rd_kafka_conf_t *, void * ctx, char * /*errstr*/, size_t /*errstr_size*/) - { - StorageKafka * self = reinterpret_cast(ctx); - rd_kafka_resp_err_t status; - - status = rd_kafka_interceptor_add_on_thread_start(rk, "init-thread", rdKafkaOnThreadStart, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - { - LOG_ERROR(self->log, "Cannot set on thread start interceptor due to {} error", status); - return status; - } - - status = rd_kafka_interceptor_add_on_thread_exit(rk, "exit-thread", rdKafkaOnThreadExit, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(self->log, "Cannot set on thread exit interceptor due to {} error", status); - - return status; - } - - static rd_kafka_resp_err_t rdKafkaOnConfDup(rd_kafka_conf_t * new_conf, const rd_kafka_conf_t * /*old_conf*/, size_t /*filter_cnt*/, const char ** /*filter*/, void * ctx) - { - StorageKafka * self = reinterpret_cast(ctx); - rd_kafka_resp_err_t status; - - // cppkafka copies configuration multiple times - status = rd_kafka_conf_interceptor_add_on_conf_dup(new_conf, "init", rdKafkaOnConfDup, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - { - LOG_ERROR(self->log, "Cannot set on conf dup interceptor due to {} error", status); - return status; - } - - status = rd_kafka_conf_interceptor_add_on_new(new_conf, "init", rdKafkaOnNew, ctx); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(self->log, "Cannot set on conf new interceptor due to {} error", status); - - return status; - } -}; - class ReadFromStorageKafka final : public ReadFromStreamLikeEngine { public: @@ -241,182 +135,6 @@ private: StorageSnapshotPtr storage_snapshot; }; -namespace -{ - const String CONFIG_KAFKA_TAG = "kafka"; - const String CONFIG_KAFKA_TOPIC_TAG = "kafka_topic"; - const String CONFIG_KAFKA_CONSUMER_TAG = "consumer"; - const String CONFIG_KAFKA_PRODUCER_TAG = "producer"; - const String CONFIG_NAME_TAG = "name"; - - void setKafkaConfigValue(cppkafka::Configuration & kafka_config, const String & key, const String & value) - { - /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. - /// See https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md - const String setting_name_in_kafka_config = (key == "log_level") ? key : boost::replace_all_copy(key, "_", "."); - kafka_config.set(setting_name_in_kafka_config, value); - } - - void loadConfigProperty(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const String & tag) - { - const String property_path = config_prefix + "." + tag; - const String property_value = config.getString(property_path); - - setKafkaConfigValue(kafka_config, tag, property_value); - } - - void loadNamedCollectionConfig(cppkafka::Configuration & kafka_config, const String & collection_name, const String & config_prefix) - { - const auto & collection = NamedCollectionFactory::instance().get(collection_name); - for (const auto & key : collection->getKeys(-1, config_prefix)) - { - // Cut prefix with '.' before actual config tag. - const auto param_name = key.substr(config_prefix.size() + 1); - setKafkaConfigValue(kafka_config, param_name, collection->get(key)); - } - } - - void loadLegacyTopicConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & config_prefix) - { - if (!collection_name.empty()) - { - loadNamedCollectionConfig(kafka_config, collection_name, config_prefix); - return; - } - - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - loadConfigProperty(kafka_config, config, config_prefix, tag); - } - } - - /// Read server configuration into cppkafa configuration, used by new per-topic configuration - void loadTopicConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & config_prefix, const String & topic) - { - if (!collection_name.empty()) - { - const auto topic_prefix = fmt::format("{}.{}", config_prefix, CONFIG_KAFKA_TOPIC_TAG); - const auto & collection = NamedCollectionFactory::instance().get(collection_name); - for (const auto & key : collection->getKeys(1, config_prefix)) - { - /// Only consider key . Multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. - if (!key.starts_with(topic_prefix)) - continue; - - const String kafka_topic_path = config_prefix + "." + key; - const String kafka_topic_name_path = kafka_topic_path + "." + CONFIG_NAME_TAG; - if (topic == collection->get(kafka_topic_name_path)) - /// Found it! Now read the per-topic configuration into cppkafka. - loadNamedCollectionConfig(kafka_config, collection_name, kafka_topic_path); - } - } - else - { - /// Read all tags one level below - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - if (tag == CONFIG_NAME_TAG) - continue; // ignore , it is used to match topic configurations - loadConfigProperty(kafka_config, config, config_prefix, tag); - } - } - } - - /// Read server configuration into cppkafka configuration, used by global configuration and by legacy per-topic configuration - void loadFromConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & config_prefix, const Names & topics) - { - if (!collection_name.empty()) - { - loadNamedCollectionConfig(kafka_config, collection_name, config_prefix); - return; - } - - /// Read all tags one level below - Poco::Util::AbstractConfiguration::Keys tags; - config.keys(config_prefix, tags); - - for (const auto & tag : tags) - { - if (tag == CONFIG_KAFKA_PRODUCER_TAG || tag == CONFIG_KAFKA_CONSUMER_TAG) - /// Do not load consumer/producer properties, since they should be separated by different configuration objects. - continue; - - if (tag.starts_with(CONFIG_KAFKA_TOPIC_TAG)) /// multiple occurrences given as "kafka_topic", "kafka_topic[1]", etc. - { - // Update consumer topic-specific configuration (new syntax). Example with topics "football" and "baseball": - // - // - // football - // 250 - // 5000 - // - // - // baseball - // 300 - // 2000 - // - // - // Advantages: The period restriction no longer applies (e.g. sports.football will work), everything - // Kafka-related is below . - for (const auto & topic : topics) - { - /// Read topic name between ... - const String kafka_topic_path = config_prefix + "." + tag; - const String kafka_topic_name_path = kafka_topic_path + "." + CONFIG_NAME_TAG; - const String topic_name = config.getString(kafka_topic_name_path); - - if (topic_name != topic) - continue; - loadTopicConfig(kafka_config, config, collection_name, kafka_topic_path, topic); - } - continue; - } - if (tag.starts_with(CONFIG_KAFKA_TAG)) - /// skip legacy configuration per topic e.g. . - /// it will be processed is a separate function - continue; - // Update configuration from the configuration. Example: - // - // 250 - // 100000 - // - loadConfigProperty(kafka_config, config, config_prefix, tag); - } - } - - void loadLegacyConfigSyntax(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & prefix, const Names & topics) - { - for (const auto & topic : topics) - { - const String kafka_topic_path = prefix + "." + CONFIG_KAFKA_TAG + "_" + topic; - loadLegacyTopicConfig(kafka_config, config, collection_name, kafka_topic_path); - } - } - - void loadConsumerConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & prefix, const Names & topics) - { - const String consumer_path = prefix + "." + CONFIG_KAFKA_CONSUMER_TAG; - loadLegacyConfigSyntax(kafka_config, config, collection_name, prefix, topics); - // A new syntax has higher priority - loadFromConfig(kafka_config, config, collection_name, consumer_path, topics); - } - - void loadProducerConfig(cppkafka::Configuration & kafka_config, const Poco::Util::AbstractConfiguration & config, const String & collection_name, const String & prefix, const Names & topics) - { - const String producer_path = prefix + "." + CONFIG_KAFKA_PRODUCER_TAG; - loadLegacyConfigSyntax(kafka_config, config, collection_name, prefix, topics); - // A new syntax has higher priority - loadFromConfig(kafka_config, config, collection_name, producer_path, topics); - - } -} - StorageKafka::StorageKafka( const StorageID & table_id_, ContextPtr context_, @@ -428,19 +146,20 @@ StorageKafka::StorageKafka( , WithContext(context_->getGlobalContext()) , kafka_settings(std::move(kafka_settings_)) , macros_info{.table_id = table_id_} - , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) + , topics(StorageKafkaUtils::parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) , client_id( - kafka_settings->kafka_client_id.value.empty() ? getDefaultClientId(table_id_) - : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) + kafka_settings->kafka_client_id.value.empty() + ? StorageKafkaUtils::getDefaultClientId(table_id_) + : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) , num_consumers(kafka_settings->kafka_num_consumers.value) , log(getLogger("StorageKafka (" + table_id_.table_name + ")")) , intermediate_commit(kafka_settings->kafka_commit_every_batch.value) - , settings_adjustments(createSettingsAdjustments()) + , settings_adjustments(StorageKafkaUtils::createSettingsAdjustments(*kafka_settings, schema_name)) , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) , collection_name(collection_name_) { @@ -456,7 +175,7 @@ StorageKafka::StorageKafka( storage_metadata.setColumns(columns_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode)); + setVirtuals(StorageKafkaUtils::createVirtuals(kafka_settings->kafka_handle_error_mode)); auto task_count = thread_per_consumer ? num_consumers : 1; for (size_t i = 0; i < task_count; ++i) @@ -481,76 +200,6 @@ StorageKafka::StorageKafka( StorageKafka::~StorageKafka() = default; -VirtualColumnsDescription StorageKafka::createVirtuals(StreamingHandleErrorMode handle_error_mode) -{ - VirtualColumnsDescription desc; - - desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_key", std::make_shared(), ""); - desc.addEphemeral("_offset", std::make_shared(), ""); - desc.addEphemeral("_partition", std::make_shared(), ""); - desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); - desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); - desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); - - if (handle_error_mode == StreamingHandleErrorMode::STREAM) - { - desc.addEphemeral("_raw_message", std::make_shared(), ""); - desc.addEphemeral("_error", std::make_shared(), ""); - } - - return desc; -} - -SettingsChanges StorageKafka::createSettingsAdjustments() -{ - SettingsChanges result; - // Needed for backward compatibility - if (!kafka_settings->input_format_skip_unknown_fields.changed) - { - // Always skip unknown fields regardless of the context (JSON or TSKV) - kafka_settings->input_format_skip_unknown_fields = true; - } - - if (!kafka_settings->input_format_allow_errors_ratio.changed) - { - kafka_settings->input_format_allow_errors_ratio = 0.; - } - - if (!kafka_settings->input_format_allow_errors_num.changed) - { - kafka_settings->input_format_allow_errors_num = kafka_settings->kafka_skip_broken_messages.value; - } - - if (!schema_name.empty()) - result.emplace_back("format_schema", schema_name); - - for (const auto & setting : *kafka_settings) - { - const auto & name = setting.getName(); - if (name.find("kafka_") == std::string::npos) - result.emplace_back(name, setting.getValue()); - } - return result; -} - -Names StorageKafka::parseTopics(String topic_list) -{ - Names result; - boost::split(result,topic_list,[](char c){ return c == ','; }); - for (String & topic : result) - { - boost::trim(topic); - } - return result; -} - -String StorageKafka::getDefaultClientId(const StorageID & table_id_) -{ - return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id_.database_name, table_id_.table_name); -} - void StorageKafka::read( QueryPlan & query_plan, const Names & column_names, @@ -751,65 +400,26 @@ KafkaConsumerPtr StorageKafka::createKafkaConsumer(size_t consumer_number) topics); return kafka_consumer_ptr; } - cppkafka::Configuration StorageKafka::getConsumerConfiguration(size_t consumer_number) { - cppkafka::Configuration conf; - - conf.set("metadata.broker.list", brokers); - conf.set("group.id", group); - if (num_consumers > 1) - { - conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number)); - } - else - { - conf.set("client.id", client_id); - } - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start - - // that allows to prevent fast draining of the librdkafka queue - // during building of single insert block. Improves performance - // significantly, but may lead to bigger memory consumption. - size_t default_queued_min_messages = 100000; // must be greater than or equal to default - size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value - conf.set("queued.min.messages", std::min(std::max(getMaxBlockSize(), default_queued_min_messages), max_allowed_queued_min_messages)); - - updateGlobalConfiguration(conf); - updateConsumerConfiguration(conf); - - // those settings should not be changed by users. - conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished - conf.set("enable.auto.offset.store", "false"); // Update offset automatically - to commit them all at once. - conf.set("enable.partition.eof", "false"); // Ignore EOF messages - - for (auto & property : conf.get_all()) - { - LOG_TRACE(log, "Consumer set property {}:{}", property.first, property.second); - } - - return conf; + KafkaConfigLoader::ConsumerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + group, + num_consumers > 1, + consumer_number, + client_id, + getMaxBlockSize()}; + return KafkaConfigLoader::getConsumerConfiguration(*this, params); } cppkafka::Configuration StorageKafka::getProducerConfiguration() { - cppkafka::Configuration conf; - conf.set("metadata.broker.list", brokers); - conf.set("client.id", client_id); - conf.set("client.software.name", VERSION_NAME); - conf.set("client.software.version", VERSION_DESCRIBE); - - updateGlobalConfiguration(conf); - updateProducerConfiguration(conf); - - for (auto & property : conf.get_all()) - { - LOG_TRACE(log, "Producer set property {}:{}", property.first, property.second); - } - - return conf; + KafkaConfigLoader::ProducerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + client_id}; + return KafkaConfigLoader::getProducerConfiguration(*this, params); } void StorageKafka::cleanConsumers() @@ -887,126 +497,6 @@ size_t StorageKafka::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } -void StorageKafka::updateGlobalConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - loadFromConfig(kafka_config, config, collection_name, CONFIG_KAFKA_TAG, topics); - -#if USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) - LOG_WARNING(log, "sasl.kerberos.kinit.cmd configuration parameter is ignored."); - - kafka_config.set("sasl.kerberos.kinit.cmd",""); - kafka_config.set("sasl.kerberos.min.time.before.relogin","0"); - - if (kafka_config.has_property("sasl.kerberos.keytab") && kafka_config.has_property("sasl.kerberos.principal")) - { - String keytab = kafka_config.get("sasl.kerberos.keytab"); - String principal = kafka_config.get("sasl.kerberos.principal"); - LOG_DEBUG(log, "Running KerberosInit"); - try - { - kerberosInit(keytab,principal); - } - catch (const Exception & e) - { - LOG_ERROR(log, "KerberosInit failure: {}", getExceptionMessage(e, false)); - } - LOG_DEBUG(log, "Finished KerberosInit"); - } -#else // USE_KRB5 - if (kafka_config.has_property("sasl.kerberos.keytab") || kafka_config.has_property("sasl.kerberos.principal")) - LOG_WARNING(log, "Ignoring Kerberos-related parameters because ClickHouse was built without krb5 library support."); -#endif // USE_KRB5 - - // No need to add any prefix, messages can be distinguished - kafka_config.set_log_callback( - [this](cppkafka::KafkaHandleBase & handle, int level, const std::string & facility, const std::string & message) - { - auto [poco_level, client_logs_level] = parseSyslogLevel(level); - const auto & kafka_object_config = handle.get_configuration(); - const std::string client_id_key{"client.id"}; - chassert(kafka_object_config.has_property(client_id_key) && "Kafka configuration doesn't have expected client.id set"); - LOG_IMPL( - log, - client_logs_level, - poco_level, - "[client.id:{}] [rdk:{}] {}", - kafka_object_config.get(client_id_key), - facility, - message); - }); - - /// NOTE: statistics should be consumed, otherwise it creates too much - /// entries in the queue, that leads to memory leak and slow shutdown. - if (!kafka_config.has_property("statistics.interval.ms")) - { - // every 3 seconds by default. set to 0 to disable. - kafka_config.set("statistics.interval.ms", "3000"); - } - - // Configure interceptor to change thread name - // - // TODO: add interceptors support into the cppkafka. - // XXX: rdkafka uses pthread_set_name_np(), but glibc-compatibliity overrides it to noop. - { - // This should be safe, since we wait the rdkafka object anyway. - void * self = static_cast(this); - - int status; - - status = rd_kafka_conf_interceptor_add_on_new(kafka_config.get_handle(), - "init", StorageKafkaInterceptors::rdKafkaOnNew, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(log, "Cannot set new interceptor due to {} error", status); - - // cppkafka always copy the configuration - status = rd_kafka_conf_interceptor_add_on_conf_dup(kafka_config.get_handle(), - "init", StorageKafkaInterceptors::rdKafkaOnConfDup, self); - if (status != RD_KAFKA_RESP_ERR_NO_ERROR) - LOG_ERROR(log, "Cannot set dup conf interceptor due to {} error", status); - } -} - -void StorageKafka::updateConsumerConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - loadConsumerConfig(kafka_config, config, collection_name, CONFIG_KAFKA_TAG, topics); -} - -void StorageKafka::updateProducerConfiguration(cppkafka::Configuration & kafka_config) -{ - const auto & config = getContext()->getConfigRef(); - loadProducerConfig(kafka_config, config, collection_name, CONFIG_KAFKA_TAG, topics); -} - -bool StorageKafka::checkDependencies(const StorageID & table_id) -{ - // Check if all dependencies are attached - auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); - if (view_ids.empty()) - return true; - - // Check the dependencies are ready? - for (const auto & view_id : view_ids) - { - auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); - if (!view) - return false; - - // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(view.get()); - if (materialized_view && !materialized_view->tryGetTargetTable()) - return false; - - // Check all its dependencies - if (!checkDependencies(view_id)) - return false; - } - - return true; -} - void StorageKafka::threadFunc(size_t idx) { assert(idx < tasks.size()); @@ -1027,7 +517,7 @@ void StorageKafka::threadFunc(size_t idx) // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) { - if (!checkDependencies(table_id)) + if (!StorageKafkaUtils::checkDependencies(table_id, getContext())) break; LOG_DEBUG(log, "Started streaming to {} attached views", num_views); @@ -1109,7 +599,7 @@ bool StorageKafka::streamToViews() /* allow_materialized */ false, /* no_squash */ true, /* no_destination */ true, - /* async_isnert */ false); + /* async_insert */ false); auto block_io = interpreter.execute(); // Create a stream for each consumer and join them in a union stream @@ -1167,164 +657,4 @@ bool StorageKafka::streamToViews() return some_stream_is_stalled; } - -void registerStorageKafka(StorageFactory & factory) -{ - auto creator_fn = [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - size_t args_count = engine_args.size(); - const bool has_settings = args.storage_def->settings; - - auto kafka_settings = std::make_unique(); - String collection_name; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) - { - for (const auto & setting : kafka_settings->all()) - { - const auto & setting_name = setting.getName(); - if (named_collection->has(setting_name)) - kafka_settings->set(setting_name, named_collection->get(setting_name)); - } - collection_name = assert_cast(args.engine_args[0].get())->name(); - } - - if (has_settings) - { - kafka_settings->loadFromQuery(*args.storage_def); - } - - // Check arguments and settings - #define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ - /* One of the four required arguments is not specified */ \ - if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && \ - !kafka_settings->PAR_NAME.changed) \ - { \ - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,\ - "Required parameter '{}' " \ - "for storage Kafka not specified", \ - #PAR_NAME); \ - } \ - if (args_count >= (ARG_NUM)) \ - { \ - /* The same argument is given in two places */ \ - if (has_settings && \ - kafka_settings->PAR_NAME.changed) \ - { \ - throw Exception(ErrorCodes::BAD_ARGUMENTS, \ - "The argument №{} of storage Kafka " \ - "and the parameter '{}' " \ - "in SETTINGS cannot be specified at the same time", \ - #ARG_NUM, #PAR_NAME); \ - } \ - /* move engine args to settings */ \ - else \ - { \ - if ((EVAL) == 1) \ - { \ - engine_args[(ARG_NUM)-1] = \ - evaluateConstantExpressionAsLiteral( \ - engine_args[(ARG_NUM)-1], \ - args.getLocalContext()); \ - } \ - if ((EVAL) == 2) \ - { \ - engine_args[(ARG_NUM)-1] = \ - evaluateConstantExpressionOrIdentifierAsLiteral( \ - engine_args[(ARG_NUM)-1], \ - args.getLocalContext()); \ - } \ - kafka_settings->PAR_NAME = \ - engine_args[(ARG_NUM)-1]->as().value; \ - } \ - } - - /** Arguments of engine is following: - * - Kafka broker list - * - List of topics - * - Group ID (may be a constant expression with a string result) - * - Message format (string) - * - Row delimiter - * - Schema (optional, if the format supports it) - * - Number of consumers - * - Max block size for background consumption - * - Skip (at least) unreadable messages number - * - Do intermediate commits when the batch consumed and handled - */ - - /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ - /// In case of named collection we already validated the arguments. - if (collection_name.empty()) - { - CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) - CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) - CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) - CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) - } - - #undef CHECK_KAFKA_STORAGE_ARGUMENT - - auto num_consumers = kafka_settings->kafka_num_consumers.value; - auto max_consumers = std::max(getNumberOfPhysicalCPUCores(), 16); - - if (!args.getLocalContext()->getSettingsRef().kafka_disable_num_consumers_limit && num_consumers > max_consumers) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The number of consumers can not be bigger than {}. " - "A single consumer can read any number of partitions. " - "Extra consumers are relatively expensive, " - "and using a lot of them can lead to high memory and CPU usage. " - "To achieve better performance " - "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " - "and ensure you have enough threads " - "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " - "See also https://clickhouse.com/docs/en/integrations/kafka#tuning-performance", max_consumers); - } - else if (num_consumers < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); - } - - if (kafka_settings->kafka_max_block_size.changed && kafka_settings->kafka_max_block_size.value < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_max_block_size can not be lower than 1"); - } - - if (kafka_settings->kafka_poll_max_batch_size.changed && kafka_settings->kafka_poll_max_batch_size.value < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); - } - NamesAndTypesList supported_columns; - for (const auto & column : args.columns) - { - if (column.default_desc.kind == ColumnDefaultKind::Alias) - supported_columns.emplace_back(column.name, column.type); - if (column.default_desc.kind == ColumnDefaultKind::Default && !column.default_desc.expression) - supported_columns.emplace_back(column.name, column.type); - } - // Kafka engine allows only ordinary columns without default expression or alias columns. - if (args.columns.getAll() != supported_columns) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL expressions for columns. " - "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); - } - - return std::make_shared(args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); - }; - - factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); -} - } diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 31e1a6076b6..966d818d675 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -23,7 +23,8 @@ class ReadFromStorageKafka; class StorageSystemKafkaConsumers; class ThreadStatus; -struct StorageKafkaInterceptors; +template +struct KafkaInterceptors; using KafkaConsumerPtr = std::shared_ptr; using ConsumerPtr = std::shared_ptr; @@ -33,7 +34,8 @@ using ConsumerPtr = std::shared_ptr; */ class StorageKafka final : public IStorage, WithContext { - friend struct StorageKafkaInterceptors; + using KafkaInterceptors = KafkaInterceptors; + friend KafkaInterceptors; public: StorageKafka( @@ -133,7 +135,6 @@ private: std::mutex thread_statuses_mutex; std::list> thread_statuses; - SettingsChanges createSettingsAdjustments(); /// Creates KafkaConsumer object without real consumer (cppkafka::Consumer) KafkaConsumerPtr createKafkaConsumer(size_t consumer_number); /// Returns full consumer related configuration, also the configuration @@ -148,33 +149,15 @@ private: std::atomic shutdown_called = false; - // Load Kafka global configuration - // https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md#global-configuration-properties - void updateGlobalConfiguration(cppkafka::Configuration & kafka_config); - // Load Kafka properties from consumer configuration - // NOTE: librdkafka allow to set a consumer property to a producer and vice versa, - // but a warning will be generated e.g: - // "Configuration property session.timeout.ms is a consumer property and - // will be ignored by this producer instance" - void updateConsumerConfiguration(cppkafka::Configuration & kafka_config); - // Load Kafka properties from producer configuration - void updateProducerConfiguration(cppkafka::Configuration & kafka_config); - void threadFunc(size_t idx); size_t getPollMaxBatchSize() const; size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; - static Names parseTopics(String topic_list); - static String getDefaultClientId(const StorageID & table_id_); - bool streamToViews(); - bool checkDependencies(const StorageID & table_id); void cleanConsumers(); - - static VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); }; } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp new file mode 100644 index 00000000000..3574b46e3b0 --- /dev/null +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -0,0 +1,1289 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +namespace CurrentMetrics +{ +// TODO: Add proper metrics, similar to old StorageKafka +extern const Metric KafkaBackgroundReads; +extern const Metric KafkaWrites; +} + +namespace ProfileEvents +{ +extern const Event KafkaBackgroundReads; +extern const Event KafkaMessagesRead; +extern const Event KafkaMessagesFailed; +extern const Event KafkaRowsRead; +extern const Event KafkaWrites; +} + + +namespace DB +{ + +namespace fs = std::filesystem; + +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +extern const int LOGICAL_ERROR; +extern const int REPLICA_ALREADY_EXISTS; +extern const int TABLE_IS_DROPPED; +extern const int NO_ZOOKEEPER; +extern const int REPLICA_IS_ALREADY_ACTIVE; +} + +namespace +{ +constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; +constexpr auto MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS = 15000; +} + +StorageKafka2::StorageKafka2( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & comment, + std::unique_ptr kafka_settings_, + const String & collection_name_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , keeper(getContext()->getZooKeeper()) + , keeper_path(kafka_settings_->kafka_keeper_path.value) + , replica_path(keeper_path + "/replicas/" + kafka_settings_->kafka_replica_name.value) + , kafka_settings(std::move(kafka_settings_)) + , macros_info{.table_id = table_id_} + , topics(StorageKafkaUtils::parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) + , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) + , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) + , client_id( + kafka_settings->kafka_client_id.value.empty() + ? StorageKafkaUtils::getDefaultClientId(table_id_) + : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) + , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) + , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) + , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) + , num_consumers(kafka_settings->kafka_num_consumers.value) + , log(getLogger("StorageKafka2 (" + table_id_.getNameForLogs() + ")")) + , semaphore(0, static_cast(num_consumers)) + , settings_adjustments(StorageKafkaUtils::createSettingsAdjustments(*kafka_settings, schema_name)) + , thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value) + , collection_name(collection_name_) + , active_node_identifier(toString(ServerUUID::get())) +{ + if (kafka_settings->kafka_num_consumers > 1 && !thread_per_consumer) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "With multiple consumers, it is required to use `kafka_thread_per_consumer` setting"); + + if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM) + { + kafka_settings->input_format_allow_errors_num = 0; + kafka_settings->input_format_allow_errors_ratio = 0; + } + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + setVirtuals(StorageKafkaUtils::createVirtuals(kafka_settings->kafka_handle_error_mode)); + + auto task_count = thread_per_consumer ? num_consumers : 1; + for (size_t i = 0; i < task_count; ++i) + { + auto task = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this, i] { threadFunc(i); }); + task->deactivate(); + tasks.emplace_back(std::make_shared(std::move(task))); + } + + const auto first_replica = createTableIfNotExists(); + + if (!first_replica) + createReplica(); + + activating_task = getContext()->getSchedulePool().createTask(log->name() + "(activating task)", [this]() { activateAndReschedule(); }); + activating_task->deactivate(); +} + +void StorageKafka2::partialShutdown() +{ + // This is called in a background task within a catch block, thus this function shouldn't throw + LOG_TRACE(log, "Cancelling streams"); + for (auto & task : tasks) + { + task->stream_cancelled = true; + } + + LOG_TRACE(log, "Waiting for cleanup"); + for (auto & task : tasks) + { + task->holder->deactivate(); + } + is_active = false; +} + +bool StorageKafka2::activate() +{ + LOG_TEST(log, "Activate task"); + if (is_active && !getZooKeeper()->expired()) + { + LOG_TEST(log, "No need to activate"); + return true; + } + + if (!is_active) + { + LOG_WARNING(log, "Table was not active. Will try to activate it"); + } + else if (getZooKeeper()->expired()) + { + LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session"); + partialShutdown(); + } + else + { + UNREACHABLE(); + } + + try + { + setZooKeeper(); + } + catch (const Coordination::Exception &) + { + /// The exception when you try to zookeeper_init usually happens if DNS does not work or the connection with ZK fails + tryLogCurrentException(log, "Failed to establish a new ZK connection. Will try again"); + assert(!is_active); + return false; + } + + if (shutdown_called) + return false; + + auto activate_in_keeper = [this]() + { + try + { + auto zookeeper = getZooKeeper(); + + String is_active_path = fs::path(replica_path) / "is_active"; + zookeeper->deleteEphemeralNodeIfContentMatches(is_active_path, active_node_identifier); + + try + { + /// Simultaneously declare that this replica is active, and update the host. + zookeeper->create(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::Error::ZNODEEXISTS) + throw Exception( + ErrorCodes::REPLICA_IS_ALREADY_ACTIVE, + "Replica {} appears to be already active. If you're sure it's not, " + "try again in a minute or remove znode {}/is_active manually", + replica_path, + replica_path); + + throw; + } + replica_is_active_node = zkutil::EphemeralNodeHolder::existing(is_active_path, *zookeeper); + + return true; + } + catch (const Coordination::Exception & e) + { + replica_is_active_node = nullptr; + LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); + return false; + + } + catch (const Exception & e) + { + replica_is_active_node = nullptr; + if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE) + throw; + + LOG_ERROR(log, "Couldn't start replica: {}. {}", e.what(), DB::getCurrentExceptionMessage(true)); + return false; + } + }; + + if (!activate_in_keeper()) + { + assert(!is_active); + return false; + } + + is_active = true; + + // Start the reader threads + for (auto & task : tasks) + { + task->stream_cancelled = false; + task->holder->activateAndSchedule(); + } + + LOG_DEBUG(log, "Table activated successfully"); + return true; +} + +void StorageKafka2::activateAndReschedule() +{ + if (shutdown_called) + return; + + /// It would be ideal to introduce a setting for this + constexpr static size_t check_period_ms = 60000; + /// In case of any exceptions we want to rerun the this task as fast as possible but we also don't want to keep retrying immediately + /// in a close loop (as fast as tasks can be processed), so we'll retry in between 100 and 10000 ms + const size_t backoff_ms = 100 * ((consecutive_activate_failures + 1) * (consecutive_activate_failures + 2)) / 2; + const size_t next_failure_retry_ms = std::min(size_t{10000}, backoff_ms); + + try + { + bool replica_is_active = activate(); + if (replica_is_active) + { + consecutive_activate_failures = 0; + activating_task->scheduleAfter(check_period_ms); + } + else + { + consecutive_activate_failures++; + activating_task->scheduleAfter(next_failure_retry_ms); + } + } + catch (...) + { + consecutive_activate_failures++; + activating_task->scheduleAfter(next_failure_retry_ms); + + /// We couldn't activate table let's set it into readonly mode if necessary + /// We do this after scheduling the task in case it throws + partialShutdown(); + tryLogCurrentException(log, "Failed to restart the table. Will try again"); + } +} + +void StorageKafka2::assertActive() const +{ + // TODO(antaljanosbenjamin): change LOGICAL_ERROR to something sensible + if (!is_active) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table is not active (replica path: {})", replica_path); +} + + +Pipe StorageKafka2::read( + const Names & /*column_names */, + const StorageSnapshotPtr & /* storage_snapshot */, + SelectQueryInfo & /* query_info */, + ContextPtr /* local_context */, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + size_t /* num_streams */) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Direct read from the new Kafka storage is not implemented"); +} + + +SinkToStoragePtr +StorageKafka2::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) +{ + auto modified_context = Context::createCopy(local_context); + modified_context->applySettingsChanges(settings_adjustments); + + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaWrites}; + ProfileEvents::increment(ProfileEvents::KafkaWrites); + + if (topics.size() > 1) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Can't write to Kafka table with multiple topics!"); + + cppkafka::Configuration conf = getProducerConfiguration(); + + const Settings & settings = getContext()->getSettingsRef(); + size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); + const auto & header = metadata_snapshot->getSampleBlockNonMaterialized(); + + auto producer = std::make_unique( + std::make_shared(conf), topics[0], std::chrono::milliseconds(poll_timeout), shutdown_called, header); + + LOG_TRACE(log, "Kafka producer created"); + + size_t max_rows = max_rows_per_message; + /// Need for backward compatibility. + if (format_name == "Avro" && local_context->getSettingsRef().output_format_avro_rows_in_file.changed) + max_rows = local_context->getSettingsRef().output_format_avro_rows_in_file.value; + return std::make_shared(header, getFormatName(), max_rows, std::move(producer), getName(), modified_context); +} + +void StorageKafka2::startup() +{ + for (size_t i = 0; i < num_consumers; ++i) + { + try + { + consumers.push_back(ConsumerAndAssignmentInfo{.consumer = createConsumer(i), .keeper = getZooKeeper()}); + LOG_DEBUG(log, "Created #{} consumer", num_created_consumers); + ++num_created_consumers; + + consumers.back().consumer->subscribeIfNotSubscribedYet(); + } + catch (const cppkafka::Exception &) + { + tryLogCurrentException(log); + } + } + activating_task->activateAndSchedule(); +} + + +void StorageKafka2::shutdown(bool) +{ + shutdown_called = true; + activating_task->deactivate(); + partialShutdown(); + LOG_TRACE(log, "Closing consumers"); + consumers.clear(); + LOG_TRACE(log, "Consumers closed"); +} + +void StorageKafka2::drop() +{ + dropReplica(); +} + +KafkaConsumer2Ptr StorageKafka2::createConsumer(size_t consumer_number) +{ + // Create a consumer and subscribe to topics + auto consumer_impl = std::make_shared(getConsumerConfiguration(consumer_number)); + consumer_impl->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); + + /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. + chassert((thread_per_consumer || num_consumers == 1) && "StorageKafka2 cannot handle multiple consumers on a single thread"); + auto & stream_cancelled = tasks[consumer_number]->stream_cancelled; + return std::make_shared( + consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_cancelled, topics); + +} + + +cppkafka::Configuration StorageKafka2::getConsumerConfiguration(size_t consumer_number) +{ + KafkaConfigLoader::ConsumerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + group, + num_consumers > 1, + consumer_number, + client_id, + getMaxBlockSize()}; + auto kafka_config = KafkaConfigLoader::getConsumerConfiguration(*this, params); + // It is disabled, because in case of no materialized views are attached, it can cause live memory leak. To enable it, a similar cleanup mechanism must be introduced as for StorageKafka. + kafka_config.set("statistics.interval.ms", "0"); + return kafka_config; +} + +cppkafka::Configuration StorageKafka2::getProducerConfiguration() +{ + KafkaConfigLoader::ProducerConfigParams params{ + {getContext()->getConfigRef(), collection_name, topics, log}, + brokers, + client_id}; + return KafkaConfigLoader::getProducerConfiguration(*this, params); +} + +size_t StorageKafka2::getMaxBlockSize() const +{ + return kafka_settings->kafka_max_block_size.changed ? kafka_settings->kafka_max_block_size.value + : (getContext()->getSettingsRef().max_insert_block_size.value / num_consumers); +} + +size_t StorageKafka2::getPollMaxBatchSize() const +{ + size_t batch_size = kafka_settings->kafka_poll_max_batch_size.changed ? kafka_settings->kafka_poll_max_batch_size.value + : getContext()->getSettingsRef().max_block_size.value; + + return std::min(batch_size, getMaxBlockSize()); +} + +size_t StorageKafka2::getPollTimeoutMillisecond() const +{ + return kafka_settings->kafka_poll_timeout_ms.changed ? kafka_settings->kafka_poll_timeout_ms.totalMilliseconds() + : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); +} + +namespace +{ +const std::string lock_file_name{"lock"}; +const std::string commit_file_name{"committed"}; +const std::string intent_file_name{"intention"}; + +std::optional getNumber(zkutil::ZooKeeper & keeper, const fs::path & path) +{ + std::string result; + if (!keeper.tryGet(path, result)) + return std::nullopt; + + return DB::parse(result); +} +} + +bool StorageKafka2::createTableIfNotExists() +{ + // Heavily based on StorageReplicatedMergeTree::createTableIfNotExists + const auto my_keeper_path = fs::path(keeper_path); + const auto replicas_path = my_keeper_path / "replicas"; + + for (auto i = 0; i < 1000; ++i) + { + if (keeper->exists(replicas_path)) + { + LOG_DEBUG(log, "This table {} is already created, will add new replica", keeper_path); + return false; + } + + /// There are leftovers from incompletely dropped table. + if (keeper->exists(my_keeper_path / "dropped")) + { + /// This condition may happen when the previous drop attempt was not completed + /// or when table is dropped by another replica right now. + /// This is Ok because another replica is definitely going to drop the table. + + LOG_WARNING(log, "Removing leftovers from table {}", keeper_path); + String drop_lock_path = my_keeper_path / "dropped" / "lock"; + Coordination::Error code = keeper->tryCreate(drop_lock_path, "", zkutil::CreateMode::Ephemeral); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + LOG_WARNING(log, "The leftovers from table {} were removed by another replica", keeper_path); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception::fromPath(code, drop_lock_path); + } + else + { + auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *keeper); + if (!removeTableNodesFromZooKeeper(keeper, metadata_drop_lock)) + { + /// Someone is recursively removing table right now, we cannot create new table until old one is removed + continue; + } + } + } + + keeper->createAncestors(keeper_path); + Coordination::Requests ops; + + ops.emplace_back(zkutil::makeCreateRequest(keeper_path, "", zkutil::CreateMode::Persistent)); + + const auto topics_path = my_keeper_path / "topics"; + ops.emplace_back(zkutil::makeCreateRequest(topics_path, "", zkutil::CreateMode::Persistent)); + + for (const auto & topic : topics) + { + LOG_DEBUG(log, "Creating path in keeper for topic {}", topic); + + const auto topic_path = topics_path / topic; + ops.emplace_back(zkutil::makeCreateRequest(topic_path, "", zkutil::CreateMode::Persistent)); + + const auto partitions_path = topic_path / "partitions"; + ops.emplace_back(zkutil::makeCreateRequest(partitions_path, "", zkutil::CreateMode::Persistent)); + } + + // Create the first replica + ops.emplace_back(zkutil::makeCreateRequest(replicas_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent)); + + + Coordination::Responses responses; + const auto code = keeper->tryMulti(ops, responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "It looks like the table {} was created by another replica at the same moment, will retry", keeper_path); + continue; + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + + LOG_INFO(log, "Table {} created successfully ", keeper_path); + + return true; + } + + throw Exception( + ErrorCodes::REPLICA_ALREADY_EXISTS, + "Cannot create table, because it is created concurrently every time or because " + "of wrong zookeeper_path or because of logical error"); +} + + +bool StorageKafka2::removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr keeper_to_use, const zkutil::EphemeralNodeHolder::Ptr & drop_lock) +{ + bool completely_removed = false; + + Strings children; + if (const auto code = keeper_to_use->tryGetChildren(keeper_path, children); code == Coordination::Error::ZNONODE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal. It's a bug"); + + const auto my_keeper_path = fs::path(keeper_path); + for (const auto & child : children) + if (child != "dropped") + keeper_to_use->tryRemoveRecursive(my_keeper_path / child); + + Coordination::Requests ops; + Coordination::Responses responses; + ops.emplace_back(zkutil::makeRemoveRequest(drop_lock->getPath(), -1)); + ops.emplace_back(zkutil::makeRemoveRequest(my_keeper_path / "dropped", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(my_keeper_path, -1)); + const auto code = keeper_to_use->tryMulti(ops, responses, /* check_session_valid */ true); + + if (code == Coordination::Error::ZNONODE) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of replicated table. It's a bug"); + } + else if (code == Coordination::Error::ZNOTEMPTY) + { + LOG_ERROR( + log, + "Table was not completely removed from Keeper, {} still exists and may contain some garbage," + "but someone is removing it right now.", + keeper_path); + } + else if (code != Coordination::Error::ZOK) + { + /// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation. + zkutil::KeeperMultiException::check(code, ops, responses); + } + else + { + drop_lock->setAlreadyRemoved(); + completely_removed = true; + LOG_INFO(log, "Table {} was successfully removed from ZooKeeper", keeper_path); + } + + return completely_removed; +} + +void StorageKafka2::createReplica() +{ + LOG_INFO(log, "Creating replica {}", replica_path); + // TODO: This can cause issues if a new table is created with the same path. To make this work, we should store some metadata + // about the table to be able to identify that the same table is created, not a new one. + const auto code = keeper->tryCreate(replica_path, "", zkutil::CreateMode::Persistent); + + switch (code) + { + case Coordination::Error::ZNODEEXISTS: + LOG_INFO(log, "Replica {} already exists, will try to use it", replica_path); + break; + case Coordination::Error::ZOK: + LOG_INFO(log, "Replica {} created", replica_path); + break; + case Coordination::Error::ZNONODE: + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} was suddenly removed", keeper_path); + default: + throw Coordination::Exception::fromPath(code, replica_path); + } +} + + +void StorageKafka2::dropReplica() +{ + LOG_INFO(log, "Trying to drop replica {}", replica_path); + auto my_keeper = getZooKeeperIfTableShutDown(); + + LOG_INFO(log, "Removing replica {}", replica_path); + + if (!my_keeper->exists(replica_path)) + { + LOG_INFO(log, "Removing replica {} does not exist", replica_path); + return; + } + + my_keeper->tryRemoveChildrenRecursive(replica_path); + + if (my_keeper->tryRemove(replica_path) != Coordination::Error::ZOK) + LOG_ERROR(log, "Replica was not completely removed from Keeper, {} still exists and may contain some garbage.", replica_path); + + /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. + Strings replicas; + if (Coordination::Error::ZOK != my_keeper->tryGetChildren(keeper_path + "/replicas", replicas) || !replicas.empty()) + return; + + LOG_INFO(log, "{} is the last replica, will remove table", replica_path); + + /** At this moment, another replica can be created and we cannot remove the table. + * Try to remove /replicas node first. If we successfully removed it, + * it guarantees that we are the only replica that proceed to remove the table + * and no new replicas can be created after that moment (it requires the existence of /replicas node). + * and table cannot be recreated with new /replicas node on another servers while we are removing data, + * because table creation is executed in single transaction that will conflict with remaining nodes. + */ + + /// Node /dropped works like a lock that protects from concurrent removal of old table and creation of new table. + /// But recursive removal may fail in the middle of operation leaving some garbage in zookeeper_path, so + /// we remove it on table creation if there is /dropped node. Creating thread may remove /dropped node created by + /// removing thread, and it causes race condition if removing thread is not finished yet. + /// To avoid this we also create ephemeral child before starting recursive removal. + /// (The existence of child node does not allow to remove parent node). + Coordination::Requests ops; + Coordination::Responses responses; + fs::path my_keeper_path = keeper_path; + String drop_lock_path = my_keeper_path / "dropped" / "lock"; + ops.emplace_back(zkutil::makeRemoveRequest(my_keeper_path / "replicas", -1)); + ops.emplace_back(zkutil::makeCreateRequest(my_keeper_path / "dropped", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(drop_lock_path, "", zkutil::CreateMode::Ephemeral)); + Coordination::Error code = my_keeper->tryMulti(ops, responses); + + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + LOG_WARNING(log, "Table {} is already started to be removing by another replica right now", replica_path); + } + else if (code == Coordination::Error::ZNOTEMPTY) + { + LOG_WARNING(log, "Another replica was suddenly created, will keep the table {}", replica_path); + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + else + { + auto drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *my_keeper); + LOG_INFO(log, "Removing table {} (this might take several minutes)", keeper_path); + removeTableNodesFromZooKeeper(my_keeper, drop_lock); + } +} + +std::optional +StorageKafka2::lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions) +{ + std::vector topic_partition_paths; + topic_partition_paths.reserve(topic_partitions.size()); + for (const auto & topic_partition : topic_partitions) + topic_partition_paths.emplace_back(getTopicPartitionPath(topic_partition)); + + Coordination::Requests ops; + + static constexpr auto ignore_if_exists = true; + + for (const auto & topic_partition_path : topic_partition_paths) + { + const auto lock_file_path = String(topic_partition_path / lock_file_name); + LOG_TRACE(log, "Creating locking ops for: {}", lock_file_path); + ops.push_back(zkutil::makeCreateRequest(topic_partition_path, "", zkutil::CreateMode::Persistent, ignore_if_exists)); + ops.push_back(zkutil::makeCreateRequest(lock_file_path, kafka_settings->kafka_replica_name.value, zkutil::CreateMode::Ephemeral)); + } + Coordination::Responses responses; + + if (const auto code = keeper_to_use.tryMulti(ops, responses); code != Coordination::Error::ZOK) + { + if (code != Coordination::Error::ZNODEEXISTS) + zkutil::KeeperMultiException::check(code, ops, responses); + + // Possible optimization: check the content of lock files, if we locked them, then we can clean them up and retry to lock them. + return std::nullopt; + } + + // We have the locks, let's gather the information we needed + TopicPartitionLocks locks; + { + auto tp_it = topic_partitions.begin(); + auto path_it = topic_partition_paths.begin(); + for (; tp_it != topic_partitions.end(); ++tp_it, ++path_it) + { + using zkutil::EphemeralNodeHolder; + LockedTopicPartitionInfo lock_info{ + EphemeralNodeHolder::existing(*path_it / lock_file_name, keeper_to_use), + getNumber(keeper_to_use, *path_it / commit_file_name), + getNumber(keeper_to_use, *path_it / intent_file_name)}; + + LOG_TRACE( + log, + "Locked topic partition: {}:{} at offset {} with intent size {}", + tp_it->topic, + tp_it->partition_id, + lock_info.committed_offset.value_or(0), + lock_info.intent_size.value_or(0)); + locks.emplace(TopicPartition(*tp_it), std::move(lock_info)); + } + } + + return locks; +} + + +void StorageKafka2::saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition) +{ + const auto partition_prefix = getTopicPartitionPath(topic_partition); + keeper_to_use.createOrUpdate(partition_prefix / commit_file_name, toString(topic_partition.offset), zkutil::CreateMode::Persistent); + // This is best effort, if it fails we will try to remove in the next round + keeper_to_use.tryRemove(partition_prefix / intent_file_name, -1); + LOG_TEST( + log, "Saved offset {} for topic-partition [{}:{}]", topic_partition.offset, topic_partition.topic, topic_partition.partition_id); +} + +void StorageKafka2::saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent) +{ + LOG_TEST( + log, + "Saving intent of {} for topic-partition [{}:{}] at offset {}", + intent, + topic_partition.topic, + topic_partition.partition_id, + topic_partition.offset); + keeper_to_use.createOrUpdate( + getTopicPartitionPath(topic_partition) / intent_file_name, toString(intent), zkutil::CreateMode::Persistent); +} + + +StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( + KafkaConsumer2 & consumer, + const TopicPartition & topic_partition, + std::optional message_count, + Stopwatch & total_stopwatch, + const ContextPtr & modified_context) +{ + LOG_TEST(log, "Polling consumer"); + PolledBatchInfo batch_info; + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + Block non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()); + auto virtual_header = getVirtualsHeader(); + + // now it's one-time usage InputStream + // one block of the needed size (or with desired flush timeout) is formed in one internal iteration + // otherwise external iteration will reuse that and logic will became even more fuzzy + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto put_error_to_stream = kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM; + + EmptyReadBuffer empty_buf; + auto input_format = FormatFactory::instance().getInput( + getFormatName(), empty_buf, non_virtual_header, modified_context, getMaxBlockSize(), std::nullopt, 1); + + std::optional exception_message; + size_t total_rows = 0; + size_t failed_poll_attempts = 0; + + auto on_error = [&](const MutableColumns & result_columns, Exception & e) + { + ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed); + + if (put_error_to_stream) + { + exception_message = e.message(); + for (const auto & column : result_columns) + { + // read_kafka_message could already push some rows to result_columns + // before exception, we need to fix it. + auto cur_rows = column->size(); + if (cur_rows > total_rows) + column->popBack(cur_rows - total_rows); + + // all data columns will get default value in case of error + column->insertDefault(); + } + + return 1; + } + else + { + e.addMessage( + "while parsing Kafka message (topic: {}, partition: {}, offset: {})'", + consumer.currentTopic(), + consumer.currentPartition(), + consumer.currentOffset()); + throw std::move(e); + } + }; + + StreamingFormatExecutor executor(non_virtual_header, input_format, std::move(on_error)); + + + Poco::Timespan max_execution_time = kafka_settings->kafka_flush_interval_ms.changed + ? kafka_settings->kafka_flush_interval_ms + : getContext()->getSettingsRef().stream_flush_interval_ms; + + const auto check_time_limit = [&max_execution_time, &total_stopwatch]() + { + if (max_execution_time != 0) + { + auto elapsed_ns = total_stopwatch.elapsed(); + + if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) + return false; + } + + return true; + }; + + while (true) + { + size_t new_rows = 0; + exception_message.reset(); + if (auto buf = consumer.consume(topic_partition, message_count)) + { + ProfileEvents::increment(ProfileEvents::KafkaMessagesRead); + new_rows = executor.execute(*buf); + } + + if (new_rows) + { + ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows); + + const auto & header_list = consumer.currentHeaderList(); + + Array headers_names; + Array headers_values; + + if (!header_list.empty()) + { + headers_names.reserve(header_list.size()); + headers_values.reserve(header_list.size()); + for (const auto & header : header_list) + { + headers_names.emplace_back(header.get_name()); + headers_values.emplace_back(static_cast(header.get_value())); + } + } + + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(consumer.currentTopic()); + virtual_columns[1]->insert(consumer.currentKey()); + virtual_columns[2]->insert(consumer.currentOffset()); + virtual_columns[3]->insert(consumer.currentPartition()); + + + auto timestamp_raw = consumer.currentTimestamp(); + if (timestamp_raw) + { + auto ts = timestamp_raw->get_timestamp(); + virtual_columns[4]->insert(std::chrono::duration_cast(ts).count()); + virtual_columns[5]->insert( + DecimalField(std::chrono::duration_cast(ts).count(), 3)); + } + else + { + virtual_columns[4]->insertDefault(); + virtual_columns[5]->insertDefault(); + } + virtual_columns[6]->insert(headers_names); + virtual_columns[7]->insert(headers_values); + if (put_error_to_stream) + { + if (exception_message) + { + virtual_columns[8]->insert(consumer.currentPayload()); + virtual_columns[9]->insert(*exception_message); + } + else + { + virtual_columns[8]->insertDefault(); + virtual_columns[9]->insertDefault(); + } + } + } + + total_rows = total_rows + new_rows; + batch_info.last_offset = consumer.currentOffset(); + } + else if (consumer.isStalled()) + { + ++failed_poll_attempts; + } + else + { + // We came here in case of tombstone (or sometimes zero-length) messages, and it is not something abnormal + // TODO: it seems like in case of put_error_to_stream=true we may need to process those differently + // currently we just skip them with note in logs. + LOG_DEBUG( + log, + "Parsing of message (topic: {}, partition: {}, offset: {}) return no rows.", + consumer.currentTopic(), + consumer.currentPartition(), + consumer.currentOffset()); + } + + if (!consumer.hasMorePolledMessages() + && (total_rows >= getMaxBlockSize() || !check_time_limit() || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS + || consumer.needsOffsetUpdate())) + { + LOG_TRACE( + log, + "Stopped collecting message for current batch. There are {} failed polled attempts, {} total rows and consumer needs " + "offset update is {}", + failed_poll_attempts, + total_rows, + consumer.needsOffsetUpdate()); + break; + } + } + + if (total_rows == 0) + return {}; + + /// MATERIALIZED columns can be added here, but I think + // they are not needed here: + // and it's misleading to use them here, + // as columns 'materialized' that way stays 'ephemeral' + // i.e. will not be stored anywhere + // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. + + auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); + + batch_info.blocks.emplace_back(std::move(result_block)); + return batch_info; +} + +void StorageKafka2::threadFunc(size_t idx) +{ + chassert(idx < tasks.size()); + auto task = tasks[idx]; + std::optional maybe_stall_reason; + try + { + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size(); + if (num_views) + { + auto start_time = std::chrono::steady_clock::now(); + + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!task->stream_cancelled && num_created_consumers > 0) + { + maybe_stall_reason.reset(); + if (!StorageKafkaUtils::checkDependencies(table_id, getContext())) + break; + + LOG_DEBUG(log, "Started streaming to {} attached views", num_views); + + // Exit the loop & reschedule if some stream stalled + if (maybe_stall_reason = streamToViews(idx); maybe_stall_reason.has_value()) + { + LOG_TRACE(log, "Stream stalled."); + break; + } + + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts - start_time); + if (duration.count() > KAFKA_MAX_THREAD_WORK_DURATION_MS) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + break; + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + if (!task->stream_cancelled) + { + // Keeper related problems should be solved relatively fast, it makes sense wait less time + if (maybe_stall_reason.has_value() + && (*maybe_stall_reason == StallReason::KeeperSessionEnded || *maybe_stall_reason == StallReason::CouldNotAcquireLocks)) + task->holder->scheduleAfter(KAFKA_RESCHEDULE_MS / 10); + else + task->holder->scheduleAfter(KAFKA_RESCHEDULE_MS); + } +} + +std::optional StorageKafka2::streamToViews(size_t idx) +{ + // This function is written assuming that each consumer has their own thread. This means once this is changed, this function should be revisited. + // The return values should be revisited, as stalling all consumers because of a single one stalled is not a good idea. + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs()); + + CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; + ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads); + + auto & consumer_info = consumers[idx]; + consumer_info.watch.restart(); + auto & consumer = consumer_info.consumer; + // In case the initial subscribe in startup failed, let's subscribe now + consumer->subscribeIfNotSubscribedYet(); + + // To keep the consumer alive + const auto wait_for_assignment = consumer_info.locks.empty(); + LOG_TRACE(log, "Polling consumer {} for events", idx); + consumer->pollEvents(); + + if (wait_for_assignment) + { + while (nullptr == consumer->getKafkaAssignment() && consumer_info.watch.elapsedMilliseconds() < MAX_TIME_TO_WAIT_FOR_ASSIGNMENT_MS) + consumer->pollEvents(); + LOG_INFO(log, "Consumer has assignment: {}", nullptr == consumer->getKafkaAssignment()); + } + + try + { + if (consumer->needsOffsetUpdate() || consumer_info.locks.empty()) + { + LOG_TRACE(log, "Consumer needs update offset"); + // First release the locks so let other consumers acquire them ASAP + consumer_info.locks.clear(); + consumer_info.topic_partitions.clear(); + + const auto * current_assignment = consumer->getKafkaAssignment(); + if (current_assignment == nullptr) + { + // The consumer lost its assignment and haven't received a new one. + // By returning true this function reports the current consumer as a "stalled" stream, which + LOG_TRACE(log, "No assignment"); + return StallReason::NoAssignment; + } + consumer_info.consume_from_topic_partition_index = 0; + + if (consumer_info.keeper->expired()) + { + consumer_info.keeper = getZooKeeperAndAssertActive(); + LOG_TEST(log, "Got new zookeeper"); + } + + auto maybe_locks = lockTopicPartitions(*consumer_info.keeper, *current_assignment); + + if (!maybe_locks.has_value()) + { + // We couldn't acquire locks, probably some other consumers are still holding them. + LOG_TRACE(log, "Couldn't acquire locks"); + return StallReason::CouldNotAcquireLocks; + } + + consumer_info.locks = std::move(*maybe_locks); + + consumer_info.topic_partitions.reserve(current_assignment->size()); + for (const auto & topic_partition : *current_assignment) + { + TopicPartition topic_partition_copy{topic_partition}; + if (const auto & maybe_committed_offset = consumer_info.locks.at(topic_partition).committed_offset; + maybe_committed_offset.has_value()) + { + topic_partition_copy.offset = *maybe_committed_offset; + } + // in case no saved offset, we will get the offset from Kafka as a best effort. This is important to not to duplicate message when recreating the table. + + consumer_info.topic_partitions.push_back(std::move(topic_partition_copy)); + } + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + } + + if (consumer_info.topic_partitions.empty()) + { + LOG_TRACE(log, "Consumer {} has assignment, but has no partitions, probably because there are more consumers in the consumer group than partitions.", idx); + return StallReason::NoPartitions; + } + LOG_TRACE(log, "Trying to consume from consumer {}", idx); + const auto maybe_rows = streamFromConsumer(consumer_info); + if (maybe_rows.has_value()) + { + const auto milliseconds = consumer_info.watch.elapsedMilliseconds(); + LOG_DEBUG( + log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(*maybe_rows), table_id.getNameForLogs(), milliseconds); + } + else + { + LOG_DEBUG(log, "Couldn't stream any messages"); + return StallReason::NoMessages; + } + } + catch (const zkutil::KeeperException & e) + { + if (Coordination::isHardwareError(e.code)) + { + LOG_INFO(log, "Cleaning up topic-partitions locks because of exception: {}", e.displayText()); + consumer_info.locks.clear(); + activating_task->schedule(); + return StallReason::KeeperSessionEnded; + } + + throw; + } + return {}; +} + + +std::optional StorageKafka2::streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info) +{ + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = getStorageID(); + + auto kafka_context = Context::createCopy(getContext()); + kafka_context->makeQueryContext(); + kafka_context->applySettingsChanges(settings_adjustments); + + // Create a stream for each consumer and join them in a union stream + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter( + insert, + kafka_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_insert */ false); + auto block_io = interpreter.execute(); + + auto & topic_partition = consumer_info.topic_partitions[consumer_info.consume_from_topic_partition_index]; + LOG_TRACE( + log, + "Will fetch {}:{} (consume_from_topic_partition_index is {})", + topic_partition.topic, + topic_partition.partition_id, + consumer_info.consume_from_topic_partition_index); + consumer_info.consume_from_topic_partition_index + = (consumer_info.consume_from_topic_partition_index + 1) % consumer_info.topic_partitions.size(); + + bool needs_offset_reset = true; + SCOPE_EXIT({ + if (!needs_offset_reset) + return; + consumer_info.consumer->updateOffsets(consumer_info.topic_partitions); + }); + auto [blocks, last_read_offset] = pollConsumer( + *consumer_info.consumer, topic_partition, consumer_info.locks[topic_partition].intent_size, consumer_info.watch, kafka_context); + + if (blocks.empty()) + { + LOG_TRACE(log, "Didn't get any messages"); + needs_offset_reset = false; + return std::nullopt; + } + + auto converting_dag = ActionsDAG::makeConvertingActions( + blocks.front().cloneEmpty().getColumnsWithTypeAndName(), + block_io.pipeline.getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + + for (auto & block : blocks) + converting_actions->execute(block); + + // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. + // It will be cancelled on underlying layer (kafka buffer) + + auto & keeper_to_use = *consumer_info.keeper; + auto & lock_info = consumer_info.locks.at(topic_partition); + lock_info.intent_size = last_read_offset - lock_info.committed_offset.value_or(0); + saveIntent(keeper_to_use, topic_partition, *lock_info.intent_size); + std::atomic_size_t rows = 0; + { + block_io.pipeline.complete(Pipe{std::make_shared(std::move(blocks))}); + + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } + lock_info.committed_offset = last_read_offset + 1; + topic_partition.offset = last_read_offset + 1; + saveCommittedOffset(keeper_to_use, topic_partition); + consumer_info.consumer->commit(topic_partition); + lock_info.intent_size.reset(); + needs_offset_reset = false; + + return rows; +} + +void StorageKafka2::setZooKeeper() +{ + std::unique_lock lock{keeper_mutex}; + keeper = getContext()->getZooKeeper(); +} + +zkutil::ZooKeeperPtr StorageKafka2::tryGetZooKeeper() const +{ + std::unique_lock lock{keeper_mutex}; + return keeper; +} + +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeper() const +{ + auto res = tryGetZooKeeper(); + if (!res) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); + return res; +} + +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeperAndAssertActive() const +{ + auto res = getZooKeeper(); + assertActive(); + return res; +} + +zkutil::ZooKeeperPtr StorageKafka2::getZooKeeperIfTableShutDown() const +{ + zkutil::ZooKeeperPtr new_zookeeper = getContext()->getZooKeeper(); + new_zookeeper->sync(keeper_path); + return new_zookeeper; +} + +fs::path StorageKafka2::getTopicPartitionPath(const TopicPartition & topic_partition) +{ + return fs::path(keeper_path) / "topics" / topic_partition.topic / "partitions" / std::to_string(topic_partition.partition_id); +} + +} diff --git a/src/Storages/Kafka/StorageKafka2.h b/src/Storages/Kafka/StorageKafka2.h new file mode 100644 index 00000000000..f85fedb316a --- /dev/null +++ b/src/Storages/Kafka/StorageKafka2.h @@ -0,0 +1,241 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +namespace cppkafka +{ + +class Configuration; + +} + +namespace DB +{ + +template +struct KafkaInterceptors; + +using KafkaConsumer2Ptr = std::shared_ptr; + +/// Implements a Kafka queue table engine that can be used as a persistent queue / buffer, +/// or as a basic building block for creating pipelines with a continuous insertion / ETL. +/// +/// It is similar to the already existing StorageKafka, it instead of storing the offsets +/// in Kafka, its main source of information about offsets is Keeper. On top of the +/// offsets, it also stores the number of messages (intent size) it tried to insert from +/// each topic. By storing the intent sizes it is possible to retry the same batch of +/// messages in case of any errors and giving deduplication a chance to deduplicate +/// blocks. +/// +/// To not complicate things too much, the current implementation makes sure to fetch +/// messages only from a single topic-partition on a single thread at a time by +/// manipulating the queues of librdkafka. By pulling from multiple topic-partitions +/// the order of messages are not guaranteed, therefore they would have different +/// hashes for deduplication. +class StorageKafka2 final : public IStorage, WithContext +{ + using KafkaInterceptors = KafkaInterceptors; + friend KafkaInterceptors; + +public: + StorageKafka2( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & comment, + std::unique_ptr kafka_settings_, + const String & collection_name_); + + std::string getName() const override { return "Kafka"; } + + bool noPushingToViews() const override { return true; } + + void startup() override; + void shutdown(bool is_drop) override; + + void drop() override; + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr + write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; + + /// We want to control the number of rows in a chunk inserted into Kafka + bool prefersLargeBlocks() const override { return false; } + + const auto & getFormatName() const { return format_name; } + + StreamingHandleErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } + +private: + using TopicPartition = KafkaConsumer2::TopicPartition; + using TopicPartitions = KafkaConsumer2::TopicPartitions; + + struct LockedTopicPartitionInfo + { + zkutil::EphemeralNodeHolderPtr lock; + std::optional committed_offset; + std::optional intent_size; + }; + + using TopicPartitionLocks = std::unordered_map< + TopicPartition, + LockedTopicPartitionInfo, + KafkaConsumer2::OnlyTopicNameAndPartitionIdHash, + KafkaConsumer2::OnlyTopicNameAndPartitionIdEquality>; + + struct ConsumerAndAssignmentInfo + { + KafkaConsumer2Ptr consumer; + size_t consume_from_topic_partition_index{0}; + TopicPartitions topic_partitions{}; + zkutil::ZooKeeperPtr keeper; + TopicPartitionLocks locks{}; + Stopwatch watch{CLOCK_MONOTONIC_COARSE}; + }; + + struct PolledBatchInfo + { + BlocksList blocks; + int64_t last_offset; + }; + + // Stream thread + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled{false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } + }; + + enum class AssignmentChange + { + NotChanged, + Updated, + Lost + }; + + // Configuration and state + mutable std::mutex keeper_mutex; + zkutil::ZooKeeperPtr keeper; + String keeper_path; + String replica_path; + std::unique_ptr kafka_settings; + Macros::MacroExpansionInfo macros_info; + const Names topics; + const String brokers; + const String group; + const String client_id; + const String format_name; + const size_t max_rows_per_message; + const String schema_name; + const size_t num_consumers; /// total number of consumers + LoggerPtr log; + Poco::Semaphore semaphore; + const SettingsChanges settings_adjustments; + /// Can differ from num_consumers in case of exception in startup() (or if startup() hasn't been called). + /// In this case we still need to be able to shutdown() properly. + size_t num_created_consumers = 0; /// number of actually created consumers. + std::vector consumers; + std::vector> tasks; + bool thread_per_consumer = false; + /// For memory accounting in the librdkafka threads. + std::mutex thread_statuses_mutex; + std::list> thread_statuses; + /// If named_collection is specified. + String collection_name; + std::atomic shutdown_called = false; + + // Handling replica activation. + std::atomic is_active = false; + zkutil::EphemeralNodeHolderPtr replica_is_active_node; + BackgroundSchedulePool::TaskHolder activating_task; + String active_node_identifier; + UInt64 consecutive_activate_failures = 0; + bool activate(); + void activateAndReschedule(); + void partialShutdown(); + + void assertActive() const; + KafkaConsumer2Ptr createConsumer(size_t consumer_number); + // Returns full consumer related configuration, also the configuration + // contains global kafka properties. + cppkafka::Configuration getConsumerConfiguration(size_t consumer_number); + // Returns full producer related configuration, also the configuration + // contains global kafka properties. + cppkafka::Configuration getProducerConfiguration(); + + void threadFunc(size_t idx); + + size_t getPollMaxBatchSize() const; + size_t getMaxBlockSize() const; + size_t getPollTimeoutMillisecond() const; + + enum class StallReason + { + NoAssignment, + CouldNotAcquireLocks, + NoPartitions, + NoMessages, + KeeperSessionEnded, + }; + + std::optional streamToViews(size_t idx); + + std::optional streamFromConsumer(ConsumerAndAssignmentInfo & consumer_info); + + // Returns true if this is the first replica + bool createTableIfNotExists(); + // Returns true if all of the nodes were cleaned up + bool removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr keeper_to_use, const zkutil::EphemeralNodeHolder::Ptr & drop_lock); + // Creates only the replica in ZooKeeper. Shouldn't be called on the first replica as it is created in createTableIfNotExists + void createReplica(); + void dropReplica(); + + // Takes lock over topic partitions and sets the committed offset in topic_partitions. + std::optional lockTopicPartitions(zkutil::ZooKeeper & keeper_to_use, const TopicPartitions & topic_partitions); + void saveCommittedOffset(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition); + void saveIntent(zkutil::ZooKeeper & keeper_to_use, const TopicPartition & topic_partition, int64_t intent); + + PolledBatchInfo pollConsumer( + KafkaConsumer2 & consumer, + const TopicPartition & topic_partition, + std::optional message_count, + Stopwatch & watch, + const ContextPtr & context); + + void setZooKeeper(); + zkutil::ZooKeeperPtr tryGetZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeperAndAssertActive() const; + zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const; + + + std::filesystem::path getTopicPartitionPath(const TopicPartition & topic_partition); +}; + +} diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp new file mode 100644 index 00000000000..cdc32d775eb --- /dev/null +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -0,0 +1,452 @@ +#include + + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#if USE_KRB5 +# include +#endif // USE_KRB5 + +namespace ProfileEvents +{ +extern const Event KafkaConsumerErrors; +} + +namespace DB +{ + +using namespace std::chrono_literals; + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SUPPORT_IS_DISABLED; +} + + +void registerStorageKafka(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + size_t args_count = engine_args.size(); + const bool has_settings = args.storage_def->settings; + + auto kafka_settings = std::make_unique(); + String collection_name; + if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext())) + { + for (const auto & setting : kafka_settings->all()) + { + const auto & setting_name = setting.getName(); + if (named_collection->has(setting_name)) + kafka_settings->set(setting_name, named_collection->get(setting_name)); + } + collection_name = assert_cast(args.engine_args[0].get())->name(); + } + + if (has_settings) + { + kafka_settings->loadFromQuery(*args.storage_def); + } + +// Check arguments and settings +#define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME, EVAL) \ + /* One of the four required arguments is not specified */ \ + if (args_count < (ARG_NUM) && (ARG_NUM) <= 4 && !kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, \ + "Required parameter '{}' " \ + "for storage Kafka not specified", \ + #PAR_NAME); \ + } \ + if (args_count >= (ARG_NUM)) \ + { \ + /* The same argument is given in two places */ \ + if (has_settings && kafka_settings->PAR_NAME.changed) \ + { \ + throw Exception( \ + ErrorCodes::BAD_ARGUMENTS, \ + "The argument №{} of storage Kafka " \ + "and the parameter '{}' " \ + "in SETTINGS cannot be specified at the same time", \ + #ARG_NUM, \ + #PAR_NAME); \ + } \ + /* move engine args to settings */ \ + else \ + { \ + if constexpr ((EVAL) == 1) \ + { \ + engine_args[(ARG_NUM)-1] = evaluateConstantExpressionAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ + } \ + if constexpr ((EVAL) == 2) \ + { \ + engine_args[(ARG_NUM)-1] \ + = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[(ARG_NUM)-1], args.getLocalContext()); \ + } \ + kafka_settings->PAR_NAME = engine_args[(ARG_NUM)-1]->as().value; \ + } \ + } + + /** Arguments of engine is following: + * - Kafka broker list + * - List of topics + * - Group ID (may be a constant expression with a string result) + * - Message format (string) + * - Row delimiter + * - Schema (optional, if the format supports it) + * - Number of consumers + * - Max block size for background consumption + * - Skip (at least) unreadable messages number + * - Do intermediate commits when the batch consumed and handled + */ + + /* 0 = raw, 1 = evaluateConstantExpressionAsLiteral, 2=evaluateConstantExpressionOrIdentifierAsLiteral */ + /// In case of named collection we already validated the arguments. + if (collection_name.empty()) + { + CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list, 1) + CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id, 2) + CHECK_KAFKA_STORAGE_ARGUMENT(12, kafka_poll_timeout_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(13, kafka_flush_interval_ms, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(14, kafka_thread_per_consumer, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(15, kafka_handle_error_mode, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(16, kafka_commit_on_select, 0) + CHECK_KAFKA_STORAGE_ARGUMENT(17, kafka_max_rows_per_message, 0) + } + +#undef CHECK_KAFKA_STORAGE_ARGUMENT + + auto num_consumers = kafka_settings->kafka_num_consumers.value; + auto max_consumers = std::max(getNumberOfPhysicalCPUCores(), 16); + + if (!args.getLocalContext()->getSettingsRef().kafka_disable_num_consumers_limit && num_consumers > max_consumers) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The number of consumers can not be bigger than {}. " + "A single consumer can read any number of partitions. " + "Extra consumers are relatively expensive, " + "and using a lot of them can lead to high memory and CPU usage. " + "To achieve better performance " + "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " + "and ensure you have enough threads " + "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " + "See also https://clickhouse.com/docs/integrations/kafka/kafka-table-engine#tuning-performance", + max_consumers); + } + else if (num_consumers < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); + } + + if (kafka_settings->kafka_max_block_size.changed && kafka_settings->kafka_max_block_size.value < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_max_block_size can not be lower than 1"); + } + + if (kafka_settings->kafka_poll_max_batch_size.changed && kafka_settings->kafka_poll_max_batch_size.value < 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); + } + NamesAndTypesList supported_columns; + for (const auto & column : args.columns) + { + if (column.default_desc.kind == ColumnDefaultKind::Alias) + supported_columns.emplace_back(column.name, column.type); + if (column.default_desc.kind == ColumnDefaultKind::Default && !column.default_desc.expression) + supported_columns.emplace_back(column.name, column.type); + } + // Kafka engine allows only ordinary columns without default expression or alias columns. + if (args.columns.getAll() != supported_columns) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL expressions for columns. " + "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); + } + + const auto has_keeper_path = kafka_settings->kafka_keeper_path.changed && !kafka_settings->kafka_keeper_path.value.empty(); + const auto has_replica_name = kafka_settings->kafka_replica_name.changed && !kafka_settings->kafka_replica_name.value.empty(); + + if (!has_keeper_path && !has_replica_name) + return std::make_shared( + args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); + + if (!args.getLocalContext()->getSettingsRef().allow_experimental_kafka_offsets_storage_in_keeper && !args.query.attach) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "Storing the Kafka offsets in Keeper is experimental. Set `allow_experimental_kafka_offsets_storage_in_keeper` setting " + "to enable it"); + + if (!has_keeper_path || !has_replica_name) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Either specify both zookeeper path and replica name or none of them"); + + const auto is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + const auto is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY + && DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; + + // UUID macro is only allowed: + // - with Atomic database only with ON CLUSTER queries, otherwise it is easy to misuse: each replica would have separate uuid generated. + // - with Replicated database + // - with attach queries, as those are used on server startup + const auto allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; + + auto context = args.getContext(); + // Unfold {database} and {table} macro on table creation, so table can be renamed. + if (args.mode < LoadingStrictnessLevel::ATTACH) + { + Macros::MacroExpansionInfo info; + /// NOTE: it's not recursive + info.expand_special_macros_only = true; + info.table_id = args.table_id; + // We could probably unfold UUID here too, but let's keep it similar to ReplicatedMergeTree, which doesn't do the unfolding. + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); + + info.level = 0; + kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); + } + + + auto * settings_query = args.storage_def->settings; + chassert(has_settings && "Unexpected settings query in StorageKafka"); + + settings_query->changes.setSetting("kafka_keeper_path", kafka_settings->kafka_keeper_path.value); + settings_query->changes.setSetting("kafka_replica_name", kafka_settings->kafka_replica_name.value); + + // Expand other macros (such as {replica}). We do not expand them on previous step to make possible copying metadata files between replicas. + // Disable expanding {shard} macro, because it can lead to incorrect behavior and it doesn't make sense to shard Kafka tables. + Macros::MacroExpansionInfo info; + info.table_id = args.table_id; + if (is_replicated_database) + { + auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); + info.shard.reset(); + info.replica = getReplicatedDatabaseReplicaName(database); + } + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_keeper_path.value = context->getMacros()->expand(kafka_settings->kafka_keeper_path.value, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + kafka_settings->kafka_replica_name.value = context->getMacros()->expand(kafka_settings->kafka_replica_name.value, info); + + return std::make_shared( + args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name); + }; + + factory.registerStorage( + "Kafka", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + }); +} + +namespace StorageKafkaUtils +{ +Names parseTopics(String topic_list) +{ + Names result; + boost::split(result, topic_list, [](char c) { return c == ','; }); + for (String & topic : result) + boost::trim(topic); + return result; +} + +String getDefaultClientId(const StorageID & table_id) +{ + return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id.database_name, table_id.table_name); +} + +void drainConsumer( + cppkafka::Consumer & consumer, const std::chrono::milliseconds drain_timeout, const LoggerPtr & log, ErrorHandler error_handler) +{ + auto start_time = std::chrono::steady_clock::now(); + cppkafka::Error last_error(RD_KAFKA_RESP_ERR_NO_ERROR); + + while (true) + { + auto msg = consumer.poll(100ms); + if (!msg) + break; + + auto error = msg.get_error(); + + if (error) + { + if (msg.is_eof() || error == last_error) + { + break; + } + else + { + LOG_ERROR(log, "Error during draining: {}", error); + error_handler(error); + } + } + + // i don't stop draining on first error, + // only if it repeats once again sequentially + last_error = error; + + auto ts = std::chrono::steady_clock::now(); + if (std::chrono::duration_cast(ts - start_time) > drain_timeout) + { + LOG_ERROR(log, "Timeout during draining."); + break; + } + } +} + +void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler) +{ + size_t skipped = std::erase_if( + messages, + [&](auto & message) + { + if (auto error = message.get_error()) + { + ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); + LOG_ERROR(log, "Consumer error: {}", error); + error_handler(error); + return true; + } + return false; + }); + + if (skipped) + LOG_ERROR(log, "There were {} messages with an error", skipped); +} + +SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name) +{ + SettingsChanges result; + // Needed for backward compatibility + if (!kafka_settings.input_format_skip_unknown_fields.changed) + { + // Always skip unknown fields regardless of the context (JSON or TSKV) + kafka_settings.input_format_skip_unknown_fields = true; + } + + if (!kafka_settings.input_format_allow_errors_ratio.changed) + { + kafka_settings.input_format_allow_errors_ratio = 0.; + } + + if (!kafka_settings.input_format_allow_errors_num.changed) + { + kafka_settings.input_format_allow_errors_num = kafka_settings.kafka_skip_broken_messages.value; + } + + if (!schema_name.empty()) + result.emplace_back("format_schema", schema_name); + + for (const auto & setting : kafka_settings) + { + const auto & name = setting.getName(); + if (name.find("kafka_") == std::string::npos) + result.emplace_back(name, setting.getValue()); + } + return result; +} + + +bool checkDependencies(const StorageID & table_id, const ContextPtr& context) +{ + // Check if all dependencies are attached + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & view_id : view_ids) + { + auto view = DatabaseCatalog::instance().tryGetTable(view_id, context); + if (!view) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(view.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(view_id, context)) + return false; + } + + return true; +} + + +VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode) +{ + VirtualColumnsDescription desc; + + desc.addEphemeral("_topic", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_key", std::make_shared(), ""); + desc.addEphemeral("_offset", std::make_shared(), ""); + desc.addEphemeral("_partition", std::make_shared(), ""); + desc.addEphemeral("_timestamp", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_timestamp_ms", std::make_shared(std::make_shared(3)), ""); + desc.addEphemeral("_headers.name", std::make_shared(std::make_shared()), ""); + desc.addEphemeral("_headers.value", std::make_shared(std::make_shared()), ""); + + if (handle_error_mode == StreamingHandleErrorMode::STREAM) + { + desc.addEphemeral("_raw_message", std::make_shared(), ""); + desc.addEphemeral("_error", std::make_shared(), ""); + } + + return desc; +} +} +} diff --git a/src/Storages/Kafka/StorageKafkaUtils.h b/src/Storages/Kafka/StorageKafkaUtils.h new file mode 100644 index 00000000000..cc956dde78d --- /dev/null +++ b/src/Storages/Kafka/StorageKafkaUtils.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace Poco +{ +namespace Util +{ + class AbstractConfiguration; +} +} + +namespace DB +{ + +class VirtualColumnsDescription; +struct KafkaSettings; + +namespace StorageKafkaUtils +{ +Names parseTopics(String topic_list); +String getDefaultClientId(const StorageID & table_id); + +using ErrorHandler = std::function; + +void drainConsumer( + cppkafka::Consumer & consumer, + std::chrono::milliseconds drain_timeout, + const LoggerPtr & log, + ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); + +using Messages = std::vector; +void eraseMessageErrors(Messages & messages, const LoggerPtr & log, ErrorHandler error_handler = [](const cppkafka::Error & /*err*/) {}); + +SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const String & schema_name); + +bool checkDependencies(const StorageID & table_id, const ContextPtr& context); + +VirtualColumnsDescription createVirtuals(StreamingHandleErrorMode handle_error_mode); +} +} + +template <> +struct fmt::formatter : fmt::ostream_formatter +{ +}; +template <> +struct fmt::formatter : fmt::ostream_formatter +{ +}; diff --git a/src/Storages/Kafka/parseSyslogLevel.cpp b/src/Storages/Kafka/parseSyslogLevel.cpp index 43630a5001f..828cffc311b 100644 --- a/src/Storages/Kafka/parseSyslogLevel.cpp +++ b/src/Storages/Kafka/parseSyslogLevel.cpp @@ -1,4 +1,5 @@ -#include "parseSyslogLevel.h" +#include + #include /// Must be in a separate compilation unit due to macros overlaps: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 051d52a71cd..a291c39f29d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3359,6 +3359,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER MODIFY REFRESH is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_SQL_SECURITY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ALTER MODIFY SQL SECURITY is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -6268,10 +6272,13 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr } else { - String partition_id = getPartitionIDFromQuery(partition, local_context); + String partition_id; + bool all = partition->as()->all; + if (!all) + partition_id = getPartitionIDFromQuery(partition, local_context); DetachedPartsInfo detached_parts = getDetachedParts(); for (const auto & part_info : detached_parts) - if (part_info.valid_name && part_info.partition_id == partition_id + if (part_info.valid_name && (all || part_info.partition_id == partition_id) && part_info.prefix != "attaching" && part_info.prefix != "deleting") renamed_parts.addPart(part_info.dir_name, "deleting_" + part_info.dir_name, part_info.disk); } diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 57dff958b9a..432659f51f8 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -19,7 +19,7 @@ void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String par Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(const Field & val, Float64 rows) const { if (part_statistics.empty()) - return default_normal_cond_factor * rows; + return default_cond_range_factor * rows; Float64 result = 0; Float64 part_rows = 0; for (const auto & [key, estimator] : part_statistics) @@ -39,13 +39,7 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual { if (part_statistics.empty()) { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val) - return default_unknown_cond_factor * rows; - else if (float_val.value() < - threshold || float_val.value() > threshold) - return default_normal_cond_factor * rows; - else - return default_good_cond_factor * rows; + return default_cond_equal_factor * rows; } Float64 result = 0; Float64 partial_cnt = 0; @@ -149,30 +143,22 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode auto [op, val] = extractBinaryOp(node, col); + if (dummy) + { + if (op == "equals") + return default_cond_equal_factor * total_rows; + else if (op == "less" || op == "lessOrEquals" || op == "greater" || op == "greaterOrEquals") + return default_cond_range_factor * total_rows; + else + return default_unknown_cond_factor * total_rows; + } + if (op == "equals") - { - if (dummy) - { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val || (float_val < - threshold || float_val > threshold)) - return default_normal_cond_factor * total_rows; - else - return default_good_cond_factor * total_rows; - } return estimator.estimateEqual(val, total_rows); - } else if (op == "less" || op == "lessOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateLess(val, total_rows); - } else if (op == "greater" || op == "greaterOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateGreater(val, total_rows); - } else return default_unknown_cond_factor * total_rows; } diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index ce7fdd12e92..269ee9ac6cb 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -38,12 +38,10 @@ private: std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; - static constexpr auto default_good_cond_factor = 0.1; - static constexpr auto default_normal_cond_factor = 0.5; - static constexpr auto default_unknown_cond_factor = 1.0; - /// Conditions like "x = N" are considered good if abs(N) > threshold. - /// This is used to assume that condition is likely to have good selectivity. - static constexpr auto threshold = 2; + /// Used to estimate the selectivity of a condition when there is no statistics. + static constexpr auto default_cond_range_factor = 0.5; + static constexpr auto default_cond_equal_factor = 0.01; + static constexpr auto default_unknown_cond_factor = 1; UInt64 total_rows = 0; std::map column_estimators; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index ade3326288a..fd686c5f0aa 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,14 +1,17 @@ #include +#include +#include +#include +#include #include #include +#include #include #include #include #include #include #include -#include -#include #include "config.h" /// USE_DATASKETCHES @@ -27,33 +30,26 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) +std::optional StatisticsUtils::tryConvertToFloat64(const Field & value, const DataTypePtr & data_type) { - switch (field.getType()) + if (data_type->isValueRepresentedByNumber()) { - case Field::Types::Int64: - return field.get(); - case Field::Types::UInt64: - return field.get(); - case Field::Types::Float64: - return field.get(); - case Field::Types::Int128: - return field.get(); - case Field::Types::UInt128: - return field.get(); - case Field::Types::Int256: - return field.get(); - case Field::Types::UInt256: - return field.get(); - default: - return {}; - } -} + Field value_converted; -std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) -{ - if (field.getType() == Field::Types::String) - return field.get(); + if (isInteger(data_type) && (value.getType() == Field::Types::Float64 || value.getType() == Field::Types::String)) + /// For case val_int32 < 10.5 or val_int32 < '10.5' we should convert 10.5 to Float64. + value_converted = convertFieldToType(value, *DataTypeFactory::instance().get("Float64")); + else + /// We should convert value to the real column data type and then translate it to Float64. + /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. + value_converted = convertFieldToType(value, *data_type); + + if (value_converted.isNull()) + return {}; + + Float64 value_as_float = applyVisitor(FieldVisitorConvertToNumber(), value_converted); + return value_as_float; + } return {}; } @@ -89,21 +85,23 @@ Float64 IStatistics::estimateLess(const Field & /*val*/) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Less-than estimation is not implemented for this type of statistics"); } -/// ------------------------------------- -/// Implementation of the estimation: -/// Note: Each statistics object supports certain types predicates natively, e.g. -/// - TDigest: '< X' (less-than predicates) -/// - Count-min sketches: '= X' (equal predicates) -/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) -/// If multiple statistics objects are available per column, it is sometimes also possible to combine them in a clever way. -/// For that reason, all estimation are performed in a central place (here), and we don't simply pass the predicate to the first statistics -/// object that supports it natively. +/// Notes: +/// - Statistics object usually only support estimation for certain types of predicates, e.g. +/// - TDigest: '< X' (less-than predicates) +/// - Count-min sketches: '= X' (equal predicates) +/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) +/// +/// If multiple statistics objects in a column support estimating a predicate, we want to try statistics in order of descending accuracy +/// (e.g. MinMax statistics are simpler than TDigest statistics and thus worse for estimating 'less' predicates). +/// +/// Sometimes, it is possible to combine multiple statistics in a clever way. For that reason, all estimation are performed in a central +/// place (here), and we don't simply pass the predicate to the first statistics object that supports it natively. Float64 ColumnStatistics::estimateLess(const Field & val) const { if (stats.contains(StatisticsType::TDigest)) return stats.at(StatisticsType::TDigest)->estimateLess(val); - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; + return rows * ConditionSelectivityEstimator::default_cond_range_factor; } Float64 ColumnStatistics::estimateGreater(const Field & val) const @@ -113,8 +111,7 @@ Float64 ColumnStatistics::estimateGreater(const Field & val) const Float64 ColumnStatistics::estimateEqual(const Field & val) const { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (float_val.has_value() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) + if (stats_desc.data_type->isValueRepresentedByNumber() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) { /// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) for every bucket. if (stats.at(StatisticsType::Uniq)->estimateCardinality() < 2048) @@ -124,10 +121,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (stats.contains(StatisticsType::CountMinSketch)) return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val); #endif - if (!float_val.has_value() && (float_val < - ConditionSelectivityEstimator::threshold || float_val > ConditionSelectivityEstimator::threshold)) - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; - else - return rows * ConditionSelectivityEstimator::default_good_cond_factor; + return rows * ConditionSelectivityEstimator::default_cond_equal_factor; } /// ------------------------------------- @@ -204,15 +198,15 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { - registerValidator(StatisticsType::TDigest, tdigestValidator); - registerCreator(StatisticsType::TDigest, tdigestCreator); + registerValidator(StatisticsType::TDigest, tdigestStatisticsValidator); + registerCreator(StatisticsType::TDigest, tdigestStatisticsCreator); - registerValidator(StatisticsType::Uniq, uniqValidator); - registerCreator(StatisticsType::Uniq, uniqCreator); + registerValidator(StatisticsType::Uniq, uniqStatisticsValidator); + registerCreator(StatisticsType::Uniq, uniqStatisticsCreator); #if USE_DATASKETCHES - registerValidator(StatisticsType::CountMinSketch, countMinSketchValidator); - registerCreator(StatisticsType::CountMinSketch, countMinSketchCreator); + registerValidator(StatisticsType::CountMinSketch, countMinSketchStatisticsValidator); + registerCreator(StatisticsType::CountMinSketch, countMinSketchStatisticsCreator); #endif } @@ -222,7 +216,7 @@ MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance() return instance; } -void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const +void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & stats, const DataTypePtr & data_type) const { for (const auto & [type, desc] : stats.types_to_desc) { diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 16f0c67eabd..2a30c0de315 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -14,12 +14,11 @@ namespace DB constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; - struct StatisticsUtils { /// Returns std::nullopt if input Field cannot be converted to a concrete value - static std::optional tryConvertToFloat64(const Field & field); - static std::optional tryConvertToString(const Field & field); + /// - `data_type` is the type of the column on which the statistics object was build on + static std::optional tryConvertToFloat64(const Field & value, const DataTypePtr & data_type); }; /// Statistics describe properties of the values in the column, @@ -87,10 +86,10 @@ class MergeTreeStatisticsFactory : private boost::noncopyable public: static MergeTreeStatisticsFactory & instance(); - void validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const; + void validate(const ColumnStatisticsDescription & stats, const DataTypePtr & data_type) const; - using Validator = std::function; - using Creator = std::function; + using Validator = std::function; + using Creator = std::function; ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const; ColumnsStatistics getMany(const ColumnsDescription & columns) const; diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index e69bbc1515b..0dc01f5fcf0 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -25,8 +25,8 @@ extern const int ILLEGAL_STATISTICS; static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; -StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_) +StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & description, const DataTypePtr & data_type_) + : IStatistics(description) , sketch(num_hashes, num_buckets) , data_type(data_type_) { @@ -84,17 +84,17 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) + DataTypePtr inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber() && !isStringOrFixedString(inner_data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index 6c8b74f8c35..d1de1a3aea5 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -14,7 +14,7 @@ namespace DB class StatisticsCountMinSketch : public IStatistics { public: - StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + StatisticsCountMinSketch(const SingleStatisticsDescription & description, const DataTypePtr & data_type_); Float64 estimateEqual(const Field & val) const override; @@ -31,8 +31,8 @@ private: }; -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 66150e00fdb..285b779036f 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -1,33 +1,29 @@ #include -#include #include +#include namespace DB { namespace ErrorCodes { extern const int ILLEGAL_STATISTICS; -extern const int LOGICAL_ERROR; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_) - : IStatistics(stat_) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_) + : IStatistics(description) + , data_type(data_type_) { } void StatisticsTDigest::update(const ColumnPtr & column) { - size_t rows = column->size(); - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < column->size(); ++row) { - Field field; - column->get(row, field); - - if (field.isNull()) + if (column->isNullAt(row)) continue; - if (auto field_as_float = StatisticsUtils::tryConvertToFloat64(field)) - t_digest.add(*field_as_float, 1); + auto data = column->getFloat64(row); + t_digest.add(data, 1); } } @@ -43,31 +39,31 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountLessThan(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) + return 0; + return t_digest.getCountLessThan(*val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountEqual(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) + return 0; + return t_digest.getCountEqual(*val_as_float); } -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) + DataTypePtr inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber()) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { - return std::make_shared(stat); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 614973e5d8b..5e744fee2ce 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -9,7 +9,7 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_); + explicit StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_); void update(const ColumnPtr & column) override; @@ -21,9 +21,10 @@ public: private: QuantileTDigest t_digest; + DataTypePtr data_type; }; -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 8f60ffcf0b5..07311b5b86d 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type) - : IStatistics(stat_) +StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type) + : IStatistics(description) { arena = std::make_unique(); AggregateFunctionProperties properties; @@ -52,17 +52,17 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) + DataTypePtr inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber()) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index faabde8d47c..1fdcab8bd89 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -10,7 +10,7 @@ namespace DB class StatisticsUniq : public IStatistics { public: - StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); + StatisticsUniq(const SingleStatisticsDescription & description, const DataTypePtr & data_type); ~StatisticsUniq() override; void update(const ColumnPtr & column) override; @@ -27,7 +27,7 @@ private: }; -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp new file mode 100644 index 00000000000..6e8f425f8dc --- /dev/null +++ b/src/Storages/StorageFuzzQuery.cpp @@ -0,0 +1,169 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +ColumnPtr FuzzQuerySource::createColumn() +{ + auto column = ColumnString::create(); + ColumnString::Chars & data_to = column->getChars(); + ColumnString::Offsets & offsets_to = column->getOffsets(); + + offsets_to.resize(block_size); + IColumn::Offset offset = 0; + + auto fuzz_base = query; + size_t row_num = 0; + + while (row_num < block_size) + { + ASTPtr new_query = fuzz_base->clone(); + + auto base_before_fuzz = fuzz_base->formatForErrorMessage(); + fuzzer.fuzzMain(new_query); + auto fuzzed_text = new_query->formatForErrorMessage(); + + if (base_before_fuzz == fuzzed_text) + continue; + + /// AST is too long, will start from the original query. + if (config.max_query_length > 500) + { + fuzz_base = query; + continue; + } + + IColumn::Offset next_offset = offset + fuzzed_text.size() + 1; + data_to.resize(next_offset); + + std::copy(fuzzed_text.begin(), fuzzed_text.end(), &data_to[offset]); + + data_to[offset + fuzzed_text.size()] = 0; + offsets_to[row_num] = next_offset; + + offset = next_offset; + fuzz_base = new_query; + ++row_num; + } + + return column; +} + +StorageFuzzQuery::StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_) + : IStorage(table_id_), config(config_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment_); + setInMemoryMetadata(storage_metadata); +} + +Pipe StorageFuzzQuery::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + storage_snapshot->check(column_names); + + Pipes pipes; + pipes.reserve(num_streams); + + const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); + Block block_header; + for (const auto & name : column_names) + { + const auto & name_type = our_columns.get(name); + MutableColumnPtr column = name_type.type->createColumn(); + block_header.insert({std::move(column), name_type.type, name_type.name}); + } + + const char * begin = config.query.data(); + const char * end = begin + config.query.size(); + + ParserQuery parser(end, false); + auto query = parseQuery(parser, begin, end, "", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + for (UInt64 i = 0; i < num_streams; ++i) + pipes.emplace_back(std::make_shared(max_block_size, block_header, config, query)); + + return Pipe::unitePipes(std::move(pipes)); +} + +StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine_args, ContextPtr local_context) +{ + StorageFuzzQuery::Configuration configuration{}; + + // Supported signatures: + // + // FuzzQuery(query) + // FuzzQuery(query, max_query_length) + // FuzzQuery(query, max_query_length, random_seed) + if (engine_args.empty() || engine_args.size() > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 3 arguments: query, max_query_length, random_seed"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); + configuration.query = std::move(first_arg); + + if (engine_args.size() >= 2) + { + const auto & literal = engine_args[1]->as(); + if (!literal.value.isNull()) + configuration.max_query_length = checkAndGetLiteralArgument(literal, "max_query_length"); + } + + if (engine_args.size() == 3) + { + const auto & literal = engine_args[2]->as(); + if (!literal.value.isNull()) + configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); + } + + return configuration; +} + +void registerStorageFuzzQuery(StorageFactory & factory) +{ + factory.registerStorage( + "FuzzQuery", + [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + + if (engine_args.empty()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage FuzzQuery must have arguments."); + + StorageFuzzQuery::Configuration configuration = StorageFuzzQuery::getConfiguration(engine_args, args.getLocalContext()); + + for (const auto& col : args.columns) + if (col.type->getTypeId() != TypeIndex::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'StorageFuzzQuery' supports only columns of String type, got {}.", col.type->getName()); + + return std::make_shared(args.table_id, args.columns, args.comment, configuration); + }); +} + +} diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h new file mode 100644 index 00000000000..125ef960e74 --- /dev/null +++ b/src/Storages/StorageFuzzQuery.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class NamedCollection; + +class StorageFuzzQuery final : public IStorage +{ +public: + struct Configuration : public StatelessTableEngineConfiguration + { + String query; + UInt64 max_query_length = 500; + UInt64 random_seed = randomSeed(); + }; + + StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_); + + std::string getName() const override { return "FuzzQuery"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + static StorageFuzzQuery::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); + +private: + const Configuration config; +}; + + +class FuzzQuerySource : public ISource +{ +public: + FuzzQuerySource( + UInt64 block_size_, Block block_header_, const StorageFuzzQuery::Configuration & config_, ASTPtr query_) + : ISource(block_header_) + , block_size(block_size_) + , block_header(std::move(block_header_)) + , config(config_) + , query(query_) + , fuzzer(config_.random_seed) + { + } + + String getName() const override { return "FuzzQuery"; } + +protected: + Chunk generate() override + { + Columns columns; + columns.reserve(block_header.columns()); + for (const auto & col : block_header) + { + chassert(col.type->getTypeId() == TypeIndex::String); + columns.emplace_back(createColumn()); + } + + return {std::move(columns), block_size}; + } + +private: + ColumnPtr createColumn(); + + UInt64 block_size; + Block block_header; + + StorageFuzzQuery::Configuration config; + ASTPtr query; + + QueryFuzzer fuzzer; +}; + +} diff --git a/src/Storages/StorageTimeSeries.cpp b/src/Storages/StorageTimeSeries.cpp new file mode 100644 index 00000000000..3ff57aaf3e5 --- /dev/null +++ b/src/Storages/StorageTimeSeries.cpp @@ -0,0 +1,477 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; + extern const int UNEXPECTED_TABLE_ENGINE; +} + + +namespace +{ + namespace fs = std::filesystem; + + /// Loads TimeSeries storage settings from a create query. + std::shared_ptr getTimeSeriesSettingsFromQuery(const ASTCreateQuery & query) + { + auto storage_settings = std::make_shared(); + if (query.storage) + storage_settings->loadFromQuery(*query.storage); + return storage_settings; + } + + /// Creates an inner target table or just makes its storage ID. + /// This function is used by the constructor of StorageTimeSeries to find (or create) its target tables. + StorageID initTarget( + ViewTarget::Kind kind, + const ViewTarget * target_info, + const ContextPtr & context, + const StorageID & time_series_storage_id, + const ColumnsDescription & time_series_columns, + const TimeSeriesSettings & time_series_settings, + LoadingStrictnessLevel mode) + { + StorageID target_table_id = StorageID::createEmpty(); + + bool is_external_target = target_info && !target_info->table_id.empty(); + if (is_external_target) + { + /// A target table is specified. + target_table_id = target_info->table_id; + + if (mode < LoadingStrictnessLevel::ATTACH) + { + /// If it's not an ATTACH request then + /// check that the specified target table has all the required columns. + auto target_table = DatabaseCatalog::instance().getTable(target_table_id, context); + auto target_metadata = target_table->getInMemoryMetadataPtr(); + const auto & target_columns = target_metadata->columns; + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateTargetColumns(kind, target_table_id, target_columns); + } + } + else + { + TimeSeriesInnerTablesCreator inner_tables_creator{context, time_series_storage_id, time_series_columns, time_series_settings}; + auto inner_uuid = target_info ? target_info->inner_uuid : UUIDHelpers::Nil; + + /// An inner target table should be used. + if (mode >= LoadingStrictnessLevel::ATTACH) + { + /// If it's an ATTACH request, then the inner target table must be already created. + target_table_id = inner_tables_creator.getInnerTableID(kind, inner_uuid); + } + else + { + /// Create the inner target table. + auto inner_table_engine = target_info ? target_info->inner_engine : nullptr; + target_table_id = inner_tables_creator.createInnerTable(kind, inner_uuid, inner_table_engine); + } + } + + return target_table_id; + } +} + + +void StorageTimeSeries::normalizeTableDefinition(ASTCreateQuery & create_query, const ContextPtr & local_context) +{ + StorageID time_series_storage_id{create_query.getDatabase(), create_query.getTable()}; + TimeSeriesSettings time_series_settings; + if (create_query.storage) + time_series_settings.loadFromQuery(*create_query.storage); + std::shared_ptr as_create_query; + if (!create_query.as_table.empty()) + { + auto as_database = local_context->resolveDatabase(create_query.as_database); + as_create_query = typeid_cast>( + DatabaseCatalog::instance().getDatabase(as_database)->getCreateTableQuery(create_query.as_table, local_context)); + } + TimeSeriesDefinitionNormalizer normalizer{time_series_storage_id, time_series_settings, as_create_query.get()}; + normalizer.normalize(create_query); +} + + +StorageTimeSeries::StorageTimeSeries( + const StorageID & table_id, + const ContextPtr & local_context, + LoadingStrictnessLevel mode, + const ASTCreateQuery & query, + const ColumnsDescription & columns, + const String & comment) + : IStorage(table_id) + , WithContext(local_context->getGlobalContext()) +{ + if (mode <= LoadingStrictnessLevel::CREATE && !local_context->getSettingsRef().allow_experimental_time_series_table) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental TimeSeries table engine " + "is not enabled (the setting 'allow_experimental_time_series_table')"); + } + + storage_settings = getTimeSeriesSettingsFromQuery(query); + + if (mode < LoadingStrictnessLevel::ATTACH) + { + TimeSeriesColumnsValidator validator{table_id, *storage_settings}; + validator.validateColumns(columns); + } + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns); + if (!comment.empty()) + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + has_inner_tables = false; + + for (auto target_kind : {ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}) + { + const ViewTarget * target_info = query.targets ? query.targets->tryGetTarget(target_kind) : nullptr; + auto & target = targets.emplace_back(); + target.kind = target_kind; + target.table_id = initTarget(target_kind, target_info, local_context, getStorageID(), columns, *storage_settings, mode); + target.is_inner_table = target_info && target_info->table_id.empty(); + + if (target_kind == ViewTarget::Metrics && !target.is_inner_table) + { + auto table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + auto metadata = table->getInMemoryMetadataPtr(); + + for (const auto & column : metadata->columns) + if (column.type->lowCardinality()) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "External metrics table cannot have LowCardnality columns for now."); + } + + has_inner_tables |= target.is_inner_table; + } +} + + +StorageTimeSeries::~StorageTimeSeries() = default; + + +TimeSeriesSettings StorageTimeSeries::getStorageSettings() const +{ + return *getStorageSettingsPtr(); +} + +void StorageTimeSeries::startup() +{ +} + +void StorageTimeSeries::shutdown(bool) +{ +} + + +void StorageTimeSeries::drop() +{ + /// Sync flag and the setting make sense for Atomic databases only. + /// However, with Atomic databases, IStorage::drop() can be called only from a background task in DatabaseCatalog. + /// Running synchronous DROP from that task leads to deadlock. + dropInnerTableIfAny(/* sync= */ false, getContext()); +} + +void StorageTimeSeries::dropInnerTableIfAny(bool sync, ContextPtr local_context) +{ + if (!has_inner_tables) + return; + + for (const auto & target : targets) + { + if (target.is_inner_table && DatabaseCatalog::instance().tryGetTable(target.table_id, getContext())) + { + /// Best-effort to make them work: the inner table name is almost always less than the TimeSeries name (so it's safe to lock DDLGuard). + /// (See the comment in StorageMaterializedView::dropInnerTableIfAny.) + bool may_lock_ddl_guard = getStorageID().getQualifiedName() < target.table_id.getQualifiedName(); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target.table_id, + sync, /* ignore_sync_setting= */ true, may_lock_ddl_guard); + } + } +} + +void StorageTimeSeries::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) +{ + if (!has_inner_tables) + return; + + for (const auto & target : targets) + { + /// We truncate only inner tables here. + if (target.is_inner_table) + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Truncate, getContext(), local_context, target.table_id, /* sync= */ true); + } +} + + +StorageID StorageTimeSeries::getTargetTableId(ViewTarget::Kind target_kind) const +{ + for (const auto & target : targets) + { + if (target.kind == target_kind) + return target.table_id; + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected target kind {}", toString(target_kind)); +} + +StoragePtr StorageTimeSeries::getTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const +{ + return DatabaseCatalog::instance().getTable(getTargetTableId(target_kind), local_context); +} + +StoragePtr StorageTimeSeries::tryGetTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const +{ + return DatabaseCatalog::instance().tryGetTable(getTargetTableId(target_kind), local_context); +} + + +std::optional StorageTimeSeries::totalRows(const Settings & settings) const +{ + UInt64 total_rows = 0; + if (has_inner_tables) + { + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!inner_table) + return std::nullopt; + + auto total_rows_in_inner_table = inner_table->totalRows(settings); + if (!total_rows_in_inner_table) + return std::nullopt; + + total_rows += *total_rows_in_inner_table; + } + } + } + return total_rows; +} + +std::optional StorageTimeSeries::totalBytes(const Settings & settings) const +{ + UInt64 total_bytes = 0; + if (has_inner_tables) + { + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!inner_table) + return std::nullopt; + + auto total_bytes_in_inner_table = inner_table->totalBytes(settings); + if (!total_bytes_in_inner_table) + return std::nullopt; + + total_bytes += *total_bytes_in_inner_table; + } + } + } + return total_bytes; +} + +std::optional StorageTimeSeries::totalBytesUncompressed(const Settings & settings) const +{ + UInt64 total_bytes = 0; + if (has_inner_tables) + { + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!inner_table) + return std::nullopt; + + auto total_bytes_in_inner_table = inner_table->totalBytesUncompressed(settings); + if (!total_bytes_in_inner_table) + return std::nullopt; + + total_bytes += *total_bytes_in_inner_table; + } + } + } + return total_bytes; +} + +Strings StorageTimeSeries::getDataPaths() const +{ + Strings data_paths; + for (const auto & target : targets) + { + auto table = DatabaseCatalog::instance().tryGetTable(target.table_id, getContext()); + if (!table) + continue; + + insertAtEnd(data_paths, table->getDataPaths()); + } + return data_paths; +} + + +bool StorageTimeSeries::optimize( + const ASTPtr & query, + const StorageMetadataPtr &, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + bool cleanup, + ContextPtr local_context) +{ + if (!has_inner_tables) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, "TimeSeries table {} targets only existing tables. Execute the statement directly on it.", + getStorageID().getNameForLogs()); + } + + bool optimized = false; + for (const auto & target : targets) + { + if (target.is_inner_table) + { + auto inner_table = DatabaseCatalog::instance().getTable(target.table_id, local_context); + optimized |= inner_table->optimize(query, inner_table->getInMemoryMetadataPtr(), partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); + } + } + + return optimized; +} + + +void StorageTimeSeries::checkAlterIsPossible(const AlterCommands & commands, ContextPtr) const +{ + for (const auto & command : commands) + { + if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_SQL_SECURITY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); + } +} + +void StorageTimeSeries::alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & table_lock_holder) +{ + IStorage::alter(params, local_context, table_lock_holder); +} + + +void StorageTimeSeries::renameInMemory(const StorageID & /* new_table_id */) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Renaming is not supported by storage {} yet", getName()); +} + + +void StorageTimeSeries::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional &) +{ + for (const auto & target : targets) + { + /// We backup the target table's data only if it's inner. + if (target.is_inner_table) + { + auto table = DatabaseCatalog::instance().getTable(target.table_id, getContext()); + table->backupData(backup_entries_collector, fs::path{data_path_in_backup} / toString(target.kind), {}); + } + } +} + +void StorageTimeSeries::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional &) +{ + for (const auto & target : targets) + { + /// We backup the target table's data only if it's inner. + if (target.is_inner_table) + { + auto table = DatabaseCatalog::instance().getTable(target.table_id, getContext()); + table->restoreDataFromBackup(restorer, fs::path{data_path_in_backup} / toString(target.kind), {}); + } + } +} + + +void StorageTimeSeries::read( + QueryPlan & /* query_plan */, + const Names & /* column_names */, + const StorageSnapshotPtr & /* storage_snapshot */, + SelectQueryInfo & /* query_info */, + ContextPtr /* local_context */, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + size_t /* num_streams */) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SELECT is not supported by storage {} yet", getName()); +} + + +SinkToStoragePtr StorageTimeSeries::write( + const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr /* local_context */, bool /* async_insert */) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "INSERT is not supported by storage {} yet", getName()); +} + + +std::shared_ptr storagePtrToTimeSeries(StoragePtr storage) +{ + if (auto res = typeid_cast>(storage)) + return res; + + throw Exception( + ErrorCodes::UNEXPECTED_TABLE_ENGINE, + "This operation can be executed on a TimeSeries table only, the engine of table {} is not TimeSeries", + storage->getStorageID().getNameForLogs()); +} + +std::shared_ptr storagePtrToTimeSeries(ConstStoragePtr storage) +{ + if (auto res = typeid_cast>(storage)) + return res; + + throw Exception( + ErrorCodes::UNEXPECTED_TABLE_ENGINE, + "This operation can be executed on a TimeSeries table only, the engine of table {} is not TimeSeries", + storage->getStorageID().getNameForLogs()); +} + + +void registerStorageTimeSeries(StorageFactory & factory) +{ + factory.registerStorage("TimeSeries", [](const StorageFactory::Arguments & args) + { + /// Pass local_context here to convey setting to inner tables. + return std::make_shared( + args.table_id, args.getLocalContext(), args.mode, args.query, args.columns, args.comment); + } + , + { + .supports_settings = true, + .supports_schema_inference = true, + }); +} + +} diff --git a/src/Storages/StorageTimeSeries.h b/src/Storages/StorageTimeSeries.h new file mode 100644 index 00000000000..35db3131a0b --- /dev/null +++ b/src/Storages/StorageTimeSeries.h @@ -0,0 +1,111 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +struct TimeSeriesSettings; +using TimeSeriesSettingsPtr = std::shared_ptr; + +/// Represents a table engine to keep time series received by Prometheus protocols. +/// Examples of using this table engine: +/// +/// CREATE TABLE ts ENGINE = TimeSeries() +/// -OR- +/// CREATE TABLE ts ENGINE = TimeSeries() DATA [db].table1 TAGS [db].table2 METRICS [db].table3 +/// -OR- +/// CREATE TABLE ts ENGINE = TimeSeries() DATA ENGINE = MergeTree TAGS ENGINE = ReplacingMergeTree METRICS ENGINE = ReplacingMergeTree +/// -OR- +/// CREATE TABLE ts ( +/// id UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)) CODEC(ZSTD(3)), +/// instance LowCardinality(String), +/// job String +/// ) ENGINE = TimeSeries() +/// SETTINGS tags_to_columns = {'instance': 'instance', 'job': 'job'} +/// DATA ENGINE = ReplicatedMergeTree('zkpath', 'replica'), ... +/// +class StorageTimeSeries final : public IStorage, WithContext +{ +public: + /// Adds missing columns and reorder columns, and also adds inner table engines if they aren't specified. + static void normalizeTableDefinition(ASTCreateQuery & create_query, const ContextPtr & local_context); + + StorageTimeSeries(const StorageID & table_id, const ContextPtr & local_context, LoadingStrictnessLevel mode, + const ASTCreateQuery & query, const ColumnsDescription & columns, const String & comment); + + ~StorageTimeSeries() override; + + std::string getName() const override { return "TimeSeries"; } + + TimeSeriesSettings getStorageSettings() const; + TimeSeriesSettingsPtr getStorageSettingsPtr() const { return storage_settings; } + + StorageID getTargetTableId(ViewTarget::Kind target_kind) const; + StoragePtr getTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const; + StoragePtr tryGetTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const; + + void startup() override; + void shutdown(bool is_drop) override; + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; + + bool optimize( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + bool cleanup, + ContextPtr local_context) override; + + void drop() override; + void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; + + void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; + + void renameInMemory(const StorageID & new_table_id) override; + + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; + void alter(const AlterCommands & params, ContextPtr local_context, AlterLockHolder & table_lock_holder) override; + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + + std::optional totalRows(const Settings & settings) const override; + std::optional totalBytes(const Settings & settings) const override; + std::optional totalBytesUncompressed(const Settings & settings) const override; + Strings getDataPaths() const override; + +private: + TimeSeriesSettingsPtr storage_settings; + + struct Target + { + ViewTarget::Kind kind; + StorageID table_id = StorageID::createEmpty(); + bool is_inner_table; + }; + + std::vector targets; + bool has_inner_tables; +}; + +std::shared_ptr storagePtrToTimeSeries(StoragePtr storage); +std::shared_ptr storagePtrToTimeSeries(ConstStoragePtr storage); + +} diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 9c5c07ae49f..9493d2c97ab 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -31,6 +32,8 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() {"database_shard_name", std::make_shared(), "The name of the `Replicated` database shard (for clusters that belong to a `Replicated` database)."}, {"database_replica_name", std::make_shared(), "The name of the `Replicated` database replica (for clusters that belong to a `Replicated` database)."}, {"is_active", std::make_shared(std::make_shared()), "The status of the Replicated database replica (for clusters that belong to a Replicated database): 1 means 'replica is online', 0 means 'replica is offline', NULL means 'unknown'."}, + {"replication_lag", std::make_shared(std::make_shared()), "The replication lag of the `Replicated` database replica (for clusters that belong to a Replicated database)."}, + {"recovery_time", std::make_shared(std::make_shared()), "The recovery time of the `Replicated` database replica (for clusters that belong to a Replicated database), in milliseconds."}, }; description.setAliases({ @@ -67,6 +70,11 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std const auto & shards_info = cluster->getShardsInfo(); const auto & addresses_with_failover = cluster->getShardsAddresses(); + size_t recovery_time_column_idx = columns_mask.size() - 1, replication_lag_column_idx = columns_mask.size() - 2, is_active_column_idx = columns_mask.size() - 3; + ReplicasInfo replicas_info; + if (replicated && (columns_mask[recovery_time_column_idx] || columns_mask[replication_lag_column_idx] || columns_mask[is_active_column_idx])) + replicas_info = replicated->tryGetReplicasInfo(name_and_cluster.second); + size_t replica_idx = 0; for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { @@ -114,17 +122,46 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std res_columns[res_index++]->insert(address.database_shard_name); if (columns_mask[src_index++]) res_columns[res_index++]->insert(address.database_replica_name); + + /// make sure these three columns remain the last ones if (columns_mask[src_index++]) { - std::vector is_active; - if (replicated) - is_active = replicated->tryGetAreReplicasActive(name_and_cluster.second); - - if (is_active.empty()) + if (replicas_info.empty()) res_columns[res_index++]->insertDefault(); else - res_columns[res_index++]->insert(is_active[replica_idx++]); + { + const auto & replica_info = replicas_info[replica_idx]; + res_columns[res_index++]->insert(replica_info.is_active); + } } + if (columns_mask[src_index++]) + { + if (replicas_info.empty()) + res_columns[res_index++]->insertDefault(); + else + { + const auto & replica_info = replicas_info[replica_idx]; + if (replica_info.replication_lag != std::nullopt) + res_columns[res_index++]->insert(*replica_info.replication_lag); + else + res_columns[res_index++]->insertDefault(); + } + } + if (columns_mask[src_index++]) + { + if (replicas_info.empty()) + res_columns[res_index++]->insertDefault(); + else + { + const auto & replica_info = replicas_info[replica_idx]; + if (replica_info.recovery_time != 0) + res_columns[res_index++]->insert(replica_info.recovery_time); + else + res_columns[res_index++]->insertDefault(); + } + } + + ++replica_idx; } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index f6adb902f43..a5f6d551ca1 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -1,10 +1,10 @@ #pragma once +#include #include #include #include - namespace DB { diff --git a/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp new file mode 100644 index 00000000000..d6d258f5ff6 --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.cpp @@ -0,0 +1,472 @@ +#include + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_REQUEST_PARAMETER; +} + + +namespace +{ + /// Makes an ASTIdentifier for a column of the specified table. + ASTPtr makeASTColumn(const StorageID & table_id, const String & column_name) + { + return std::make_shared(Strings{table_id.database_name, table_id.table_name, column_name}); + } + + /// Makes an AST for condition `data_table.timestamp >= min_timestamp_ms` + ASTPtr makeASTTimestampGreaterOrEquals(Int64 min_timestamp_ms, const StorageID & data_table_id) + { + return makeASTFunction("greaterOrEquals", + makeASTColumn(data_table_id, TimeSeriesColumnNames::Timestamp), + std::make_shared(Field{DecimalField{DateTime64{min_timestamp_ms}, 3}})); + } + + /// Makes an AST for condition `data_table.timestamp <= max_timestamp_ms` + ASTPtr makeASTTimestampLessOrEquals(Int64 max_timestamp_ms, const StorageID & data_table_id) + { + return makeASTFunction("lessOrEquals", + makeASTColumn(data_table_id, TimeSeriesColumnNames::Timestamp), + std::make_shared(Field{DecimalField{DateTime64{max_timestamp_ms}, 3}})); + } + + /// Makes an AST for condition `tags_table.max_time >= min_timestamp_ms` + ASTPtr makeASTMaxTimeGreaterOrEquals(Int64 min_timestamp_ms, const StorageID & tags_table_id) + { + return makeASTFunction("greaterOrEquals", + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MaxTime), + std::make_shared(Field{DecimalField{DateTime64{min_timestamp_ms}, 3}})); + } + + /// Makes an AST for condition `tags_table.min_time <= max_timestamp_ms` + ASTPtr makeASTMinTimeLessOrEquals(Int64 max_timestamp_ms, const StorageID & tags_table_id) + { + return makeASTFunction("lessOrEquals", + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MinTime), + std::make_shared(Field{DecimalField{DateTime64{max_timestamp_ms}, 3}})); + } + + /// Makes an AST for the expression referencing a tag value. + ASTPtr makeASTLabelName(const String & label_name, const StorageID & tags_table_id, const std::unordered_map & column_name_by_tag_name) + { + if (label_name == TimeSeriesTagNames::MetricName) + return makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName); + + auto it = column_name_by_tag_name.find(label_name); + if (it != column_name_by_tag_name.end()) + return makeASTColumn(tags_table_id, it->second); + + /// arrayElement() can be used to extract a value from a Map too. + return makeASTFunction("arrayElement", makeASTColumn(tags_table_id, TimeSeriesColumnNames::Tags), std::make_shared(label_name)); + } + + /// Makes an AST for a label matcher, for example `metric_name == 'value'` or `NOT match(labels['label_name'], 'regexp')`. + ASTPtr makeASTLabelMatcher( + const prometheus::LabelMatcher & label_matcher, + const StorageID & tags_table_id, + const std::unordered_map & column_name_by_tag_name) + { + const auto & label_name = label_matcher.name(); + const auto & label_value = label_matcher.value(); + auto type = label_matcher.type(); + + if (type == prometheus::LabelMatcher::EQ) + return makeASTFunction("equals", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value)); + else if (type == prometheus::LabelMatcher::NEQ) + return makeASTFunction("notEquals", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value)); + else if (type == prometheus::LabelMatcher::RE) + return makeASTFunction("match", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value)); + else if (type == prometheus::LabelMatcher::NRE) + return makeASTFunction("not", makeASTFunction("match", makeASTLabelName(label_name, tags_table_id, column_name_by_tag_name), std::make_shared(label_value))); + else + throw Exception(ErrorCodes::BAD_REQUEST_PARAMETER, "Unexpected type of label matcher: {}", type); + } + + /// Makes an AST checking that tags match a specified label matcher and that timestamp is in range [min_timestamp_ms, max_timestamp_ms]. + ASTPtr makeASTFilterForReadingTimeSeries( + const google::protobuf::RepeatedPtrField & label_matcher, + Int64 min_timestamp_ms, + Int64 max_timestamp_ms, + const StorageID & data_table_id, + const StorageID & tags_table_id, + const std::unordered_map & column_name_by_tag_name, + bool filter_by_min_time_and_max_time) + { + ASTs filters; + + if (min_timestamp_ms) + { + filters.push_back(makeASTTimestampGreaterOrEquals(min_timestamp_ms, data_table_id)); + if (filter_by_min_time_and_max_time) + filters.push_back(makeASTMaxTimeGreaterOrEquals(min_timestamp_ms, tags_table_id)); + } + + if (max_timestamp_ms) + { + filters.push_back(makeASTTimestampLessOrEquals(max_timestamp_ms, data_table_id)); + if (filter_by_min_time_and_max_time) + filters.push_back(makeASTMinTimeLessOrEquals(max_timestamp_ms, tags_table_id)); + } + + for (const auto & label_matcher_element : label_matcher) + filters.push_back(makeASTLabelMatcher(label_matcher_element, tags_table_id, column_name_by_tag_name)); + + if (filters.empty()) + return nullptr; + + return makeASTForLogicalAnd(std::move(filters)); + } + + /// Makes a mapping from a tag name to a column name. + std::unordered_map makeColumnNameByTagNameMap(const TimeSeriesSettings & storage_settings) + { + std::unordered_map res; + const Map & tags_to_columns = storage_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & tag_name = tuple.at(0).safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + res[tag_name] = column_name; + } + return res; + } + + /// The function builds a SELECT query for reading time series: + /// SELECT tags_table.metric_name, tags_table.tag_column1, ... tags_table.tag_columnN, tags_table.tags, + /// groupArray(CAST(data_table.timestamp, 'DateTime64(3)'), CAST(data_table.value, 'Float64')) + /// FROM data_table + /// SEMI LEFT JOIN tag_table ON data_table.id = tags_table.id + /// WHERE filter + /// GROUP BY tags_table.tag_column1, ..., tags_table.tag_columnN, tags_table.tags + ASTPtr buildSelectQueryForReadingTimeSeries( + Int64 min_timestamp_ms, + Int64 max_timestamp_ms, + const google::protobuf::RepeatedPtrField & label_matcher, + const TimeSeriesSettings & time_series_settings, + const StorageID & data_table_id, + const StorageID & tags_table_id) + { + auto select_query = std::make_shared(); + + /// SELECT tags_table.metric_name, any(tags_table.tag_column1), ... any(tags_table.tag_columnN), any(tags_table.tags), + /// groupArray(data_table.timestamp, data_table.value) + { + auto exp_list = std::make_shared(); + + exp_list->children.push_back( + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + exp_list->children.push_back( + makeASTColumn(tags_table_id, column_name)); + } + + exp_list->children.push_back( + makeASTColumn(tags_table_id, TimeSeriesColumnNames::Tags)); + + exp_list->children.push_back( + makeASTFunction("groupArray", + makeASTFunction("tuple", + makeASTFunction("CAST", makeASTColumn(data_table_id, TimeSeriesColumnNames::Timestamp), std::make_shared("DateTime64(3)")), + makeASTFunction("CAST", makeASTColumn(data_table_id, TimeSeriesColumnNames::Value), std::make_shared("Float64"))))); + + select_query->setExpression(ASTSelectQuery::Expression::SELECT, exp_list); + } + + /// FROM data_table + auto tables = std::make_shared(); + + { + auto table = std::make_shared(); + auto table_exp = std::make_shared(); + table_exp->database_and_table_name = std::make_shared(data_table_id); + table_exp->children.emplace_back(table_exp->database_and_table_name); + + table->table_expression = table_exp; + tables->children.push_back(table); + } + + /// SEMI LEFT JOIN tags_table ON data_table.id = tags_table.id + { + auto table = std::make_shared(); + + auto table_join = std::make_shared(); + table_join->kind = JoinKind::Left; + table_join->strictness = JoinStrictness::Semi; + + table_join->on_expression = makeASTFunction("equals", makeASTColumn(data_table_id, TimeSeriesColumnNames::ID), makeASTColumn(tags_table_id, TimeSeriesColumnNames::ID)); + table->table_join = table_join; + + auto table_exp = std::make_shared(); + table_exp->database_and_table_name = std::make_shared(tags_table_id); + table_exp->children.emplace_back(table_exp->database_and_table_name); + + table->table_expression = table_exp; + tables->children.push_back(table); + + select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables); + } + + auto column_name_by_tag_name = makeColumnNameByTagNameMap(time_series_settings); + + /// WHERE + if (auto where = makeASTFilterForReadingTimeSeries(label_matcher, min_timestamp_ms, max_timestamp_ms, data_table_id, tags_table_id, + column_name_by_tag_name, time_series_settings.filter_by_min_time_and_max_time)) + { + select_query->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where)); + } + + /// GROUP BY tags_table.metric_name, tags_table.tag_column1, ..., tags_table.tag_columnN, tags_table.tags + { + auto exp_list = std::make_shared(); + + exp_list->children.push_back( + makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + exp_list->children.push_back( + makeASTColumn(tags_table_id, column_name)); + } + + exp_list->children.push_back(makeASTColumn(tags_table_id, TimeSeriesColumnNames::Tags)); + + select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, exp_list); + } + + return select_query; + } + + /// Sorts a list of pairs {tag_name, tag_value} by tag name. + void sortLabelsByName(std::vector> & labels) + { + auto less_by_label_name = [](const std::pair & left, const std::pair & right) + { + return left.first < right.first; + }; + std::sort(labels.begin(), labels.end(), less_by_label_name); + } + + /// Sorts a list of pairs {timestamp, value} by timestamp. + void sortTimeSeriesByTimestamp(std::vector> & time_series) + { + auto less_by_timestamp = [](const std::pair & left, const std::pair & right) + { + return left.first < right.first; + }; + std::sort(time_series.begin(), time_series.end(), less_by_timestamp); + } + + /// Converts a block generated by the SELECT query for converting time series to the protobuf format. + void convertBlockToProtobuf( + Block && block, + google::protobuf::RepeatedPtrField & out_time_series, + const StorageID & time_series_storage_id, + const TimeSeriesSettings & time_series_settings) + { + size_t num_rows = block.rows(); + if (!num_rows) + return; + + size_t column_index = 0; + + /// We analyze columns sequentially. + auto get_next_column_with_type = [&] -> const ColumnWithTypeAndName & { return block.getByPosition(column_index++); }; + auto get_next_column = [&] -> const IColumn & { return *(get_next_column_with_type().column); }; + + /// Column "metric_name". + const auto & metric_name_column_with_type = get_next_column_with_type(); + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateColumnForMetricName(metric_name_column_with_type); + const auto & metric_name_column = *metric_name_column_with_type.column; + + /// Columns corresponding to specific tags specified in the "tags_to_columns" setting. + std::unordered_map column_by_tag_name; + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & tag_name = tuple.at(0).safeGet(); + const auto & column_with_type = get_next_column_with_type(); + validator.validateColumnForTagValue(column_with_type); + const auto & column = *column_with_type.column; + column_by_tag_name[tag_name] = &column; + } + + /// Column "tags". + const auto & tags_column_with_type = get_next_column_with_type(); + validator.validateColumnForTagsMap(tags_column_with_type); + const auto & tags_column = checkAndGetColumn(*tags_column_with_type.column); + const auto & tags_names = tags_column.getNestedData().getColumn(0); + const auto & tags_values = tags_column.getNestedData().getColumn(1); + const auto & tags_offsets = tags_column.getNestedColumn().getOffsets(); + + /// Column containing time series: groupArray(CAST(data_table.timestamp, 'DateTime64(3)'), CAST(data_table.value, 'Float64')) + const auto & time_series_column = checkAndGetColumn(get_next_column()); + const auto & time_series_timestamps = checkAndGetColumn>(checkAndGetColumn(time_series_column.getData()).getColumn(0)); + const auto & time_series_values = checkAndGetColumn(checkAndGetColumn(time_series_column.getData()).getColumn(1)); + const auto & time_series_offsets = time_series_column.getOffsets(); + + /// We will sort labels lexicographically and time series by timestamp before sending them to a client. + std::vector> labels; + std::vector> time_series; + + for (size_t i = 0; i != num_rows; ++i) + { + /// Collect labels. + size_t num_labels = 1; /* 1 for a metric name */ + + for (const auto & [_, column] : column_by_tag_name) + { + if (!column->isNullAt(i) && !column->getDataAt(i).empty()) + ++num_labels; + } + + size_t tags_start_offset = tags_offsets[i - 1]; + size_t tags_end_offset = tags_offsets[i]; + num_labels += tags_end_offset - tags_start_offset; + + labels.clear(); + labels.reserve(num_labels); + + labels.emplace_back(TimeSeriesTagNames::MetricName, metric_name_column.getDataAt(i)); + + for (const auto & [tag_name, column] : column_by_tag_name) + { + if (!column->isNullAt(i) && !column->getDataAt(i).empty()) + labels.emplace_back(tag_name, column->getDataAt(i)); + } + + for (size_t j = tags_start_offset; j != tags_end_offset; ++j) + { + std::string_view tag_name{tags_names.getDataAt(j)}; + std::string_view tag_value{tags_values.getDataAt(j)}; + labels.emplace_back(tag_name, tag_value); + } + + /// Sort labels. + sortLabelsByName(labels); + + /// Collect time series. + size_t time_series_start_offset = time_series_offsets[i - 1]; + size_t time_series_end_offset = time_series_offsets[i]; + size_t num_time_series = time_series_end_offset - time_series_start_offset; + + time_series.clear(); + time_series.reserve(num_time_series); + + for (size_t j = time_series_start_offset; j != time_series_end_offset; ++j) + time_series.emplace_back(time_series_timestamps.getElement(j), time_series_values.getElement(j)); + + /// Sort time series. + sortTimeSeriesByTimestamp(time_series); + + /// Prepare a result. + auto & new_time_series = *out_time_series.Add(); + + for (const auto & [label_name, label_value] : labels) + { + auto & new_label = *new_time_series.add_labels(); + new_label.set_name(label_name); + new_label.set_value(label_value); + } + + for (const auto & [timestamp, value] : time_series) + { + auto & new_sample = *new_time_series.add_samples(); + new_sample.set_timestamp(timestamp); + new_sample.set_value(value); + } + } + } +} + + +PrometheusRemoteReadProtocol::PrometheusRemoteReadProtocol(ConstStoragePtr time_series_storage_, const ContextPtr & context_) + : WithContext{context_} + , time_series_storage(storagePtrToTimeSeries(time_series_storage_)) + , log(getLogger("PrometheusRemoteReadProtocol")) +{ +} + +PrometheusRemoteReadProtocol::~PrometheusRemoteReadProtocol() = default; + +void PrometheusRemoteReadProtocol::readTimeSeries(google::protobuf::RepeatedPtrField & out_time_series, + Int64 start_timestamp_ms, + Int64 end_timestamp_ms, + const google::protobuf::RepeatedPtrField & label_matcher, + const prometheus::ReadHints &) +{ + out_time_series.Clear(); + + auto time_series_storage_id = time_series_storage->getStorageID(); + auto time_series_settings = time_series_storage->getStorageSettingsPtr(); + auto data_table_id = time_series_storage->getTargetTableId(ViewTarget::Data); + auto tags_table_id = time_series_storage->getTargetTableId(ViewTarget::Tags); + + ASTPtr select_query = buildSelectQueryForReadingTimeSeries( + start_timestamp_ms, end_timestamp_ms, label_matcher, *time_series_settings, data_table_id, tags_table_id); + + LOG_TRACE(log, "{}: Executing query {}", + time_series_storage_id.getNameForLogs(), select_query); + + InterpreterSelectQuery interpreter(select_query, getContext(), SelectQueryOptions{}); + BlockIO io = interpreter.execute(); + PullingPipelineExecutor executor(io.pipeline); + + Block block; + while (executor.pull(block)) + { + LOG_TRACE(log, "{}: Pulled block with {} columns and {} rows", + time_series_storage_id.getNameForLogs(), block.columns(), block.rows()); + + if (block) + convertBlockToProtobuf(std::move(block), out_time_series, time_series_storage_id, *time_series_settings); + } + + LOG_TRACE(log, "{}: {} time series read", + time_series_storage_id.getNameForLogs(), out_time_series.size()); +} + +} + +#endif diff --git a/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h new file mode 100644 index 00000000000..e10e1f8c8cf --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteReadProtocol.h @@ -0,0 +1,36 @@ +#pragma once + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#include +#include +#include + + +namespace DB +{ +class StorageTimeSeries; + +/// Helper class to support the prometheus remote read protocol. +class PrometheusRemoteReadProtocol : public WithContext +{ +public: + PrometheusRemoteReadProtocol(ConstStoragePtr time_series_storage_, const ContextPtr & context_); + ~PrometheusRemoteReadProtocol(); + + /// Reads time series to send to client by remote read protocol. + void readTimeSeries(google::protobuf::RepeatedPtrField & out_time_series, + Int64 start_timestamp_ms, + Int64 end_timestamp_ms, + const google::protobuf::RepeatedPtrField & label_matcher, + const prometheus::ReadHints & read_hints); + +private: + std::shared_ptr time_series_storage; + Poco::LoggerPtr log; +}; + +} + +#endif diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp new file mode 100644 index 00000000000..702d058ee79 --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.cpp @@ -0,0 +1,601 @@ +#include + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TIME_SERIES_TAGS; + extern const int ILLEGAL_COLUMN; +} + + +namespace +{ + /// Checks that a specified set of labels is sorted and has no duplications, and there is one label named "__name__". + void checkLabels(const ::google::protobuf::RepeatedPtrField<::prometheus::Label> & labels) + { + bool metric_name_found = false; + for (size_t i = 0; i != static_cast(labels.size()); ++i) + { + const auto & label = labels[static_cast(i)]; + const auto & label_name = label.name(); + const auto & label_value = label.value(); + + if (label_name.empty()) + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Label name should not be empty"); + if (label_value.empty()) + continue; /// Empty label value is treated like the label doesn't exist. + + if (label_name == TimeSeriesTagNames::MetricName) + metric_name_found = true; + + if (i) + { + /// Check that labels are sorted. + const auto & previous_label_name = labels[static_cast(i - 1)].name(); + if (label_name <= previous_label_name) + { + if (label_name == previous_label_name) + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Found duplicate label {}", label_name); + else + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Label names are not sorted in lexicographical order ({} > {})", + previous_label_name, label_name); + } + } + } + + if (!metric_name_found) + throw Exception(ErrorCodes::ILLEGAL_TIME_SERIES_TAGS, "Metric name (label {}) not found", TimeSeriesTagNames::MetricName); + } + + /// Finds the description of an insertable column in the list. + const ColumnDescription & getInsertableColumnDescription(const ColumnsDescription & columns, const String & column_name, const StorageID & time_series_storage_id) + { + const ColumnDescription * column = columns.tryGet(column_name); + if (!column || ((column->default_desc.kind != ColumnDefaultKind::Default) && (column->default_desc.kind != ColumnDefaultKind::Ephemeral))) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{}: Column {} {}", + time_series_storage_id.getNameForLogs(), column_name, column ? "non-insertable" : "doesn't exist"); + } + return *column; + } + + /// Calculates the identifier of each time series in "tags_block" using the default expression for the "id" column, + /// and adds column "id" with the results to "tags_block". + IColumn & calculateId(const ContextPtr & context, const ColumnDescription & id_column_description, Block & tags_block) + { + auto blocks = std::make_shared(); + blocks->push_back(tags_block); + + auto header = tags_block.cloneEmpty(); + auto pipe = Pipe(std::make_shared(blocks, header)); + + Block header_with_id; + const auto & id_name = id_column_description.name; + auto id_type = id_column_description.type; + header_with_id.insert(ColumnWithTypeAndName{id_type, id_name}); + + auto adding_missing_defaults_dag = addMissingDefaults( + pipe.getHeader(), + header_with_id.getNamesAndTypesList(), + ColumnsDescription{id_column_description}, + context); + + auto adding_missing_defaults_actions = std::make_shared(std::move(adding_missing_defaults_dag)); + pipe.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, adding_missing_defaults_actions); + }); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + header_with_id.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto actions = std::make_shared( + std::move(convert_actions_dag), + ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); + pipe.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header, actions); + }); + + QueryPipeline pipeline{std::move(pipe)}; + PullingPipelineExecutor executor{pipeline}; + + MutableColumnPtr id_column; + + Block block_from_executor; + while (executor.pull(block_from_executor)) + { + if (block_from_executor) + { + MutableColumnPtr id_column_part = block_from_executor.getByName(id_name).column->assumeMutable(); + if (id_column) + id_column->insertRangeFrom(*id_column_part, 0, id_column_part->size()); + else + id_column = std::move(id_column_part); + } + } + + if (!id_column) + id_column = id_type->createColumn(); + + IColumn & id_column_ref = *id_column; + tags_block.insert(0, ColumnWithTypeAndName{std::move(id_column), id_type, id_name}); + return id_column_ref; + } + + /// Converts a timestamp in milliseconds to a DateTime64 with a specified scale. + DateTime64 scaleTimestamp(Int64 timestamp_ms, UInt32 scale) + { + if (scale == 3) + return timestamp_ms; + else if (scale > 3) + return timestamp_ms * DecimalUtils::scaleMultiplier(scale - 3); + else + return timestamp_ms / DecimalUtils::scaleMultiplier(3 - scale); + } + + /// Finds min time and max time in a time series. + std::pair findMinTimeAndMaxTime(const google::protobuf::RepeatedPtrField & samples) + { + chassert(!samples.empty()); + Int64 min_time = std::numeric_limits::max(); + Int64 max_time = std::numeric_limits::min(); + for (const auto & sample : samples) + { + Int64 timestamp = sample.timestamp(); + if (timestamp < min_time) + min_time = timestamp; + if (timestamp > max_time) + max_time = timestamp; + } + return {min_time, max_time}; + } + + struct BlocksToInsert + { + std::vector> blocks; + }; + + /// Converts time series from the protobuf format to prepared blocks for inserting into target tables. + BlocksToInsert toBlocks(const google::protobuf::RepeatedPtrField & time_series, + const ContextPtr & context, + const StorageID & time_series_storage_id, + const StorageInMemoryMetadata & time_series_storage_metadata, + const TimeSeriesSettings & time_series_settings) + { + size_t num_tags_rows = time_series.size(); + + size_t num_data_rows = 0; + for (const auto & element : time_series) + num_data_rows += element.samples_size(); + + if (!num_data_rows) + return {}; /// Nothing to insert into target tables. + + /// Column types must be extracted from the target tables' metadata. + const auto & columns_description = time_series_storage_metadata.columns; + + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + return getInsertableColumnDescription(columns_description, column_name, time_series_storage_id); + }; + + /// We're going to prepare two blocks - one for the "data" table, and one for the "tags" table. + Block data_block, tags_block; + + auto make_column_for_data_block = [&](const ColumnDescription & column_description) -> IColumn & + { + auto column = column_description.type->createColumn(); + column->reserve(num_data_rows); + auto * column_ptr = column.get(); + data_block.insert(ColumnWithTypeAndName{std::move(column), column_description.type, column_description.name}); + return *column_ptr; + }; + + auto make_column_for_tags_block = [&](const ColumnDescription & column_description) -> IColumn & + { + auto column = column_description.type->createColumn(); + column->reserve(num_tags_rows); + auto * column_ptr = column.get(); + tags_block.insert(ColumnWithTypeAndName{std::move(column), column_description.type, column_description.name}); + return *column_ptr; + }; + + /// Create columns. + + /// Column "id". + const auto & id_description = get_column_description(TimeSeriesColumnNames::ID); + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateColumnForID(id_description); + auto & id_column_in_data_table = make_column_for_data_block(id_description); + + /// Column "timestamp". + const auto & timestamp_description = get_column_description(TimeSeriesColumnNames::Timestamp); + UInt32 timestamp_scale; + validator.validateColumnForTimestamp(timestamp_description, timestamp_scale); + auto & timestamp_column = make_column_for_data_block(timestamp_description); + + /// Column "value". + const auto & value_description = get_column_description(TimeSeriesColumnNames::Value); + validator.validateColumnForValue(value_description); + auto & value_column = make_column_for_data_block(value_description); + + /// Column "metric_name". + const auto & metric_name_description = get_column_description(TimeSeriesColumnNames::MetricName); + validator.validateColumnForMetricName(metric_name_description); + auto & metric_name_column = make_column_for_tags_block(metric_name_description); + + /// Columns we should check explicitly that they're filled after filling each row. + std::vector columns_to_fill_in_tags_table; + + /// Columns corresponding to specific tags specified in the "tags_to_columns" setting. + std::unordered_map columns_by_tag_name; + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & tag_name = tuple.at(0).safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + const auto & column_description = get_column_description(column_name); + validator.validateColumnForTagValue(column_description); + auto & column = make_column_for_tags_block(column_description); + columns_by_tag_name[tag_name] = &column; + columns_to_fill_in_tags_table.emplace_back(&column); + } + + /// Column "tags". + const auto & tags_description = get_column_description(TimeSeriesColumnNames::Tags); + validator.validateColumnForTagsMap(tags_description); + auto & tags_column = typeid_cast(make_column_for_tags_block(tags_description)); + IColumn & tags_names = tags_column.getNestedData().getColumn(0); + IColumn & tags_values = tags_column.getNestedData().getColumn(1); + auto & tags_offsets = tags_column.getNestedColumn().getOffsets(); + + /// Column "all_tags". + IColumn * all_tags_names = nullptr; + IColumn * all_tags_values = nullptr; + IColumn::Offsets * all_tags_offsets = nullptr; + if (time_series_settings.use_all_tags_column_to_generate_id) + { + const auto & all_tags_description = get_column_description(TimeSeriesColumnNames::AllTags); + validator.validateColumnForTagsMap(all_tags_description); + auto & all_tags_column = typeid_cast(make_column_for_tags_block(all_tags_description)); + all_tags_names = &all_tags_column.getNestedData().getColumn(0); + all_tags_values = &all_tags_column.getNestedData().getColumn(1); + all_tags_offsets = &all_tags_column.getNestedColumn().getOffsets(); + } + + /// Columns "min_time" and "max_time". + IColumn * min_time_column = nullptr; + IColumn * max_time_column = nullptr; + UInt32 min_time_scale = 0; + UInt32 max_time_scale = 0; + if (time_series_settings.store_min_time_and_max_time) + { + const auto & min_time_description = get_column_description(TimeSeriesColumnNames::MinTime); + const auto & max_time_description = get_column_description(TimeSeriesColumnNames::MaxTime); + validator.validateColumnForTimestamp(min_time_description, min_time_scale); + validator.validateColumnForTimestamp(max_time_description, max_time_scale); + min_time_column = &make_column_for_tags_block(min_time_description); + max_time_column = &make_column_for_tags_block(max_time_description); + columns_to_fill_in_tags_table.emplace_back(min_time_column); + columns_to_fill_in_tags_table.emplace_back(max_time_column); + } + + /// Prepare a block for inserting into the "tags" table. + size_t current_row_in_tags = 0; + for (size_t i = 0; i != static_cast(time_series.size()); ++i) + { + const auto & element = time_series[static_cast(i)]; + if (!element.samples_size()) + continue; + + const auto & labels = element.labels(); + checkLabels(labels); + + for (size_t j = 0; j != static_cast(labels.size()); ++j) + { + const auto & label = labels[static_cast(j)]; + const auto & tag_name = label.name(); + const auto & tag_value = label.value(); + + if (tag_name == TimeSeriesTagNames::MetricName) + { + metric_name_column.insertData(tag_value.data(), tag_value.length()); + } + else + { + if (time_series_settings.use_all_tags_column_to_generate_id) + { + all_tags_names->insertData(tag_name.data(), tag_name.length()); + all_tags_values->insertData(tag_value.data(), tag_value.length()); + } + + auto it = columns_by_tag_name.find(tag_name); + bool has_column_for_tag_value = (it != columns_by_tag_name.end()); + if (has_column_for_tag_value) + { + auto * column = it->second; + column->insertData(tag_value.data(), tag_value.length()); + } + else + { + tags_names.insertData(tag_name.data(), tag_name.length()); + tags_values.insertData(tag_value.data(), tag_value.length()); + } + } + } + + tags_offsets.push_back(tags_names.size()); + + if (time_series_settings.use_all_tags_column_to_generate_id) + all_tags_offsets->push_back(all_tags_names->size()); + + if (time_series_settings.store_min_time_and_max_time) + { + auto [min_time, max_time] = findMinTimeAndMaxTime(element.samples()); + min_time_column->insert(scaleTimestamp(min_time, min_time_scale)); + max_time_column->insert(scaleTimestamp(max_time, max_time_scale)); + } + + for (auto * column : columns_to_fill_in_tags_table) + { + if (column->size() == current_row_in_tags) + column->insertDefault(); + } + + ++current_row_in_tags; + } + + /// Calculate an identifier for each time series, make a new column from those identifiers, and add it to "tags_block". + auto & id_column_in_tags_table = calculateId(context, columns_description.get(TimeSeriesColumnNames::ID), tags_block); + + /// Prepare a block for inserting to the "data" table. + current_row_in_tags = 0; + for (size_t i = 0; i != static_cast(time_series.size()); ++i) + { + const auto & element = time_series[static_cast(i)]; + if (!element.samples_size()) + continue; + + id_column_in_data_table.insertManyFrom(id_column_in_tags_table, current_row_in_tags, element.samples_size()); + for (const auto & sample : element.samples()) + { + timestamp_column.insert(scaleTimestamp(sample.timestamp(), timestamp_scale)); + value_column.insert(sample.value()); + } + + ++current_row_in_tags; + } + + /// The "all_tags" column in the "tags" table is either ephemeral or doesn't exists. + /// We've used the "all_tags" column to calculate the "id" column already, + /// and now we don't need it to insert to the "tags" table. + tags_block.erase(TimeSeriesColumnNames::AllTags); + + BlocksToInsert res; + + /// A block to the "tags" table should be inserted first. + /// (Because any INSERT can fail and we don't want to have rows in the data table with no corresponding "id" written to the "tags" table.) + res.blocks.emplace_back(ViewTarget::Tags, std::move(tags_block)); + res.blocks.emplace_back(ViewTarget::Data, std::move(data_block)); + + return res; + } + + std::string_view metricTypeToString(prometheus::MetricMetadata::MetricType metric_type) + { + using namespace std::literals; + switch (metric_type) + { + case prometheus::MetricMetadata::UNKNOWN: return "unknown"sv; + case prometheus::MetricMetadata::COUNTER: return "counter"sv; + case prometheus::MetricMetadata::GAUGE: return "gauge"sv; + case prometheus::MetricMetadata::HISTOGRAM: return "histogram"sv; + case prometheus::MetricMetadata::GAUGEHISTOGRAM: return "gaugehistogram"sv; + case prometheus::MetricMetadata::SUMMARY: return "summary"sv; + case prometheus::MetricMetadata::INFO: return "info"sv; + case prometheus::MetricMetadata::STATESET: return "stateset"sv; + default: break; + } + return ""; + } + + /// Converts metrics metadata from the protobuf format to prepared blocks for inserting into target tables. + BlocksToInsert toBlocks(const google::protobuf::RepeatedPtrField & metrics_metadata, + const StorageID & time_series_storage_id, + const StorageInMemoryMetadata & time_series_storage_metadata, + const TimeSeriesSettings & time_series_settings) + { + size_t num_rows = metrics_metadata.size(); + + if (!num_rows) + return {}; /// Nothing to insert into target tables. + + /// Column types must be extracted from the target tables' metadata. + const auto & columns_description = time_series_storage_metadata.columns; + + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + return getInsertableColumnDescription(columns_description, column_name, time_series_storage_id); + }; + + /// We're going to prepare one blocks for the "metrics" table. + Block block; + + auto make_column = [&](const ColumnDescription & column_description) -> IColumn & + { + auto column = column_description.type->createColumn(); + column->reserve(num_rows); + auto * column_ptr = column.get(); + block.insert(ColumnWithTypeAndName{std::move(column), column_description.type, column_description.name}); + return *column_ptr; + }; + + /// Create columns. + + /// Column "metric_family_name". + const auto & metric_family_name_description = get_column_description(TimeSeriesColumnNames::MetricFamilyName); + TimeSeriesColumnsValidator validator{time_series_storage_id, time_series_settings}; + validator.validateColumnForMetricFamilyName(metric_family_name_description); + auto & metric_family_name_column = make_column(metric_family_name_description); + + /// Column "type". + const auto & type_description = get_column_description(TimeSeriesColumnNames::Type); + validator.validateColumnForType(type_description); + auto & type_column = make_column(type_description); + + /// Column "unit". + const auto & unit_description = get_column_description(TimeSeriesColumnNames::Unit); + validator.validateColumnForUnit(unit_description); + auto & unit_column = make_column(unit_description); + + /// Column "help". + const auto & help_description = get_column_description(TimeSeriesColumnNames::Help); + validator.validateColumnForHelp(help_description); + auto & help_column = make_column(help_description); + + /// Fill those columns. + for (const auto & element : metrics_metadata) + { + const auto & metric_family_name = element.metric_family_name(); + const auto & type_str = metricTypeToString(element.type()); + const auto & help = element.help(); + const auto & unit = element.unit(); + + metric_family_name_column.insertData(metric_family_name.data(), metric_family_name.length()); + type_column.insertData(type_str.data(), type_str.length()); + unit_column.insertData(unit.data(), unit.length()); + help_column.insertData(help.data(), help.length()); + } + + /// Prepare a result. + BlocksToInsert res; + res.blocks.emplace_back(ViewTarget::Metrics, std::move(block)); + return res; + } + + /// Inserts blocks to target tables. + void insertToTargetTables(BlocksToInsert && blocks, StorageTimeSeries & time_series_storage, ContextPtr context, Poco::Logger * log) + { + auto time_series_storage_id = time_series_storage.getStorageID(); + + for (auto & [table_kind, block] : blocks.blocks) + { + if (block) + { + const auto & target_table_id = time_series_storage.getTargetTableId(table_kind); + + LOG_INFO(log, "{}: Inserting {} rows to the {} table", + time_series_storage_id.getNameForLogs(), block.rows(), toString(table_kind)); + + auto insert_query = std::make_shared(); + insert_query->table_id = target_table_id; + + auto columns_ast = std::make_shared(); + for (const auto & name : block.getNames()) + columns_ast->children.emplace_back(std::make_shared(name)); + insert_query->columns = columns_ast; + + ContextMutablePtr insert_context = Context::createCopy(context); + insert_context->setCurrentQueryId(context->getCurrentQueryId() + ":" + String{toString(table_kind)}); + + LOG_TEST(log, "{}: Executing query: {}", time_series_storage_id.getNameForLogs(), queryToString(insert_query)); + + InterpreterInsertQuery interpreter( + insert_query, + insert_context, + /* allow_materialized= */ false, + /* no_squash= */ false, + /* no_destination= */ false, + /* async_insert= */ false); + + BlockIO io = interpreter.execute(); + PushingPipelineExecutor executor(io.pipeline); + + executor.start(); + executor.push(std::move(block)); + executor.finish(); + } + } + } +} + + +PrometheusRemoteWriteProtocol::PrometheusRemoteWriteProtocol(StoragePtr time_series_storage_, const ContextPtr & context_) + : WithContext(context_) + , time_series_storage(storagePtrToTimeSeries(time_series_storage_)) + , log(getLogger("PrometheusRemoteWriteProtocol")) +{ +} + +PrometheusRemoteWriteProtocol::~PrometheusRemoteWriteProtocol() = default; + + +void PrometheusRemoteWriteProtocol::writeTimeSeries(const google::protobuf::RepeatedPtrField & time_series) +{ + auto time_series_storage_id = time_series_storage->getStorageID(); + + LOG_TRACE(log, "{}: Writing {} time series", + time_series_storage_id.getNameForLogs(), time_series.size()); + + auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr(); + auto time_series_settings = time_series_storage->getStorageSettingsPtr(); + + auto blocks = toBlocks(time_series, getContext(), time_series_storage_id, *time_series_storage_metadata, *time_series_settings); + insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get()); + + LOG_TRACE(log, "{}: {} time series written", + time_series_storage_id.getNameForLogs(), time_series.size()); +} + +void PrometheusRemoteWriteProtocol::writeMetricsMetadata(const google::protobuf::RepeatedPtrField & metrics_metadata) +{ + auto time_series_storage_id = time_series_storage->getStorageID(); + + LOG_TRACE(log, "{}: Writing {} metrics metadata", + time_series_storage_id.getNameForLogs(), metrics_metadata.size()); + + auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr(); + auto time_series_settings = time_series_storage->getStorageSettingsPtr(); + + auto blocks = toBlocks(metrics_metadata, time_series_storage_id, *time_series_storage_metadata, *time_series_settings); + insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get()); + + LOG_TRACE(log, "{}: {} metrics metadata written", + time_series_storage_id.getNameForLogs(), metrics_metadata.size()); +} + +} + +#endif diff --git a/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h new file mode 100644 index 00000000000..24c65e96cbe --- /dev/null +++ b/src/Storages/TimeSeries/PrometheusRemoteWriteProtocol.h @@ -0,0 +1,35 @@ +#pragma once + +#include "config.h" +#if USE_PROMETHEUS_PROTOBUFS + +#include +#include +#include + + +namespace DB +{ +class StorageTimeSeries; + +/// Helper class to support the prometheus remote write protocol. +class PrometheusRemoteWriteProtocol : WithContext +{ +public: + PrometheusRemoteWriteProtocol(StoragePtr time_series_storage_, const ContextPtr & context_); + ~PrometheusRemoteWriteProtocol(); + + /// Insert time series received by remote write protocol to our table. + void writeTimeSeries(const google::protobuf::RepeatedPtrField & time_series); + + /// Insert metrics metadata received by remote write protocol to our table. + void writeMetricsMetadata(const google::protobuf::RepeatedPtrField & metrics_metadata); + +private: + std::shared_ptr time_series_storage; + Poco::LoggerPtr log; +}; + +} + +#endif diff --git a/src/Storages/TimeSeries/TimeSeriesColumnNames.h b/src/Storages/TimeSeries/TimeSeriesColumnNames.h new file mode 100644 index 00000000000..d7b12fdeea8 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesColumnNames.h @@ -0,0 +1,38 @@ +#pragma once + + +namespace DB +{ + +struct TimeSeriesColumnNames +{ + /// The "data" table contains time series: + static constexpr const char * ID = "id"; + static constexpr const char * Timestamp = "timestamp"; + static constexpr const char * Value = "value"; + + /// The "tags" table contains identifiers for each combination of a metric name with corresponding tags (labels): + + /// The default expression specified for the "id" column contains an expression for calculating an identifier of a time series by a metric name and tags. + //static constexpr const char * kID = "id"; + static constexpr const char * MetricName = "metric_name"; + + /// Contains tags which have no corresponding columns specified in the "tags_to_columns" setting. + static constexpr const char * Tags = "tags"; + + /// Contains all tags, including those ones which have corresponding columns specified in the "tags_to_columns" setting. + /// This is a generated column, it's not stored anywhere, it's generated on the fly. + static constexpr const char * AllTags = "all_tags"; + + /// Contains the time range of a time series. + static constexpr const char * MinTime = "min_time"; + static constexpr const char * MaxTime = "max_time"; + + /// The "metrics" table contains general information (metadata) about metrics: + static constexpr const char * MetricFamilyName = "metric_family_name"; + static constexpr const char * Type = "type"; + static constexpr const char * Unit = "unit"; + static constexpr const char * Help = "help"; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp new file mode 100644 index 00000000000..a2308857e2e --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.cpp @@ -0,0 +1,272 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int INCOMPATIBLE_COLUMNS; + extern const int THERE_IS_NO_COLUMN; +} + + +TimeSeriesColumnsValidator::TimeSeriesColumnsValidator(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_) + : time_series_storage_id(std::move(time_series_storage_id_)) + , time_series_settings(time_series_settings_) +{ +} + + +void TimeSeriesColumnsValidator::validateColumns(const ColumnsDescription & columns) const +{ + try + { + validateColumnsImpl(columns); + } + catch (Exception & e) + { + e.addMessage("While checking columns of TimeSeries table {}", time_series_storage_id.getNameForLogs()); + throw; + } +} + + +void TimeSeriesColumnsValidator::validateColumnsImpl(const ColumnsDescription & columns) const +{ + + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + const auto * column = columns.tryGet(column_name); + if (!column) + { + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is required for the TimeSeries table engine", column_name); + } + return *column; + }; + + /// Validate columns for the "data" table. + validateColumnForID(get_column_description(TimeSeriesColumnNames::ID)); + validateColumnForTimestamp(get_column_description(TimeSeriesColumnNames::Timestamp)); + validateColumnForValue(get_column_description(TimeSeriesColumnNames::Value)); + + /// Validate columns for the "tags" table. + validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + validateColumnForTagValue(get_column_description(column_name)); + } + + validateColumnForTagsMap(get_column_description(TimeSeriesColumnNames::Tags)); + validateColumnForTagsMap(get_column_description(TimeSeriesColumnNames::AllTags)); + + /// Validate columns for the "metrics" table. + validateColumnForMetricFamilyName(get_column_description(TimeSeriesColumnNames::MetricFamilyName)); + validateColumnForType(get_column_description(TimeSeriesColumnNames::Type)); + validateColumnForUnit(get_column_description(TimeSeriesColumnNames::Unit)); + validateColumnForHelp(get_column_description(TimeSeriesColumnNames::Help)); +} + + +void TimeSeriesColumnsValidator::validateTargetColumns(ViewTarget::Kind target_kind, const StorageID & target_table_id, const ColumnsDescription & target_columns) const +{ + try + { + validateTargetColumnsImpl(target_kind, target_columns); + } + catch (Exception & e) + { + e.addMessage("While checking columns of table {} which is the {} target of TimeSeries table {}", target_table_id.getNameForLogs(), + toString(target_kind), time_series_storage_id.getNameForLogs()); + throw; + } +} + + +void TimeSeriesColumnsValidator::validateTargetColumnsImpl(ViewTarget::Kind target_kind, const ColumnsDescription & target_columns) const +{ + auto get_column_description = [&](const String & column_name) -> const ColumnDescription & + { + const auto * column = target_columns.tryGet(column_name); + if (!column) + { + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is required for the TimeSeries table engine", column_name); + } + return *column; + }; + + switch (target_kind) + { + case ViewTarget::Data: + { + /// Here "check_default = false" because it's ok for the "id" column in the target table not to contain + /// an expression for calculating the identifier of a time series. + validateColumnForID(get_column_description(TimeSeriesColumnNames::ID), /* check_default= */ false); + + validateColumnForTimestamp(get_column_description(TimeSeriesColumnNames::Timestamp)); + validateColumnForValue(get_column_description(TimeSeriesColumnNames::Value)); + + break; + } + + case ViewTarget::Tags: + { + validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName)); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + validateColumnForTagValue(get_column_description(column_name)); + } + + validateColumnForTagsMap(get_column_description(TimeSeriesColumnNames::Tags)); + + break; + } + + case ViewTarget::Metrics: + { + validateColumnForMetricFamilyName(get_column_description(TimeSeriesColumnNames::MetricFamilyName)); + validateColumnForType(get_column_description(TimeSeriesColumnNames::Type)); + validateColumnForUnit(get_column_description(TimeSeriesColumnNames::Unit)); + validateColumnForHelp(get_column_description(TimeSeriesColumnNames::Help)); + break; + } + + default: + UNREACHABLE(); + } +} + + +void TimeSeriesColumnsValidator::validateColumnForID(const ColumnDescription & column, bool check_default) const +{ + if (check_default && !column.default_desc.expression) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "The DEFAULT expression for column {} must contain an expression " + "which will be used to calculate the identifier of each time series: {} {} DEFAULT ...", + column.name, column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTimestamp(const ColumnDescription & column) const +{ + if (!isDateTime64(removeNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected DateTime64", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTimestamp(const ColumnDescription & column, UInt32 & out_scale) const +{ + auto maybe_datetime64_type = removeNullable(column.type); + if (!isDateTime64(maybe_datetime64_type)) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected DateTime64", + column.name, column.type->getName()); + } + const auto & datetime64_type = typeid_cast(*maybe_datetime64_type); + out_scale = datetime64_type.getScale(); +} + +void TimeSeriesColumnsValidator::validateColumnForValue(const ColumnDescription & column) const +{ + if (!isFloat(removeNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected Float32 or Float64", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForMetricName(const ColumnDescription & column) const +{ + validateColumnForTagValue(column); +} + +void TimeSeriesColumnsValidator::validateColumnForMetricName(const ColumnWithTypeAndName & column) const +{ + validateColumnForTagValue(column); +} + +void TimeSeriesColumnsValidator::validateColumnForTagValue(const ColumnDescription & column) const +{ + if (!isString(removeLowCardinalityAndNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected String or LowCardinality(String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTagValue(const ColumnWithTypeAndName & column) const +{ + if (!isString(removeLowCardinalityAndNullable(column.type))) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column {} has illegal data type {}, expected String or LowCardinality(String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTagsMap(const ColumnDescription & column) const +{ + if (!isMap(column.type) + || !isString(removeLowCardinality(typeid_cast(*column.type).getKeyType())) + || !isString(removeLowCardinality(typeid_cast(*column.type).getValueType()))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected Map(String, String) or Map(LowCardinality(String), String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForTagsMap(const ColumnWithTypeAndName & column) const +{ + if (!isMap(column.type) + || !isString(removeLowCardinality(typeid_cast(*column.type).getKeyType())) + || !isString(removeLowCardinality(typeid_cast(*column.type).getValueType()))) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column {} has illegal data type {}, expected Map(String, String) or Map(LowCardinality(String), String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForMetricFamilyName(const ColumnDescription & column) const +{ + if (!isString(removeLowCardinalityAndNullable(column.type))) + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Column {} has illegal data type {}, expected String or LowCardinality(String)", + column.name, column.type->getName()); + } +} + +void TimeSeriesColumnsValidator::validateColumnForType(const ColumnDescription & column) const +{ + validateColumnForMetricFamilyName(column); +} + +void TimeSeriesColumnsValidator::validateColumnForUnit(const ColumnDescription & column) const +{ + validateColumnForMetricFamilyName(column); +} + +void TimeSeriesColumnsValidator::validateColumnForHelp(const ColumnDescription & column) const +{ + validateColumnForMetricFamilyName(column); +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h new file mode 100644 index 00000000000..43a54bf2ad6 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesColumnsValidator.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ColumnsDescription; +struct ColumnDescription; +struct ColumnWithTypeAndName; +struct TimeSeriesSettings; + +/// Checks the types of columns of a TimeSeries table. +class TimeSeriesColumnsValidator +{ +public: + /// Constructor stores a reference to argument `time_series_settings_` (it's unnecessary to copy it). + TimeSeriesColumnsValidator(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_); + + /// Checks the columns of a TimeSeries table and throws an exception if some of the required columns don't exist or have illegal types. + void validateColumns(const ColumnsDescription & columns) const; + + /// Checks columns of a target table that a TimeSeries table is going to use. + /// Throws an exception if some of the required columns don't exist or have illegal types. + void validateTargetColumns(ViewTarget::Kind target_kind, const StorageID & target_table_id, const ColumnsDescription & target_columns) const; + + /// Each of the following functions validates a specific column type. + void validateColumnForID(const ColumnDescription & column, bool check_default = true) const; + void validateColumnForTimestamp(const ColumnDescription & column) const; + void validateColumnForTimestamp(const ColumnDescription & column, UInt32 & out_scale) const; + void validateColumnForValue(const ColumnDescription & column) const; + + void validateColumnForMetricName(const ColumnDescription & column) const; + void validateColumnForMetricName(const ColumnWithTypeAndName & column) const; + void validateColumnForTagValue(const ColumnDescription & column) const; + void validateColumnForTagValue(const ColumnWithTypeAndName & column) const; + void validateColumnForTagsMap(const ColumnDescription & column) const; + void validateColumnForTagsMap(const ColumnWithTypeAndName & column) const; + + void validateColumnForMetricFamilyName(const ColumnDescription & column) const; + void validateColumnForType(const ColumnDescription & column) const; + void validateColumnForUnit(const ColumnDescription & column) const; + void validateColumnForHelp(const ColumnDescription & column) const; + +private: + void validateColumnsImpl(const ColumnsDescription & columns) const; + void validateTargetColumnsImpl(ViewTarget::Kind target_kind, const ColumnsDescription & target_columns) const; + + const StorageID time_series_storage_id; + const TimeSeriesSettings & time_series_settings; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp new file mode 100644 index 00000000000..f9e7290e514 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -0,0 +1,470 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCOMPATIBLE_COLUMNS; + extern const int INCORRECT_QUERY; +} + + +TimeSeriesDefinitionNormalizer::TimeSeriesDefinitionNormalizer(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_, + const ASTCreateQuery * as_create_query_) + : time_series_storage_id(std::move(time_series_storage_id_)) + , time_series_settings(time_series_settings_) + , as_create_query(as_create_query_) +{ +} + + +void TimeSeriesDefinitionNormalizer::normalize(ASTCreateQuery & create_query) const +{ + reorderColumns(create_query); + addMissingColumns(create_query); + addMissingDefaultForIDColumn(create_query); + + if (as_create_query) + addMissingInnerEnginesFromAsTable(create_query); + + addMissingInnerEngines(create_query); +} + + +void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) const +{ + if (!create.columns_list || !create.columns_list->columns) + return; + + auto & columns = create.columns_list->columns->children; + + /// Build a map "column_name -> column_declaration". + std::unordered_map> columns_by_name; + for (const auto & column : columns) + { + auto column_declaration = typeid_cast>(column); + columns_by_name[column_declaration->name] = column_declaration; + } + + /// Remove all columns and then add them again in the canonical order. + columns.clear(); + + auto add_column_in_correct_order = [&](std::string_view column_name) + { + auto it = columns_by_name.find(column_name); + if (it != columns_by_name.end()) + { + /// Add the column back to the list. + columns.push_back(it->second); + + /// Remove the column from the map to allow the check at the end of this function + /// that all columns from the original list are added back to the list. + columns_by_name.erase(it); + } + }; + + /// Reorder columns for the "data" table. + add_column_in_correct_order(TimeSeriesColumnNames::ID); + add_column_in_correct_order(TimeSeriesColumnNames::Timestamp); + add_column_in_correct_order(TimeSeriesColumnNames::Value); + + /// Reorder columns for the "tags" table. + add_column_in_correct_order(TimeSeriesColumnNames::MetricName); + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + add_column_in_correct_order(column_name); + } + + add_column_in_correct_order(TimeSeriesColumnNames::Tags); + add_column_in_correct_order(TimeSeriesColumnNames::AllTags); + + if (time_series_settings.store_min_time_and_max_time) + { + add_column_in_correct_order(TimeSeriesColumnNames::MinTime); + add_column_in_correct_order(TimeSeriesColumnNames::MaxTime); + } + + /// Reorder columns for the "metrics" table. + add_column_in_correct_order(TimeSeriesColumnNames::MetricFamilyName); + add_column_in_correct_order(TimeSeriesColumnNames::Type); + add_column_in_correct_order(TimeSeriesColumnNames::Unit); + add_column_in_correct_order(TimeSeriesColumnNames::Help); + + /// All columns from the original list must be added back to the list. + if (!columns_by_name.empty()) + { + throw Exception( + ErrorCodes::INCOMPATIBLE_COLUMNS, + "{}: Column {} can't be used in this table. " + "The TimeSeries table engine supports only a limited set of columns (id, timestamp, value, metric_name, tags, metric_family_name, type, unit, help). " + "Extra columns representing tags must be specified in the 'tags_to_columns' setting.", + time_series_storage_id.getNameForLogs(), columns_by_name.begin()->first); + } +} + + +void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) const +{ + if (!create.as_table.empty()) + { + /// If the create query has the "AS other_table" clause ("CREATE TABLE table AS other_table") + /// then all columns must be extracted from that "other_table". + /// Function InterpreterCreateQuery::getTablePropertiesAndNormalizeCreateQuery() will do that for us, + /// we don't need to fill missing columns by default in that case. + return; + } + + if (!create.columns_list) + create.set(create.columns_list, std::make_shared()); + + if (!create.columns_list->columns) + create.columns_list->set(create.columns_list->columns, std::make_shared()); + auto & columns = create.columns_list->columns->children; + + /// Here in this function we rely on that the columns are already sorted in the canonical order (see the reorderColumns() function). + /// NOTE: The order in which this function processes columns MUST be exactly the same as the order in reorderColumns(). + size_t position = 0; + + auto is_next_column_named = [&](std::string_view column_name) + { + if (position < columns.size() && (typeid_cast(*columns[position]).name == column_name)) + { + ++position; + return true; + } + return false; + }; + + auto make_new_column = [&](const String & column_name, ASTPtr type) + { + auto new_column = std::make_shared(); + new_column->name = column_name; + new_column->type = type; + columns.insert(columns.begin() + position, new_column); + ++position; + }; + + auto get_uuid_type = [] { return makeASTDataType("UUID"); }; + auto get_datetime_type = [] { return makeASTDataType("DateTime64", std::make_shared(3ul)); }; + auto get_float_type = [] { return makeASTDataType("Float64"); }; + auto get_string_type = [] { return makeASTDataType("String"); }; + auto get_lc_string_type = [&] { return makeASTDataType("LowCardinality", get_string_type()); }; + auto get_string_to_string_map_type = [&] { return makeASTDataType("Map", get_string_type(), get_string_type()); }; + auto get_lc_string_to_string_map_type = [&] { return makeASTDataType("Map", get_lc_string_type(), get_string_type()); }; + + auto make_nullable = [&](std::shared_ptr type) + { + if (type->name == "Nullable") + return type; + else + return makeASTDataType("Nullable", type); + }; + + /// Add missing columns for the "data" table. + if (!is_next_column_named(TimeSeriesColumnNames::ID)) + make_new_column(TimeSeriesColumnNames::ID, get_uuid_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Timestamp)) + make_new_column(TimeSeriesColumnNames::Timestamp, get_datetime_type()); + + auto timestamp_column = typeid_cast>(columns[position - 1]); + auto timestamp_type = typeid_cast>(timestamp_column->type->ptr()); + + if (!is_next_column_named(TimeSeriesColumnNames::Value)) + make_new_column(TimeSeriesColumnNames::Value, get_float_type()); + + /// Add missing columns for the "tags" table. + if (!is_next_column_named(TimeSeriesColumnNames::MetricName)) + { + /// We use 'LowCardinality(String)' as the default type of the `metric_name` column: + /// it looks like a correct optimization because there are shouldn't be too many different metrics. + make_new_column(TimeSeriesColumnNames::MetricName, get_lc_string_type()); + } + + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + if (!is_next_column_named(column_name)) + make_new_column(column_name, get_string_type()); + } + + if (!is_next_column_named(TimeSeriesColumnNames::Tags)) + { + /// We use 'Map(LowCardinality(String), String)' as the default type of the `tags` column: + /// it looks like a correct optimization because there are shouldn't be too many different tag names. + make_new_column(TimeSeriesColumnNames::Tags, get_lc_string_to_string_map_type()); + } + + if (!is_next_column_named(TimeSeriesColumnNames::AllTags)) + { + /// The `all_tags` column is virtual (it's calculated on the fly and never stored anywhere) + /// so here we don't need to use the LowCardinality optimization as for the `tags` column. + make_new_column(TimeSeriesColumnNames::AllTags, get_string_to_string_map_type()); + } + + if (time_series_settings.store_min_time_and_max_time) + { + /// We use Nullable(DateTime64(3)) as the default type of the `min_time` and `max_time` columns. + /// It's nullable because it allows the aggregation (see aggregate_min_time_and_max_time) work correctly even + /// for rows in the "tags" table which doesn't have `min_time` and `max_time` (because they have no matching rows in the "data" table). + make_new_column(TimeSeriesColumnNames::MinTime, make_nullable(timestamp_type)); + make_new_column(TimeSeriesColumnNames::MaxTime, make_nullable(timestamp_type)); + } + + /// Add missing columns for the "metrics" table. + if (!is_next_column_named(TimeSeriesColumnNames::MetricFamilyName)) + make_new_column(TimeSeriesColumnNames::MetricFamilyName, get_string_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Type)) + make_new_column(TimeSeriesColumnNames::Type, get_string_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Unit)) + make_new_column(TimeSeriesColumnNames::Unit, get_string_type()); + + if (!is_next_column_named(TimeSeriesColumnNames::Help)) + make_new_column(TimeSeriesColumnNames::Help, get_string_type()); + + /// If the following fails that means the order in which columns are processed in this function doesn't match the order of columns in reorderColumns(). + chassert(position == columns.size()); +} + + +void TimeSeriesDefinitionNormalizer::addMissingDefaultForIDColumn(ASTCreateQuery & create) const +{ + /// Find the 'id' column and make a default expression for it. + if (!create.columns_list || !create.columns_list->columns) + return; + + auto & columns = create.columns_list->columns->children; + auto * it = std::find_if(columns.begin(), columns.end(), [](const ASTPtr & column) + { + return typeid_cast(*column).name == TimeSeriesColumnNames::ID; + }); + + if (it == columns.end()) + return; + + auto & column_declaration = typeid_cast(**it); + + /// We add a DEFAULT for the 'id' column only if it's not specified yet. + if (column_declaration.default_specifier.empty() && !column_declaration.default_expression) + { + column_declaration.default_specifier = "DEFAULT"; + column_declaration.default_expression = chooseIDAlgorithm(column_declaration); + } +} + + +ASTPtr TimeSeriesDefinitionNormalizer::chooseIDAlgorithm(const ASTColumnDeclaration & id_column) const +{ + /// Build a list of arguments for a hash function. + /// All hash functions below allow multiple arguments, so we use two arguments: metric_name, all_tags. + ASTs arguments_for_hash_function; + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::MetricName)); + + if (time_series_settings.use_all_tags_column_to_generate_id) + { + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::AllTags)); + } + else + { + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + arguments_for_hash_function.push_back(std::make_shared(column_name)); + } + arguments_for_hash_function.push_back(std::make_shared(TimeSeriesColumnNames::Tags)); + } + + auto make_hash_function = [&](const String & function_name) + { + auto function = std::make_shared(); + function->name = function_name; + auto arguments_list = std::make_shared(); + arguments_list->children = std::move(arguments_for_hash_function); + function->arguments = arguments_list; + return function; + }; + + /// The type of a hash function depends on the type of the 'id' column. + auto id_type = DataTypeFactory::instance().get(id_column.type); + WhichDataType id_type_which(*id_type); + + if (id_type_which.isUInt64()) + { + return make_hash_function("sipHash64"); + } + else if (id_type_which.isFixedString() && typeid_cast(*id_type).getN() == 16) + { + return make_hash_function("sipHash128"); + } + else if (id_type_which.isUUID()) + { + return makeASTFunction("reinterpretAsUUID", make_hash_function("sipHash128")); + } + else if (id_type_which.isUInt128()) + { + return makeASTFunction("reinterpretAsUInt128", make_hash_function("sipHash128")); + } + else + { + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "{}: The DEFAULT expression for column {} must contain an expression " + "which will be used to calculate the identifier of each time series: {} {} DEFAULT ... " + "If the DEFAULT expression is not specified then it can be chosen implicitly but only if the column type is one of these: UInt64, UInt128, UUID. " + "For type {} the DEFAULT expression can't be chosen automatically, so please specify it explicitly", + time_series_storage_id.getNameForLogs(), id_column.name, id_column.name, id_type->getName(), id_type->getName()); + } +} + + +void TimeSeriesDefinitionNormalizer::addMissingInnerEnginesFromAsTable(ASTCreateQuery & create) const +{ + if (!as_create_query) + return; + + for (auto target_kind : {ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}) + { + if (as_create_query->hasTargetTableID(target_kind)) + { + /// It's unlikely correct to use "CREATE table AS other_table" when "other_table" has external tables like this: + /// CREATE TABLE other_table ENGINE=TimeSeries data mydata + /// (because `table` would use the same table "mydata"). + /// Thus we just prohibit that. + QualifiedTableName as_table{as_create_query->getDatabase(), as_create_query->getTable()}; + throw Exception( + ErrorCodes::INCORRECT_QUERY, + "Cannot CREATE a table AS {}.{} because it has external tables", + backQuoteIfNeed(as_table.database), backQuoteIfNeed(as_table.table)); + } + + auto inner_table_engine = create.getTargetInnerEngine(target_kind); + if (!inner_table_engine) + { + /// Copy an inner engine's definition from the other table. + inner_table_engine = as_create_query->getTargetInnerEngine(target_kind); + if (inner_table_engine) + create.setTargetInnerEngine(target_kind, typeid_cast>(inner_table_engine->clone())); + } + } +} + + +void TimeSeriesDefinitionNormalizer::addMissingInnerEngines(ASTCreateQuery & create) const +{ + for (auto target_kind : {ViewTarget::Data, ViewTarget::Tags, ViewTarget::Metrics}) + { + if (create.hasTargetTableID(target_kind)) + continue; /// External target is set, inner engine is not needed. + + auto inner_table_engine = create.getTargetInnerEngine(target_kind); + if (inner_table_engine && inner_table_engine->engine) + continue; /// Engine is set already, skip it. + + if (!inner_table_engine) + { + /// Some part of storage definition (such as PARTITION BY) is specified, but the inner ENGINE is not: just set default one. + inner_table_engine = std::make_shared(); + create.setTargetInnerEngine(target_kind, inner_table_engine); + } + + /// Set engine by default. + setInnerEngineByDefault(target_kind, *inner_table_engine); + } +} + + +void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind inner_table_kind, ASTStorage & inner_storage_def) const +{ + switch (inner_table_kind) + { + case ViewTarget::Data: + { + inner_storage_def.set(inner_storage_def.engine, makeASTFunction("MergeTree")); + inner_storage_def.engine->no_empty_args = false; + + if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) + { + inner_storage_def.set(inner_storage_def.order_by, + makeASTFunction("tuple", + std::make_shared(TimeSeriesColumnNames::ID), + std::make_shared(TimeSeriesColumnNames::Timestamp))); + } + break; + } + + case ViewTarget::Tags: + { + String engine_name; + if (time_series_settings.aggregate_min_time_and_max_time) + engine_name = "AggregatingMergeTree"; + else + engine_name = "ReplacingMergeTree"; + + inner_storage_def.set(inner_storage_def.engine, makeASTFunction(engine_name)); + inner_storage_def.engine->no_empty_args = false; + + if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) + { + inner_storage_def.set(inner_storage_def.primary_key, + std::make_shared(TimeSeriesColumnNames::MetricName)); + + ASTs order_by_list; + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MetricName)); + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::ID)); + + if (time_series_settings.store_min_time_and_max_time && !time_series_settings.aggregate_min_time_and_max_time) + { + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MinTime)); + order_by_list.push_back(std::make_shared(TimeSeriesColumnNames::MaxTime)); + } + + auto order_by_tuple = std::make_shared(); + order_by_tuple->name = "tuple"; + auto arguments_list = std::make_shared(); + arguments_list->children = std::move(order_by_list); + order_by_tuple->arguments = arguments_list; + inner_storage_def.set(inner_storage_def.order_by, order_by_tuple); + } + break; + } + + case ViewTarget::Metrics: + { + inner_storage_def.set(inner_storage_def.engine, makeASTFunction("ReplacingMergeTree")); + inner_storage_def.engine->no_empty_args = false; + + if (!inner_storage_def.order_by && !inner_storage_def.primary_key && inner_storage_def.engine->name.ends_with("MergeTree")) + { + inner_storage_def.set(inner_storage_def.order_by, std::make_shared(TimeSeriesColumnNames::MetricFamilyName)); + } + break; + } + + default: + UNREACHABLE(); /// This function must not be called with any other `kind`. + } +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h new file mode 100644 index 00000000000..1f959eb3ce0 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTColumnDeclaration; +class ASTCreateQuery; +struct ColumnDescription; +struct TimeSeriesSettings; + +/// Normalizes a TimeSeries table definition. +class TimeSeriesDefinitionNormalizer +{ +public: + /// Constructor stores a reference to argument `time_series_settings_` (it's unnecessary to copy it). + TimeSeriesDefinitionNormalizer(StorageID time_series_storage_id_, + std::reference_wrapper time_series_settings_, + const ASTCreateQuery * as_create_query_); + + /// Adds missing columns to the definition and reorders all the columns in the canonical way. + /// Also adds engines of inner tables to the definition if they aren't specified yet. + /// The `as_table_create_query` parameter must be nullptr if it isn't a "CREATE AS query". + void normalize(ASTCreateQuery & create_query) const; + +private: + /// Reorders existing columns in the canonical way. + void reorderColumns(ASTCreateQuery & create) const; + + /// Adds missing columns with data types set by default.. + void addMissingColumns(ASTCreateQuery & create) const; + + /// Adds the DEFAULT expression for the 'id' column if it isn't specified yet. + void addMissingDefaultForIDColumn(ASTCreateQuery & create) const; + + /// Generates a formulae for calculating the identifier of a time series from the metric name and all the tags. + ASTPtr chooseIDAlgorithm(const ASTColumnDeclaration & id_column) const; + + /// Copies the definitions of inner engines from "CREATE AS
" if this is that kind of query. + void addMissingInnerEnginesFromAsTable(ASTCreateQuery & create) const; + + /// Adds engines of inner tables to the definition if they aren't specified yet. + void addMissingInnerEngines(ASTCreateQuery & create) const; + + /// Sets the engine of an inner table by default. + void setInnerEngineByDefault(ViewTarget::Kind inner_table_kind, ASTStorage & inner_storage_def) const; + + const StorageID time_series_storage_id; + const TimeSeriesSettings & time_series_settings; + const ASTCreateQuery * as_create_query = nullptr; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp new file mode 100644 index 00000000000..5f616982a6f --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.cpp @@ -0,0 +1,191 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +TimeSeriesInnerTablesCreator::TimeSeriesInnerTablesCreator(ContextPtr context_, + StorageID time_series_storage_id_, + std::reference_wrapper time_series_columns_, + std::reference_wrapper time_series_settings_) + : WithContext(context_) + , time_series_storage_id(std::move(time_series_storage_id_)) + , time_series_columns(time_series_columns_) + , time_series_settings(time_series_settings_) +{ +} + +TimeSeriesInnerTablesCreator::~TimeSeriesInnerTablesCreator() = default; + + +ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription(ViewTarget::Kind inner_table_kind) const +{ + ColumnsDescription columns; + + switch (inner_table_kind) + { + case ViewTarget::Data: + { + /// Column "id". + { + auto id_column = time_series_columns.get(TimeSeriesColumnNames::ID); + /// The expression for calculating the identifier of a time series can be transferred only to the "tags" inner table + /// (because it usually depends on columns like "metric_name" or "all_tags"). + id_column.default_desc = {}; + columns.add(std::move(id_column)); + } + + /// Column "timestamp". + columns.add(time_series_columns.get(TimeSeriesColumnNames::Timestamp)); + + /// Column "value". + columns.add(time_series_columns.get(TimeSeriesColumnNames::Value)); + break; + } + + case ViewTarget::Tags: + { + /// Column "id". + columns.add(time_series_columns.get(TimeSeriesColumnNames::ID)); + + /// Column "metric_name". + columns.add(time_series_columns.get(TimeSeriesColumnNames::MetricName)); + + /// Columns corresponding to specific tags specified in the "tags_to_columns" setting. + const Map & tags_to_columns = time_series_settings.tags_to_columns; + for (const auto & tag_name_and_column_name : tags_to_columns) + { + const auto & tuple = tag_name_and_column_name.safeGet(); + const auto & column_name = tuple.at(1).safeGet(); + columns.add(time_series_columns.get(column_name)); + } + + /// Column "tags". + columns.add(time_series_columns.get(TimeSeriesColumnNames::Tags)); + + /// Column "all_tags". + if (time_series_settings.use_all_tags_column_to_generate_id) + { + ColumnDescription all_tags_column = time_series_columns.get(TimeSeriesColumnNames::AllTags); + /// Column "all_tags" is here only to calculate the identifier of a time series for the "id" column, so it can be ephemeral. + all_tags_column.default_desc.kind = ColumnDefaultKind::Ephemeral; + if (!all_tags_column.default_desc.expression) + { + all_tags_column.default_desc.ephemeral_default = true; + all_tags_column.default_desc.expression = makeASTFunction("defaultValueOfTypeName", std::make_shared(all_tags_column.type->getName())); + } + columns.add(std::move(all_tags_column)); + } + + /// Columns "min_time" and "max_time". + if (time_series_settings.store_min_time_and_max_time) + { + auto min_time_column = time_series_columns.get(TimeSeriesColumnNames::MinTime); + auto max_time_column = time_series_columns.get(TimeSeriesColumnNames::MaxTime); + if (time_series_settings.aggregate_min_time_and_max_time) + { + AggregateFunctionProperties properties; + auto min_function = AggregateFunctionFactory::instance().get("min", NullsAction::EMPTY, {min_time_column.type}, {}, properties); + auto custom_name = std::make_unique(min_function, DataTypes{min_time_column.type}, Array{}); + min_time_column.type = DataTypeFactory::instance().getCustom(std::make_unique(std::move(custom_name))); + + auto max_function = AggregateFunctionFactory::instance().get("max", NullsAction::EMPTY, {max_time_column.type}, {}, properties); + custom_name = std::make_unique(max_function, DataTypes{max_time_column.type}, Array{}); + max_time_column.type = DataTypeFactory::instance().getCustom(std::make_unique(std::move(custom_name))); + } + columns.add(std::move(min_time_column)); + columns.add(std::move(max_time_column)); + } + + break; + } + + case ViewTarget::Metrics: + { + columns.add(time_series_columns.get(TimeSeriesColumnNames::MetricFamilyName)); + columns.add(time_series_columns.get(TimeSeriesColumnNames::Type)); + columns.add(time_series_columns.get(TimeSeriesColumnNames::Unit)); + columns.add(time_series_columns.get(TimeSeriesColumnNames::Help)); + break; + } + + default: + UNREACHABLE(); + } + + return columns; +} + + +StorageID TimeSeriesInnerTablesCreator::getInnerTableID(ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid) const +{ + StorageID res = time_series_storage_id; + if (time_series_storage_id.hasUUID()) + res.table_name = fmt::format(".inner_id.{}.{}", toString(inner_table_kind), time_series_storage_id.uuid); + else + res.table_name = fmt::format(".inner.{}.{}", toString(inner_table_kind), time_series_storage_id.table_name); + res.uuid = inner_table_uuid; + return res; +} + + +std::shared_ptr TimeSeriesInnerTablesCreator::getInnerTableCreateQuery( + ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr & inner_storage_def) const +{ + auto manual_create_query = std::make_shared(); + + auto inner_table_id = getInnerTableID(inner_table_kind, inner_table_uuid); + manual_create_query->setDatabase(inner_table_id.database_name); + manual_create_query->setTable(inner_table_id.table_name); + manual_create_query->uuid = inner_table_id.uuid; + manual_create_query->has_uuid = inner_table_id.uuid != UUIDHelpers::Nil; + + auto new_columns_list = std::make_shared(); + new_columns_list->set(new_columns_list->columns, InterpreterCreateQuery::formatColumns(getInnerTableColumnsDescription(inner_table_kind))); + manual_create_query->set(manual_create_query->columns_list, new_columns_list); + + if (inner_storage_def) + manual_create_query->set(manual_create_query->storage, inner_storage_def->clone()); + + return manual_create_query; +} + +StorageID TimeSeriesInnerTablesCreator::createInnerTable( + ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr & inner_storage_def) const +{ + /// We will make a query to create the inner target table. + auto create_context = Context::createCopy(getContext()); + + auto manual_create_query = getInnerTableCreateQuery(inner_table_kind, inner_table_uuid, inner_storage_def); + + /// Create the inner target table. + InterpreterCreateQuery create_interpreter(manual_create_query, create_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); + + return DatabaseCatalog::instance().getTable({manual_create_query->getDatabase(), manual_create_query->getTable()}, getContext())->getStorageID(); +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h new file mode 100644 index 00000000000..5778dd77398 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesInnerTablesCreator.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTCreateQuery; +class ColumnsDescription; +struct TimeSeriesSettings; + +/// Generates inner tables for the TimeSeries table engine. +class TimeSeriesInnerTablesCreator : public WithContext +{ +public: + /// Constructor stores references to arguments `time_series_columns_` and `time_series_settings_` (it's unnecessary to copy them). + TimeSeriesInnerTablesCreator(ContextPtr context_, + StorageID time_series_storage_id_, + std::reference_wrapper time_series_columns_, + std::reference_wrapper time_series_settings_); + + ~TimeSeriesInnerTablesCreator(); + + /// Returns a column description of an inner table. + ColumnsDescription getInnerTableColumnsDescription(ViewTarget::Kind inner_table_kind) const; + + /// Returns a StorageID of an inner table. + StorageID getInnerTableID(ViewTarget::Kind inner_table_kind, const UUID & inner_table_uuid) const; + + /// Generates a CREATE TABLE query for an inner table. + std::shared_ptr getInnerTableCreateQuery(ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr & inner_storage_def) const; + + /// Creates an inner table. + StorageID createInnerTable(ViewTarget::Kind inner_table_kind, + const UUID & inner_table_uuid, + const std::shared_ptr & inner_storage_def) const; + +private: + const StorageID time_series_storage_id; + const ColumnsDescription & time_series_columns; + const TimeSeriesSettings & time_series_settings; +}; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesSettings.cpp b/src/Storages/TimeSeries/TimeSeriesSettings.cpp new file mode 100644 index 00000000000..3a15be59191 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesSettings.cpp @@ -0,0 +1,34 @@ +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS) + +void TimeSeriesSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } +} + +} diff --git a/src/Storages/TimeSeries/TimeSeriesSettings.h b/src/Storages/TimeSeries/TimeSeriesSettings.h new file mode 100644 index 00000000000..4dc6a436cd0 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesSettings.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTStorage; + +#define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \ + M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \ + M(Bool, use_all_tags_column_to_generate_id, true, "When generating an expression to calculate an identifier of a time series, this flag enables using the 'all_tags' column in that calculation. The 'all_tags' is a virtual column containing all tags except the metric name", 0) \ + M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \ + M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \ + M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \ + +DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS) + +/// Settings for the TimeSeries table engine. +/// Could be loaded from a CREATE TABLE query (SETTINGS clause). For example: +/// CREATE TABLE mytable ENGINE = TimeSeries() SETTINGS tags_to_columns = {'job':'job', 'instance':'instance'} DATA ENGINE = ReplicatedMergeTree('zkpath', 'replica'), ... +struct TimeSeriesSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +using TimeSeriesSettingsPtr = std::shared_ptr; + +} diff --git a/src/Storages/TimeSeries/TimeSeriesTagNames.h b/src/Storages/TimeSeries/TimeSeriesTagNames.h new file mode 100644 index 00000000000..23b005ed414 --- /dev/null +++ b/src/Storages/TimeSeries/TimeSeriesTagNames.h @@ -0,0 +1,13 @@ +#pragma once + + +namespace DB +{ + +/// Label names with special meaning. +struct TimeSeriesTagNames +{ + static constexpr const char * MetricName = "__name__"; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 8f33314397c..4ed74763810 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -26,6 +26,9 @@ void registerStorageGenerateRandom(StorageFactory & factory); void registerStorageExecutable(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); void registerStorageLoop(StorageFactory & factory); +void registerStorageFuzzQuery(StorageFactory & factory); +void registerStorageTimeSeries(StorageFactory & factory); + #if USE_RAPIDJSON || USE_SIMDJSON void registerStorageFuzzJSON(StorageFactory & factory); #endif @@ -126,6 +129,9 @@ void registerStorages() registerStorageExecutable(factory); registerStorageWindowView(factory); registerStorageLoop(factory); + registerStorageFuzzQuery(factory); + registerStorageTimeSeries(factory); + #if USE_RAPIDJSON || USE_SIMDJSON registerStorageFuzzJSON(factory); #endif diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 6765e112bb9..5a63c118e2d 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -74,6 +75,7 @@ private: {"a", std::make_shared()}, {"b", std::make_shared()}, {"foo", std::make_shared()}, + {"is_value", DataTypeFactory::instance().get("Bool")}, }), TableWithColumnNamesAndTypes( createDBAndTable("table2"), @@ -411,6 +413,14 @@ TEST(TransformQueryForExternalDatabase, Analyzer) R"(SELECT "column" FROM "test"."table")"); check(state, 1, {"column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo"}, - "SELECT * FROM table WHERE (column) IN (1)", + "SELECT * EXCEPT (is_value) FROM table WHERE (column) IN (1)", R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))"); + + check(state, 1, {"is_value"}, + "SELECT is_value FROM table WHERE is_value = true", + R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = true)"); + + check(state, 1, {"is_value"}, + "SELECT is_value FROM table WHERE is_value = 1", + R"(SELECT "is_value" FROM "test"."table" WHERE "is_value" = 1)"); } diff --git a/src/TableFunctions/TableFunctionFuzzQuery.cpp b/src/TableFunctions/TableFunctionFuzzQuery.cpp new file mode 100644 index 00000000000..224f6666556 --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.cpp @@ -0,0 +1,54 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +void TableFunctionFuzzQuery::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); + + auto args = args_func.at(0)->children; + configuration = StorageFuzzQuery::getConfiguration(args, context); +} + +StoragePtr TableFunctionFuzzQuery::executeImpl( + const ASTPtr & /*ast_function*/, + ContextPtr context, + const std::string & table_name, + ColumnsDescription /*cached_columns*/, + bool is_insert_query) const +{ + ColumnsDescription columns = getActualTableStructure(context, is_insert_query); + auto res = std::make_shared( + StorageID(getDatabaseName(), table_name), + columns, + /* comment */ String{}, + configuration); + res->startup(); + return res; +} + +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description = "Perturbs a query string with random variations.", + .returned_value = "A table object with a single column containing perturbed query strings."}, + .allow_readonly = true}); +} + +} diff --git a/src/TableFunctions/TableFunctionFuzzQuery.h b/src/TableFunctions/TableFunctionFuzzQuery.h new file mode 100644 index 00000000000..22d10341c4d --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class TableFunctionFuzzQuery : public ITableFunction +{ +public: + static constexpr auto name = "fuzzQuery"; + std::string getName() const override { return name; } + + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + ColumnsDescription getActualTableStructure(ContextPtr /* context */, bool /* is_insert_query */) const override + { + return ColumnsDescription{{"query", std::make_shared()}}; + } + +private: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return "fuzzQuery"; } + + String source; + std::optional random_seed; + StorageFuzzQuery::Configuration configuration; +}; + +} diff --git a/src/TableFunctions/TableFunctionTimeSeries.cpp b/src/TableFunctions/TableFunctionTimeSeries.cpp new file mode 100644 index 00000000000..62ea088eba0 --- /dev/null +++ b/src/TableFunctions/TableFunctionTimeSeries.cpp @@ -0,0 +1,128 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +template +void TableFunctionTimeSeriesTarget::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + const auto & args_func = ast_function->as(); + + if (!args_func.arguments) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' must have arguments.", name); + + auto & args = args_func.arguments->children; + + if ((args.size() != 1) && (args.size() != 2)) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function '{}' requires one or two arguments: {}([database, ] time_series_table)", name, name); + + if (args.size() == 1) + { + /// timeSeriesMetrics( [my_db.]my_time_series_table ) + if (const auto * id = args[0]->as()) + { + if (auto table_id = id->createTable()) + time_series_storage_id = table_id->getTableId(); + } + } + + if (time_series_storage_id.empty()) + { + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + if (args.size() == 1) + { + /// timeSeriesMetrics( 'my_time_series_table' ) + time_series_storage_id.table_name = checkAndGetLiteralArgument(args[0], "table_name"); + } + else + { + /// timeSeriesMetrics( 'mydb', 'my_time_series_table' ) + time_series_storage_id.database_name = checkAndGetLiteralArgument(args[0], "database_name"); + time_series_storage_id.table_name = checkAndGetLiteralArgument(args[1], "table_name"); + } + } + + if (time_series_storage_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Couldn't get a table name from the arguments of the {} table function", name); + + time_series_storage_id = context->resolveStorageID(time_series_storage_id); + target_table_type_name = getTargetTable(context)->getName(); +} + + +template +StoragePtr TableFunctionTimeSeriesTarget::getTargetTable(const ContextPtr & context) const +{ + auto time_series_storage = storagePtrToTimeSeries(DatabaseCatalog::instance().getTable(time_series_storage_id, context)); + return time_series_storage->getTargetTable(target_kind, context); +} + + +template +StoragePtr TableFunctionTimeSeriesTarget::executeImpl( + const ASTPtr & /* ast_function */, + ContextPtr context, + const String & /* table_name */, + ColumnsDescription /* cached_columns */, + bool /* is_insert_query */) const +{ + return getTargetTable(context); +} + +template +ColumnsDescription TableFunctionTimeSeriesTarget::getActualTableStructure(ContextPtr context, bool /* is_insert_query */) const +{ + return getTargetTable(context)->getInMemoryMetadataPtr()->columns; +} + +template +const char * TableFunctionTimeSeriesTarget::getStorageTypeName() const +{ + return target_table_type_name.c_str(); +} + + +void registerTableFunctionTimeSeries(TableFunctionFactory & factory) +{ + factory.registerFunction>( + {.documentation = { + .description=R"(Provides direct access to the 'data' target table for a specified TimeSeries table.)", + .examples{{"timeSeriesData", "SELECT * from timeSeriesData('mydb', 'time_series_table');", ""}}, + .categories{"Time Series"}} + }); + factory.registerFunction>( + {.documentation = { + .description=R"(Provides direct access to the 'tags' target table for a specified TimeSeries table.)", + .examples{{"timeSeriesTags", "SELECT * from timeSeriesTags('mydb', 'time_series_table');", ""}}, + .categories{"Time Series"}} + }); + factory.registerFunction>( + {.documentation = { + .description=R"(Provides direct access to the 'metrics' target table for a specified TimeSeries table.)", + .examples{{"timeSeriesMetrics", "SELECT * from timeSeriesMetrics('mydb', 'time_series_table');", ""}}, + .categories{"Time Series"}} + }); +} + +} diff --git a/src/TableFunctions/TableFunctionTimeSeries.h b/src/TableFunctions/TableFunctionTimeSeries.h new file mode 100644 index 00000000000..57654413fe4 --- /dev/null +++ b/src/TableFunctions/TableFunctionTimeSeries.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Table functions timeSeriesData('mydb', 'my_ts_table'), timeSeriesTags('mydb', 'my_ts_table'), timeSeriesMetrics('mydb', 'my_ts_table') +/// return the data table, the tags table, and the metrics table respectively associated with any TimeSeries table mydb.my_ts_table +template +class TableFunctionTimeSeriesTarget : public ITableFunction +{ +public: + static constexpr auto name = (target_kind == ViewTarget::Data) + ? "timeSeriesData" + : ((target_kind == ViewTarget::Tags) ? "timeSeriesTags" : "timeSeriesMetrics"); + + String getName() const override { return name; } + +private: + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; + const char * getStorageTypeName() const override; + + StoragePtr getTargetTable(const ContextPtr & context) const; + + StorageID time_series_storage_id = StorageID::createEmpty(); + String target_table_type_name; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index ca4913898f9..0b21de00f86 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -26,6 +26,7 @@ void registerTableFunctions() registerTableFunctionMongoDB(factory); registerTableFunctionRedis(factory); registerTableFunctionMergeTreeIndex(factory); + registerTableFunctionFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerTableFunctionFuzzJSON(factory); #endif @@ -56,6 +57,7 @@ void registerTableFunctions() registerTableFunctionFormat(factory); registerTableFunctionExplain(factory); + registerTableFunctionTimeSeries(factory); registerTableFunctionObjectStorage(factory); registerTableFunctionObjectStorageCluster(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index efde4d6dcdc..1dd6341b67e 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -23,6 +23,7 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory); void registerTableFunctionMongoDB(TableFunctionFactory & factory); void registerTableFunctionRedis(TableFunctionFactory & factory); void registerTableFunctionMergeTreeIndex(TableFunctionFactory & factory); +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerTableFunctionFuzzJSON(TableFunctionFactory & factory); #endif @@ -67,6 +68,8 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory); void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory); void registerDataLakeTableFunctions(TableFunctionFactory & factory); +void registerTableFunctionTimeSeries(TableFunctionFactory & factory); + void registerTableFunctions(); } diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index a59fd3e5a29..9eeda7161ee 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -731,7 +731,8 @@ class CiCache: job_config=reference_config, ): remove_from_workflow.append(job_name) - has_test_jobs_to_skip = True + if job_name != CI.JobNames.DOCS_CHECK: + has_test_jobs_to_skip = True else: required_builds += ( job_config.required_builds if job_config.required_builds else [] diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index dcca96370bd..a6287d5629b 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -497,9 +497,9 @@ def trigger_mergeable_check( description = format_description(description) if set_from_sync: - # update Mergeable Check from sync WF only if its status already present or its new status is not SUCCESS + # update Mergeable Check from sync WF only if its status already present or its new status is FAILURE # to avoid false-positives - if mergeable_status or state != SUCCESS: + if mergeable_status or state == FAILURE: set_mergeable_check(commit, description, state) elif mergeable_status is None or mergeable_status.description != description: set_mergeable_check(commit, description, state) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 22c52521c19..84718462ab5 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -20,6 +20,7 @@ from typing import Any, Dict from env_helper import IS_CI from integration_test_images import IMAGES +from tee_popen import TeePopen MAX_RETRY = 1 NUM_WORKERS = 5 @@ -356,20 +357,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Package found in %s", full_path) log_name = "install_" + f + ".log" log_path = os.path.join(str(self.path()), log_name) - with open(log_path, "w", encoding="utf-8") as log: - cmd = f"dpkg -x {full_path} ." - logging.info("Executing installation cmd %s", cmd) - with subprocess.Popen( - cmd, shell=True, stderr=log, stdout=log - ) as proc: - if proc.wait() == 0: - logging.info( - "Installation of %s successfull", full_path - ) - else: - raise RuntimeError( - f"Installation of {full_path} failed" - ) + cmd = f"dpkg -x {full_path} ." + logging.info("Executing installation cmd %s", cmd) + with TeePopen(cmd, log_file=log_path) as proc: + if proc.wait() == 0: + logging.info("Installation of %s successfull", full_path) + else: + raise RuntimeError(f"Installation of {full_path} failed") break else: raise FileNotFoundError(f"Package with {package} not found") diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 6ffedfdecd4..525b3bf367b 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -642,7 +642,7 @@ class TestCIConfig(unittest.TestCase): release_branch=True, ) for record_t_, records_ in ci_cache.records.items(): - if record_t_.value == CiCache.RecordType.FAILED.value: + if record_t_.value == record.record_type.value: records_[record.to_str_key()] = record ci_cache.filter_out_not_affected_jobs() @@ -716,7 +716,7 @@ class TestCIConfig(unittest.TestCase): release_branch=True, ) for record_t_, records_ in ci_cache.records.items(): - if record_t_.value == CiCache.RecordType.FAILED.value: + if record_t_.value == record.record_type.value: records_[record.to_str_key()] = record ci_cache.filter_out_not_affected_jobs() @@ -726,3 +726,42 @@ class TestCIConfig(unittest.TestCase): MOCK_REQUIRED_BUILDS, ) self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do) + + def test_ci_py_filters_not_affected_jobs_in_prs_docs_check(self): + """ + checks ci.py filters not affected jobs in PRs, + Docs Check is special from ci_cache perspective - + check it ci pr pipline is filtered properly when only docs check is to be skipped + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 123 + assert pr_info.is_pr + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list") + assert not ci_cache.jobs_to_wait + assert not ci_cache.jobs_to_skip + + job_config = ci_cache.jobs_to_do[CI.JobNames.DOCS_CHECK] + for batch in range(job_config.num_batches): + # add any record into cache + record = CiCache.Record( + record_type=CiCache.RecordType.PENDING, + job_name=CI.JobNames.DOCS_CHECK, + job_digest=ci_cache.job_digests[CI.JobNames.DOCS_CHECK], + batch=batch, + num_batches=job_config.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == record.record_type.value: + records_[record.to_str_key()] = record + + expected_jobs = list(ci_cache.jobs_to_do) + expected_jobs.remove(CI.JobNames.DOCS_CHECK) + ci_cache.filter_out_not_affected_jobs() + self.assertCountEqual(list(ci_cache.jobs_to_do), expected_jobs) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index ffdd6169777..a3d7e0e922d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -267,7 +267,7 @@ def clickhouse_execute_http( max_http_retries=5, retry_error_codes=False, ): - if args.secure: + if base_args.secure: client = http.client.HTTPSConnection( host=base_args.tcp_host, port=base_args.http_port, timeout=timeout ) @@ -358,14 +358,89 @@ def clickhouse_execute_json( return rows +# Should we capture client's stacktraces via SIGTSTP +CAPTURE_CLIENT_STACKTRACE = False + + +def kill_process_group(pgid): + print(f"Killing process group {pgid}") + print(f"Processes in process group {pgid}:") + print( + subprocess.check_output( + f"pgrep --pgroup {pgid} -a", shell=True, stderr=subprocess.STDOUT + ).decode("utf-8"), + end="", + ) + try: + if CAPTURE_CLIENT_STACKTRACE: + # Let's try to dump stacktrace in client (useful to catch issues there) + os.killpg(pgid, signal.SIGTSTP) + # Wait some time for clickhouse utilities to gather stacktrace + if RELEASE_NON_SANITIZED: + sleep(0.5) + else: + sleep(10) + # NOTE: this still may leave some processes, that had been + # created by timeout(1), since it also creates new process + # group. But this should not be a problem with default + # options, since the default time for each test is 10min, + # and this is way more bigger then the timeout for each + # timeout(1) invocation. + # + # But as a workaround we are sending SIGTERM first, and + # only after SIGKILL, that way timeout(1) will have an + # ability to terminate childrens (though not always since + # signals are asynchronous). + os.killpg(pgid, signal.SIGTERM) + # We need minimal delay to let processes handle SIGTERM - 0.1 (this may + # not be enough, but at least something) + sleep(0.1) + os.killpg(pgid, signal.SIGKILL) + except OSError as e: + if e.errno == ESRCH: + print(f"Got ESRCH while killing {pgid}. Ignoring.") + else: + raise + print(f"Process group {pgid} should be killed") + + +def cleanup_child_processes(pid): + pgid = os.getpgid(os.getpid()) + print(f"Child processes of {pid}:") + print( + subprocess.check_output( + f"pgrep --parent {pid} -a", shell=True, stderr=subprocess.STDOUT + ).decode("utf-8"), + end="", + ) + # Due to start_new_session=True, it is not enough to kill by PGID, we need + # to look at children processes as well. + # But we are hoping that nobody creates session in the tests (though it is + # possible via timeout(), but we are assuming that they will be killed by + # timeout). + processes = subprocess.check_output( + f"pgrep --parent {pid}", shell=True, stderr=subprocess.STDOUT + ) + processes = processes.decode("utf-8") + processes = processes.strip() + processes = processes.split("\n") + processes = map(lambda x: int(x.strip()), processes) + processes = list(processes) + for child in processes: + child_pgid = os.getpgid(child) + if child_pgid != pgid: + kill_process_group(child_pgid) + + # SIGKILL should not be sent, since this will kill the script itself + os.killpg(pgid, signal.SIGTERM) + + +# send signal to all processes in group to avoid hung check triggering +# (to avoid terminating clickhouse-test itself, the signal should be ignored) def stop_tests(): - # send signal to all processes in group to avoid hung check triggering - # (to avoid terminating clickhouse-test itself, the signal should be ignored) - print("Sending signals") signal.signal(signal.SIGTERM, signal.SIG_IGN) - os.killpg(os.getpgid(os.getpid()), signal.SIGTERM) - signal.signal(signal.SIGTERM, signal.SIG_DFL) - print("Sending signals DONE") + cleanup_child_processes(os.getpid()) + signal.signal(signal.SIGTERM, signal_handler) def get_db_engine(args, database_name): @@ -836,7 +911,6 @@ class SettingsRandomizer: "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), - "trace_profile_events": lambda: random.randint(0, 1), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), } @@ -1249,39 +1323,35 @@ class TestCase: return None - def process_result_impl( - self, proc, stdout: str, stderr: str, debug_log: str, total_time: float - ): + def process_result_impl(self, proc, total_time: float): + if proc: + if proc.returncode is None: + kill_process_group(os.getpgid(proc.pid)) + description = "" + debug_log = "" + if os.path.exists(self.testcase_args.debug_log_file): + with open(self.testcase_args.debug_log_file, "rb") as stream: + debug_log += self.testcase_args.debug_log_file + ":\n" + debug_log += str(stream.read(), errors="replace", encoding="utf-8") + debug_log += "\n" + + stdout = "" + if os.path.exists(self.stdout_file): + with open(self.stdout_file, "rb") as stdfd: + stdout = str(stdfd.read(), errors="replace", encoding="utf-8") + + stderr = "" + if os.path.exists(self.stderr_file): + with open(self.stderr_file, "rb") as stdfd: + stderr += str(stdfd.read(), errors="replace", encoding="utf-8") + if debug_log: debug_log = "\n".join(debug_log.splitlines()[:100]) if proc: if proc.returncode is None: - try: - pgid = os.getpgid(proc.pid) - # NOTE: this still may leave some processes, that had been - # created by timeout(1), since it also creates new process - # group. But this should not be a problem with default - # options, since the default time for each test is 10min, - # and this is way more bigger then the timeout for each - # timeout(1) invocation. - # - # But as a workaround we are sending SIGTERM first, and - # only after SIGKILL, that way timeout(1) will have an - # ability to terminate childrens (though not always since - # signals are asynchronous). - os.killpg(pgid, signal.SIGTERM) - # This may not be enough, but this is at least something - # (and anyway it is OK to spend 0.1 second more in case of - # test timeout). - sleep(0.1) - os.killpg(pgid, signal.SIGKILL) - except OSError as e: - if e.errno != ESRCH: - raise - if stderr: description += stderr if debug_log: @@ -1533,7 +1603,7 @@ class TestCase: def run_single_test( self, server_logs_level, client_options - ) -> Tuple[Optional[Popen], str, str, str, float]: + ) -> Tuple[Optional[Popen], float]: args = self.testcase_args client = args.testcase_client start_time = args.testcase_start_time @@ -1610,13 +1680,6 @@ class TestCase: # Whether the test timed out will be decided later pass - debug_log = "" - if os.path.exists(self.testcase_args.debug_log_file): - with open(self.testcase_args.debug_log_file, "rb") as stream: - debug_log += self.testcase_args.debug_log_file + ":\n" - debug_log += str(stream.read(), errors="replace", encoding="utf-8") - debug_log += "\n" - total_time = (datetime.now() - start_time).total_seconds() # Normalize randomized database names in stdout, stderr files. @@ -1668,17 +1731,7 @@ class TestCase: "https://localhost:8443/", ) - stdout = "" - if os.path.exists(self.stdout_file): - with open(self.stdout_file, "rb") as stdfd: - stdout = str(stdfd.read(), errors="replace", encoding="utf-8") - - stderr = "" - if os.path.exists(self.stderr_file): - with open(self.stderr_file, "rb") as stdfd: - stderr += str(stdfd.read(), errors="replace", encoding="utf-8") - - return proc, stdout, stderr, debug_log, total_time + return proc, total_time def run(self, args, suite, client_options, server_logs_level): start_time = datetime.now() @@ -1710,14 +1763,14 @@ class TestCase: if not is_valid_utf_8(self.case_file) or ( self.reference_file and not is_valid_utf_8(self.reference_file) ): - proc, stdout, stderr, debug_log, total_time = self.run_single_test( + proc, total_time = self.run_single_test( server_logs_level, client_options ) - result = self.process_result_impl( - proc, stdout, stderr, debug_log, total_time + result = self.process_result_impl(proc, total_time) + result.check_if_need_retry( + args, result.description, result.description, self.runs_count ) - result.check_if_need_retry(args, stdout, stderr, self.runs_count) # to avoid breaking CSV parser result.description = result.description.replace("\0", "") else: @@ -1735,17 +1788,16 @@ class TestCase: ): ( proc, - stdout, - stderr, - debug_log, total_time, ) = self.run_single_test(server_logs_level, client_options) - result = self.process_result_impl( - proc, stdout, stderr, debug_log, total_time - ) + result = self.process_result_impl(proc, total_time) + result.check_if_need_retry( - args, stdout, stderr, self.runs_count + args, + result.description, + result.description, + self.runs_count, ) # to avoid breaking CSV parser result.description = result.description.replace("\0", "") @@ -2354,7 +2406,13 @@ class BuildFlags: POLYMORPHIC_PARTS = "polymorphic-parts" +# Release and non-sanitizer build +RELEASE_NON_SANITIZED = False + + def collect_build_flags(args): + global RELEASE_NON_SANITIZED + result = [] value = clickhouse_execute( @@ -2379,6 +2437,8 @@ def collect_build_flags(args): elif b"RelWithDebInfo" in value or b"Release" in value: result.append(BuildFlags.RELEASE) + RELEASE_NON_SANITIZED = result == [BuildFlags.RELEASE] + value = clickhouse_execute( args, "SELECT value FROM system.settings WHERE name = 'allow_deprecated_database_ordinary'", @@ -3391,29 +3451,36 @@ def parse_args(): default="./client.fatal.log", help="Path to file for fatal logs from client", ) + parser.add_argument( + "--capture-client-stacktrace", + action="store_true", + help="Capture stacktraces from clickhouse-client/local on errors", + ) return parser.parse_args() class Terminated(KeyboardInterrupt): - pass + def __init__(self, signal): + self.signal = signal -def signal_handler(sig, frame): - raise Terminated(f"Terminated with {sig} signal") +def signal_handler(signal, frame): + raise Terminated(signal) if __name__ == "__main__": + # Move to a new process group and kill it at exit so that we don't have any + # infinite tests processes left + # (new process group is required to avoid killing some parent processes) + os.setpgid(0, 0) + stop_time = None exit_code = multiprocessing.Value("i", 0) server_died = multiprocessing.Event() multiprocessing_manager = multiprocessing.Manager() restarted_tests = multiprocessing_manager.list() - # Move to a new process group and kill it at exit so that we don't have any - # infinite tests processes left - # (new process group is required to avoid killing some parent processes) - os.setpgid(0, 0) signal.signal(signal.SIGTERM, signal_handler) signal.signal(signal.SIGINT, signal_handler) signal.signal(signal.SIGHUP, signal_handler) @@ -3431,6 +3498,8 @@ if __name__ == "__main__": ) sys.exit(1) + CAPTURE_CLIENT_STACKTRACE = args.capture_client_stacktrace + # Autodetect the directory with queries if not specified if args.queries is None: args.queries = "queries" @@ -3552,4 +3621,14 @@ if __name__ == "__main__": if args.replace_replicated_with_shared: args.s3_storage = True - main(args) + try: + main(args) + except ServerDied as e: + print(f"{e}", file=sys.stderr) + sys.exit(1) + except Terminated as e: + print(f"Terminated with {e.signal} signal", file=sys.stderr) + sys.exit(128 + e.signal) + except KeyboardInterrupt: + print("Interrupted") + sys.exit(128 + signal.SIGINT) diff --git a/tests/integration/compose/docker_compose_prometheus.yml b/tests/integration/compose/docker_compose_prometheus.yml new file mode 100644 index 00000000000..0a1db2138ba --- /dev/null +++ b/tests/integration/compose/docker_compose_prometheus.yml @@ -0,0 +1,57 @@ +version: '2.3' +services: + prometheus_writer: + image: prom/prometheus:v2.50.1 + hostname: ${PROMETHEUS_WRITER_HOSTNAME:-prometheus_writer} + restart: always + entrypoint: | + /bin/sh -c 'truncate -s 0 /etc/prometheus/prometheus.yml + cat << EOF >> /etc/prometheus/prometheus.yml + global: + scrape_interval: 1s + scrape_configs: + - job_name: "prometheus" + static_configs: + - targets: ["localhost:${PROMETHEUS_WRITER_PORT}"] + EOF + if [ -n "${PROMETHEUS_REMOTE_WRITE_HANDLER}" ]; then + echo "remote_write:" >> /etc/prometheus/prometheus.yml + echo " - url: \"${PROMETHEUS_REMOTE_WRITE_HANDLER}\"" >> /etc/prometheus/prometheus.yml + fi + #cat "/etc/prometheus/prometheus.yml" + /bin/prometheus --config.file="/etc/prometheus/prometheus.yml" --storage.tsdb.path="/prometheus" --web.console.libraries="/usr/share/prometheus/console_libraries" --web.console.templates="/usr/share/prometheus/consoles" --web.listen-address="0.0.0.0:${PROMETHEUS_WRITER_PORT}" &> /var/log/prometheus/prometheus.log' + expose: + - ${PROMETHEUS_WRITER_PORT} + healthcheck: + test: curl -f "ttps://localhost:${PROMETHEUS_WRITER_PORT}/api/v1/status/runtimeinfo" || exit 1 + interval: 5s + timeout: 3s + retries: 30 + volumes: + - type: ${PROMETHEUS_WRITER_LOGS_FS:-tmpfs} + source: ${PROMETHEUS_WRITER_LOGS:-} + target: /var/log/prometheus + + prometheus_reader: + image: prom/prometheus:v2.50.1 + hostname: ${PROMETHEUS_READER_HOSTNAME:-prometheus_reader} + restart: always + entrypoint: | + /bin/sh -c 'truncate -s 0 /etc/prometheus/prometheus.yml + if [ -n "${PROMETHEUS_REMOTE_READ_HANDLER}" ]; then + echo "remote_read:" >> /etc/prometheus/prometheus.yml + echo " - url: \"${PROMETHEUS_REMOTE_READ_HANDLER}\"" >> /etc/prometheus/prometheus.yml + fi + #cat "/etc/prometheus/prometheus.yml" + /bin/prometheus --config.file="/etc/prometheus/prometheus.yml" --storage.tsdb.path="/prometheus" --web.console.libraries="/usr/share/prometheus/console_libraries" --web.console.templates="/usr/share/prometheus/consoles" --web.listen-address="0.0.0.0:${PROMETHEUS_READER_PORT}" &> /var/log/prometheus/prometheus.log' + expose: + - ${PROMETHEUS_READER_PORT} + healthcheck: + test: curl -f "ttps://localhost:${PROMETHEUS_READER_PORT}/api/v1/status/runtimeinfo" || exit 1 + interval: 5s + timeout: 3s + retries: 30 + volumes: + - type: ${PROMETHEUS_READER_LOGS_FS:-tmpfs} + source: ${PROMETHEUS_READER_LOGS:-} + target: /var/log/prometheus diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5e0352df617..215718463e8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -737,6 +737,25 @@ class ClickHouseCluster: self.jdbc_driver_dir = p.abspath(p.join(self.instances_dir, "jdbc_driver")) self.jdbc_driver_logs_dir = os.path.join(self.jdbc_driver_dir, "logs") + # available when with_prometheus == True + self.with_prometheus = False + self.prometheus_writer_host = "prometheus_writer" + self.prometheus_writer_port = 9090 + self.prometheus_writer_logs_dir = p.abspath( + p.join(self.instances_dir, "prometheus_writer/logs") + ) + self.prometheus_reader_host = "prometheus_reader" + self.prometheus_reader_port = 9091 + self.prometheus_reader_logs_dir = p.abspath( + p.join(self.instances_dir, "prometheus_reader/logs") + ) + self.prometheus_remote_write_handler_host = None + self.prometheus_remote_write_handler_port = 9092 + self.prometheus_remote_write_handler_path = "/write" + self.prometheus_remote_read_handler_host = None + self.prometheus_remote_read_handler_port = 9092 + self.prometheus_remote_read_handler_path = "/read" + self.docker_client = None self.is_up = False self.env = os.environ.copy() @@ -1619,6 +1638,42 @@ class ClickHouseCluster: ] return self.base_hive_cmd + def setup_prometheus_cmd(self, instance, env_variables, docker_compose_yml_dir): + env_variables["PROMETHEUS_WRITER_HOST"] = self.prometheus_writer_host + env_variables["PROMETHEUS_WRITER_PORT"] = str(self.prometheus_writer_port) + env_variables["PROMETHEUS_WRITER_LOGS"] = self.prometheus_writer_logs_dir + env_variables["PROMETHEUS_WRITER_LOGS_FS"] = "bind" + env_variables["PROMETHEUS_READER_HOST"] = self.prometheus_reader_host + env_variables["PROMETHEUS_READER_PORT"] = str(self.prometheus_reader_port) + env_variables["PROMETHEUS_READER_LOGS"] = self.prometheus_reader_logs_dir + env_variables["PROMETHEUS_READER_LOGS_FS"] = "bind" + if self.prometheus_remote_write_handler_host: + env_variables["PROMETHEUS_REMOTE_WRITE_HANDLER"] = ( + f"http://{self.prometheus_remote_write_handler_host}:{self.prometheus_remote_write_handler_port}/{self.prometheus_remote_write_handler_path.strip('/')}" + ) + if self.prometheus_remote_read_handler_host: + env_variables["PROMETHEUS_REMOTE_READ_HANDLER"] = ( + f"http://{self.prometheus_remote_read_handler_host}:{self.prometheus_remote_read_handler_port}/{self.prometheus_remote_read_handler_path.strip('/')}" + ) + if not self.with_prometheus: + self.with_prometheus = True + self.base_cmd.extend( + [ + "--file", + p.join(docker_compose_yml_dir, "docker_compose_prometheus.yml"), + ] + ) + self.base_prometheus_cmd = [ + "docker-compose", + "--env-file", + instance.env_file, + "--project-name", + self.project_name, + "--file", + p.join(docker_compose_yml_dir, "docker_compose_prometheus.yml"), + ] + return self.base_prometheus_cmd + def add_instance( self, name, @@ -1659,6 +1714,9 @@ class ClickHouseCluster: with_jdbc_bridge=False, with_hive=False, with_coredns=False, + with_prometheus=False, + handle_prometheus_remote_write=False, + handle_prometheus_remote_read=False, use_old_analyzer=None, hostname=None, env_variables=None, @@ -2001,6 +2059,17 @@ class ClickHouseCluster: self.setup_hive(instance, env_variables, docker_compose_yml_dir) ) + if with_prometheus: + if handle_prometheus_remote_write: + self.prometheus_remote_write_handler_host = instance.hostname + if handle_prometheus_remote_read: + self.prometheus_remote_read_handler_host = instance.hostname + cmds.append( + self.setup_prometheus_cmd( + instance, env_variables, docker_compose_yml_dir + ) + ) + logging.debug( "Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( self.name, @@ -3064,6 +3133,12 @@ class ClickHouseCluster: f"http://{self.jdbc_bridge_ip}:{self.jdbc_bridge_port}/ping" ) + if self.with_prometheus: + os.makedirs(self.prometheus_writer_logs_dir) + os.chmod(self.prometheus_writer_logs_dir, stat.S_IRWXU | stat.S_IRWXO) + os.makedirs(self.prometheus_reader_logs_dir) + os.chmod(self.prometheus_reader_logs_dir, stat.S_IRWXU | stat.S_IRWXO) + clickhouse_start_cmd = self.base_cmd + ["up", "-d", "--no-recreate"] logging.debug( ( diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference b/tests/integration/test_prometheus_protocols/__init__.py similarity index 100% rename from tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference rename to tests/integration/test_prometheus_protocols/__init__.py diff --git a/tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml b/tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml new file mode 100644 index 00000000000..d71cfcaf2c8 --- /dev/null +++ b/tests/integration/test_prometheus_protocols/configs/allow_experimental_time_series_table.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_prometheus_protocols/configs/prometheus.xml b/tests/integration/test_prometheus_protocols/configs/prometheus.xml new file mode 100644 index 00000000000..071a29620cf --- /dev/null +++ b/tests/integration/test_prometheus_protocols/configs/prometheus.xml @@ -0,0 +1,21 @@ + + + 9092 + + + /write + + remote_write +
default.prometheus
+ + + + /read + + remote_read + default.prometheus
+
+
+
+ + diff --git a/tests/integration/test_prometheus_protocols/test.py b/tests/integration/test_prometheus_protocols/test.py new file mode 100644 index 00000000000..0c75a8194c7 --- /dev/null +++ b/tests/integration/test_prometheus_protocols/test.py @@ -0,0 +1,177 @@ +import pytest +import time +import requests +from http import HTTPStatus +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/prometheus.xml"], + user_configs=["configs/allow_experimental_time_series_table.xml"], + with_prometheus=True, + handle_prometheus_remote_write=True, + handle_prometheus_remote_read=True, +) + + +def execute_query_on_prometheus_writer(query, timestamp): + return execute_query_impl( + cluster.get_instance_ip(cluster.prometheus_writer_host), + cluster.prometheus_writer_port, + "/api/v1/query", + query, + timestamp, + ) + + +def execute_query_on_prometheus_reader(query, timestamp): + return execute_query_impl( + cluster.get_instance_ip(cluster.prometheus_reader_host), + cluster.prometheus_reader_port, + "/api/v1/query", + query, + timestamp, + ) + + +def execute_query_impl(host, port, path, query, timestamp): + if not path.startswith("/"): + path += "/" + url = f"http://{host}:{port}/{path.strip('/')}?query={query}&time={timestamp}" + print(f"Requesting {url}") + r = requests.get(url) + print(f"Status code: {r.status_code} {HTTPStatus(r.status_code).phrase}") + if r.status_code != requests.codes.ok: + print(f"Response: {r.text}") + raise Exception(f"Got unexpected status code {r.status_code}") + return r.json() + + +def show_query_result(query): + evaluation_time = time.time() + print(f"Evaluating query: {query}") + print(f"Evaluation time: {evaluation_time}") + result_from_writer = execute_query_on_prometheus_writer(query, evaluation_time) + print(f"Result from prometheus_writer: {result_from_writer}") + result_from_reader = execute_query_on_prometheus_reader(query, evaluation_time) + print(f"Result from prometheus_reader: {result_from_reader}") + + +def compare_query(query): + timeout = 60 + start_time = time.time() + evaluation_time = start_time + print(f"Evaluating query: {query}") + print(f"Evaluation time: {evaluation_time}") + while time.time() < start_time + timeout: + result_from_writer = execute_query_on_prometheus_writer(query, evaluation_time) + time.sleep(1) + result_from_reader = execute_query_on_prometheus_reader(query, evaluation_time) + print(f"Result from prometheus_writer: {result_from_writer}") + print(f"Result from prometheus_reader: {result_from_reader}") + if result_from_writer == result_from_reader: + return + raise Exception( + f"Got different results from prometheus_writer and prometheus_reader" + ) + + +def compare_queries(): + compare_query("up") + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + node.query("DROP TABLE IF EXISTS prometheus SYNC") + node.query("DROP TABLE IF EXISTS original SYNC") + node.query("DROP TABLE IF EXISTS mydata SYNC") + node.query("DROP TABLE IF EXISTS mytable SYNC") + node.query("DROP TABLE IF EXISTS mymetrics SYNC") + + +def test_default(): + node.query("CREATE TABLE prometheus ENGINE=TimeSeries") + compare_queries() + + +def test_tags_to_columns(): + node.query( + "CREATE TABLE prometheus ENGINE=TimeSeries SETTINGS tags_to_columns = {'job': 'job', 'instance': 'instance'}" + ) + compare_queries() + + +def test_64bit_id(): + node.query("CREATE TABLE prometheus (id UInt64) ENGINE=TimeSeries") + compare_queries() + + +def test_custom_id_algorithm(): + node.query( + "CREATE TABLE prometheus (id FixedString(16) DEFAULT murmurHash3_128(metric_name, all_tags)) ENGINE=TimeSeries" + ) + compare_queries() + + +def test_create_as_table(): + node.query("CREATE TABLE original ENGINE=TimeSeries") + node.query("CREATE TABLE prometheus AS original") + compare_queries() + + +def test_inner_engines(): + node.query( + "CREATE TABLE prometheus ENGINE=TimeSeries " + "DATA ENGINE=MergeTree ORDER BY (id, timestamp) " + "TAGS ENGINE=AggregatingMergeTree ORDER BY (metric_name, id) " + "METRICS ENGINE=ReplacingMergeTree ORDER BY metric_family_name" + ) + compare_queries() + + +def test_external_tables(): + node.query("DROP TABLE IF EXISTS mydata") + node.query("DROP TABLE IF EXISTS mytags") + node.query("DROP TABLE IF EXISTS mymetrics") + node.query("DROP TABLE IF EXISTS prometheus") + + node.query( + "CREATE TABLE mydata (id UUID, timestamp DateTime64(3), value Float64) " + "ENGINE=MergeTree ORDER BY (id, timestamp)" + ) + node.query( + "CREATE TABLE mytags (" + "id UUID, " + "metric_name LowCardinality(String), " + "tags Map(LowCardinality(String), String), " + "min_time SimpleAggregateFunction(min, Nullable(DateTime64(3))), " + "max_time SimpleAggregateFunction(max, Nullable(DateTime64(3)))) " + "ENGINE=AggregatingMergeTree ORDER BY (metric_name, id)" + ) + + # FIXME: The table structure should be: + # "CREATE TABLE mymetrics (metric_family_name String, type LowCardinality(String), unit LowCardinality(String), help String)" + # Renamed it because of the bug and potential type mismatch. + node.query( + "CREATE TABLE mymetrics (metric_family_name String, type String, unit String, help String) " + "ENGINE=ReplacingMergeTree ORDER BY metric_family_name" + ) + node.query( + "CREATE TABLE prometheus ENGINE=TimeSeries " + "DATA mydata TAGS mytags METRICS mymetrics" + ) + compare_queries() diff --git a/tests/integration/test_recovery_time_metric/__init__.py b/tests/integration/test_recovery_time_metric/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_recovery_time_metric/configs/config.xml b/tests/integration/test_recovery_time_metric/configs/config.xml new file mode 100644 index 00000000000..bad9b1fa9ea --- /dev/null +++ b/tests/integration/test_recovery_time_metric/configs/config.xml @@ -0,0 +1,41 @@ + + 9000 + + + + + + + + + default + + + + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 20000 + + + + 1 + localhost + 9444 + + + + + + + localhost + 2181 + + 20000 + + + diff --git a/tests/integration/test_recovery_time_metric/test.py b/tests/integration/test_recovery_time_metric/test.py new file mode 100644 index 00000000000..6fcf2fad423 --- /dev/null +++ b/tests/integration/test_recovery_time_metric/test.py @@ -0,0 +1,61 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_recovery_time_metric(start_cluster): + node.query( + """ + DROP DATABASE IF EXISTS rdb; + CREATE DATABASE rdb + ENGINE = Replicated('/test/test_recovery_time_metric', 'shard1', 'replica1') + """ + ) + + node.query( + """ + DROP TABLE IF EXISTS rdb.t; + CREATE TABLE rdb.t + ( + `x` UInt32 + ) + ENGINE = MergeTree + ORDER BY x + """ + ) + + node.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/rdb/t.sql"]) + + node.restart_clickhouse() + + ret = int( + node.query( + """ + SELECT recovery_time + FROM system.clusters + WHERE cluster = 'rdb' + """ + ).strip() + ) + assert ret > 0 + + node.query( + """ + DROP DATABASE rdb + """ + ) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index e3d56c52020..aef5ddb3675 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -64,6 +64,17 @@ def test_read_write_storage_with_globs(started_cluster): hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" + node1.query( + "create table HDFSStorageWithDoubleAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/**.doublestar.tsv', 'TSV')" + ) + + for i in ["1", "2", "3"]: + hdfs_api.write_data(f"/subdir{i}/file{i}.doublestar.tsv", f"{i}\tMark\t72.53\n") + assert ( + hdfs_api.read_data(f"/subdir{i}/file{i}.doublestar.tsv") + == f"{i}\tMark\t72.53\n" + ) + assert ( node1.query( "select count(*) from HDFSStorageWithRange settings s3_throw_on_zero_files_match=1" @@ -73,6 +84,7 @@ def test_read_write_storage_with_globs(started_cluster): assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == "3\n" assert node1.query("select count(*) from HDFSStorageWithAsterisk") == "3\n" + assert node1.query("select count(*) from HDFSStorageWithDoubleAsterisk") == "3\n" try: node1.query("insert into HDFSStorageWithEnum values (1, 'NEW', 4.2)") @@ -95,10 +107,18 @@ def test_read_write_storage_with_globs(started_cluster): print(ex) assert "in readonly mode" in str(ex) + try: + node1.query("insert into HDFSStorageWithDoubleAsterisk values (1, 'NEW', 4.2)") + assert False, "Exception have to be thrown" + except Exception as ex: + print(ex) + assert "in readonly mode" in str(ex) + node1.query("drop table HDFSStorageWithRange") node1.query("drop table HDFSStorageWithEnum") node1.query("drop table HDFSStorageWithQuestionMark") node1.query("drop table HDFSStorageWithAsterisk") + node1.query("drop table HDFSStorageWithDoubleAsterisk") def test_storage_with_multidirectory_glob(started_cluster): diff --git a/tests/integration/test_storage_kafka/configs/kafka.xml b/tests/integration/test_storage_kafka/configs/kafka.xml index b10db879b72..a846fdbb295 100644 --- a/tests/integration/test_storage_kafka/configs/kafka.xml +++ b/tests/integration/test_storage_kafka/configs/kafka.xml @@ -48,7 +48,7 @@ - 30001 + 30001 60001 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 37457e00701..4b6c9922d74 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -29,6 +29,7 @@ from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnecti from kafka.protocol.admin import DescribeGroupsRequest_v1 from kafka.protocol.group import MemberAssignment from kafka.admin import NewTopic +from contextlib import contextmanager # protoc --version @@ -46,6 +47,13 @@ if is_arm(): # TODO: add test for run-time offset update in CH, if we manually update it on Kafka side. # TODO: add test for SELECT LIMIT is working. + +KAFKA_TOPIC_OLD = "old_t" +KAFKA_CONSUMER_GROUP_OLD = "old_cg" +KAFKA_TOPIC_NEW = "new_t" +KAFKA_CONSUMER_GROUP_NEW = "new_cg" + + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", @@ -55,10 +63,10 @@ instance = cluster.add_instance( with_zookeeper=True, # For Replicated Table macros={ "kafka_broker": "kafka1", - "kafka_topic_old": "old", - "kafka_group_name_old": "old", - "kafka_topic_new": "new", - "kafka_group_name_new": "new", + "kafka_topic_old": KAFKA_TOPIC_OLD, + "kafka_group_name_old": KAFKA_CONSUMER_GROUP_OLD, + "kafka_topic_new": KAFKA_TOPIC_NEW, + "kafka_group_name_new": KAFKA_CONSUMER_GROUP_NEW, "kafka_client_id": "instance", "kafka_format_json_each_row": "JSONEachRow", }, @@ -142,6 +150,44 @@ def kafka_delete_topic(admin_client, topic, max_retries=50): raise Exception(f"Failed to delete topics {topic}, {result}") +@contextmanager +def kafka_topic( + admin_client, + topic_name, + num_partitions=1, + replication_factor=1, + max_retries=50, + config=None, +): + kafka_create_topic( + admin_client, + topic_name, + num_partitions, + replication_factor, + max_retries, + config, + ) + try: + yield None + finally: + # Code to release resource, e.g.: + kafka_delete_topic(admin_client, topic_name, max_retries) + + +@contextmanager +def existing_kafka_topic(admin_client, topic_name, max_retries=50): + try: + yield None + finally: + kafka_delete_topic(admin_client, topic_name, max_retries) + + +def get_admin_client(kafka_cluster): + return KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): logging.debug( "kafka_produce server:{}:{} topic:{}".format( @@ -161,7 +207,7 @@ def kafka_producer_send_heartbeat_msg(max_retries=50): kafka_produce(kafka_cluster, "test_heartbeat_topic", ["test"], retries=max_retries) -def kafka_consume(kafka_cluster, topic, needDecode=True, timestamp=0): +def kafka_consume(kafka_cluster, topic, need_decode=True, timestamp=0): consumer = KafkaConsumer( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest", @@ -171,7 +217,7 @@ def kafka_consume(kafka_cluster, topic, needDecode=True, timestamp=0): if toppar.topic == topic: for message in messages: assert timestamp == 0 or message.timestamp / 1000 == timestamp - if needDecode: + if need_decode: yield message.value.decode() else: yield message.value @@ -196,7 +242,33 @@ def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messa logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) -def kafka_produce_protobuf_messages_no_delimeters( +def kafka_consume_with_retry( + kafka_cluster, + topic, + expected_messages, + need_decode=True, + timestamp=0, + retry_count=20, + sleep_time=0.1, +): + messages = [] + try_count = 0 + while try_count < retry_count: + try_count += 1 + messages.extend( + kafka_consume( + kafka_cluster, topic, need_decode=need_decode, timestamp=timestamp + ) + ) + if len(messages) == expected_messages: + break + time.sleep(sleep_time) + if len(messages) != expected_messages: + raise Exception(f"Got only {len(messages)} messages") + return messages + + +def kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, topic, start_index, num_messages ): data = "" @@ -284,10 +356,99 @@ def avro_confluent_message(schema_registry_client, value): return serializer.encode_record_with_schema("test_subject", schema, value) +def create_settings_string(settings): + if settings is None: + return "" + + def format_value(value): + if isinstance(value, str): + return f"'{value}'" + elif isinstance(value, bool): + return str(int(value)) + return str(value) + + settings_string = "SETTINGS " + keys = settings.keys() + first_key = next(iter(settings)) + settings_string += str(first_key) + " = " + format_value(settings[first_key]) + for key in keys: + if key == first_key: + continue + settings_string += ", " + str(key) + " = " + format_value(settings[key]) + return settings_string + + +def generate_old_create_table_query( + table_name, + columns_def, + database="test", + brokers="{kafka_broker}:19092", + topic_list="{kafka_topic_new}", + consumer_group="{kafka_group_name_new}", + format="{kafka_format_json_each_row}", + row_delimiter="\\n", + keeper_path=None, # it is not used, but it is easier to handle keeper_path and replica_name like this + replica_name=None, + settings=None, +): + settings_string = create_settings_string(settings) + query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') +{settings_string}""" + logging.debug(f"Generated old create query: {query}") + return query + + +def generate_new_create_table_query( + table_name, + columns_def, + database="test", + brokers="{kafka_broker}:19092", + topic_list="{kafka_topic_new}", + consumer_group="{kafka_group_name_new}", + format="{kafka_format_json_each_row}", + row_delimiter="\\n", + keeper_path=None, + replica_name=None, + settings=None, +): + if settings is None: + settings = {} + if keeper_path is None: + keeper_path = f"/clickhouse/{{database}}/{table_name}" + if replica_name is None: + replica_name = "r1" + settings["kafka_keeper_path"] = keeper_path + settings["kafka_replica_name"] = replica_name + settings_string = create_settings_string(settings) + query = f"""CREATE TABLE {database}.{table_name} ({columns_def}) ENGINE = Kafka('{brokers}', '{topic_list}', '{consumer_group}', '{format}', '{row_delimiter}') +{settings_string} +SETTINGS allow_experimental_kafka_offsets_storage_in_keeper=1""" + logging.debug(f"Generated new create query: {query}") + return query + + +def must_use_thread_per_consumer(generator): + if generator == generate_old_create_table_query: + return False + if generator == generate_new_create_table_query: + return True + raise Exception("Unexpected generator") + + +def get_topic_postfix(generator): + if generator == generate_old_create_table_query: + return "old" + if generator == generate_new_create_table_query: + return "new" + raise Exception("Unexpected generator") + + # Tests - - -def test_kafka_column_types(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator, do_direct_read", + [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], +) +def test_kafka_column_types(kafka_cluster, create_query_generator, do_direct_read): def assert_returned_exception(e): assert e.value.returncode == 36 assert ( @@ -297,57 +458,14 @@ def test_kafka_column_types(kafka_cluster): # check column with DEFAULT expression with pytest.raises(QueryRuntimeException) as exception: - instance.query( - """ - CREATE TABLE test.kafka (a Int, b Int DEFAULT 0) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') - """ - ) + instance.query(create_query_generator("kafka", "a Int, b Int DEFAULT 0")) assert_returned_exception(exception) # check EPHEMERAL with pytest.raises(QueryRuntimeException) as exception: - instance.query( - """ - CREATE TABLE test.kafka (a Int, b Int EPHEMERAL) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') - """ - ) + instance.query(create_query_generator("kafka", "a Int, b Int EPHEMERAL")) assert_returned_exception(exception) - # check ALIAS - instance.query( - """ - CREATE TABLE test.kafka (a Int, b String Alias toString(a)) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') - SETTINGS kafka_commit_on_select = 1; - """ - ) - messages = [] - for i in range(5): - messages.append(json.dumps({"a": i})) - kafka_produce(kafka_cluster, "new", messages) - result = "" - expected = TSV( - """ -0\t0 -1\t1 -2\t2 -3\t3 -4\t4 - """ - ) - retries = 50 - while retries > 0: - result += instance.query("SELECT a, b FROM test.kafka", ignore_error=True) - if TSV(result) == expected: - break - retries -= 1 - - assert TSV(result) == expected - - instance.query("DROP TABLE test.kafka SYNC") - # check MATERIALIZED with pytest.raises(QueryRuntimeException) as exception: instance.query( @@ -358,6 +476,41 @@ def test_kafka_column_types(kafka_cluster): ) assert_returned_exception(exception) + if do_direct_read: + # check ALIAS + instance.query( + create_query_generator( + "kafka", + "a Int, b String Alias toString(a)", + settings={"kafka_commit_on_select": True}, + ) + ) + messages = [] + for i in range(5): + messages.append(json.dumps({"a": i})) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, messages) + result = "" + expected = TSV( + """ + 0\t0 + 1\t1 + 2\t2 + 3\t3 + 4\t4 + """ + ) + retries = 50 + while retries > 0: + result += instance.query("SELECT a, b FROM test.kafka", ignore_error=True) + if TSV(result) == expected: + break + retries -= 1 + time.sleep(0.5) + + assert TSV(result) == expected + + instance.query("DROP TABLE test.kafka SYNC") + def test_kafka_settings_old_syntax(kafka_cluster): assert TSV( @@ -366,13 +519,13 @@ def test_kafka_settings_old_syntax(kafka_cluster): ignore_error=True, ) ) == TSV( - """kafka_broker kafka1 + f"""kafka_broker kafka1 kafka_client_id instance kafka_format_json_each_row JSONEachRow -kafka_group_name_new new -kafka_group_name_old old -kafka_topic_new new -kafka_topic_old old +kafka_group_name_new {KAFKA_CONSUMER_GROUP_NEW} +kafka_group_name_old {KAFKA_CONSUMER_GROUP_OLD} +kafka_topic_new new_t +kafka_topic_old old_t """ ) @@ -389,7 +542,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "old", messages) + kafka_produce(kafka_cluster, KAFKA_TOPIC_OLD, messages) result = "" while True: @@ -399,7 +552,7 @@ kafka_topic_old old kafka_check_result(result, True) - members = describe_consumer_group(kafka_cluster, "old") + members = describe_consumer_group(kafka_cluster, KAFKA_CONSUMER_GROUP_OLD) assert members[0]["client_id"] == "ClickHouse-instance-test-kafka" # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @@ -423,16 +576,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "new", messages) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, messages) # Insert couple of malformed messages. - kafka_produce(kafka_cluster, "new", ["}{very_broken_message,"]) - kafka_produce(kafka_cluster, "new", ["}another{very_broken_message,"]) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, ["}{very_broken_message,"]) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, ["}another{very_broken_message,"]) messages = [] for i in range(25, 50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "new", messages) + kafka_produce(kafka_cluster, KAFKA_TOPIC_NEW, messages) result = "" while True: @@ -442,7 +595,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) - members = describe_consumer_group(kafka_cluster, "new") + members = describe_consumer_group(kafka_cluster, KAFKA_CONSUMER_GROUP_NEW) assert members[0]["client_id"] == "instance test 1234" @@ -520,12 +673,13 @@ def test_kafka_json_as_string(kafka_cluster): ) -def test_kafka_formats(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_formats(kafka_cluster, create_query_generator): schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(kafka_cluster.schema_registry_port) - ) - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + {"url": f"http://localhost:{kafka_cluster.schema_registry_port}"} ) # data was dumped from clickhouse itself in a following manner @@ -649,7 +803,7 @@ def test_kafka_formats(kafka_cluster): '(id = 1, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 2, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 3, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 4, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 5, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 6, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 7, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 8, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 9, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 10, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 11, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 12, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 13, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 14, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 15, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', ], - "extra_settings": ", format_template_row='template_row.format'", + "extra_settings": {"format_template_row": "template_row.format"}, }, "Regexp": { "data_sample": [ @@ -660,7 +814,10 @@ def test_kafka_formats(kafka_cluster): # On empty message exception happens: Line "" doesn't match the regexp.: (at row 1) # /src/Processors/Formats/Impl/RegexpRowInputFormat.cpp:140: DB::RegexpRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df82fcb in /usr/bin/clickhouse ], - "extra_settings": r", format_regexp='\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)', format_regexp_escaping_rule='Escaped'", + "extra_settings": { + "format_regexp": r"\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)", + "format_regexp_escaping_rule": "Escaped", + }, }, ## BINARY FORMATS # dumped with @@ -732,7 +889,7 @@ def test_kafka_formats(kafka_cluster): # /src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp:25: DB::ProtobufRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df4cc71 in /usr/bin/clickhouse # /src/Processors/Formats/IRowInputFormat.cpp:64: DB::IRowInputFormat::generate() @ 0x1de727cf in /usr/bin/clickhouse ], - "extra_settings": ", kafka_schema='test:TestMessage'", + "extra_settings": {"kafka_schema": "test:TestMessage"}, }, "ORC": { "data_sample": [ @@ -756,7 +913,7 @@ def test_kafka_formats(kafka_cluster): # /src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp:212: DB::CapnProtoRowInputFormat::readMessage() @ 0x1ded1cab in /usr/bin/clickhouse # /src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp:241: DB::CapnProtoRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1ded205d in /usr/bin/clickhouse ], - "extra_settings": ", kafka_schema='test:TestRecordStruct'", + "extra_settings": {"kafka_schema": "test:TestRecordStruct"}, }, "Parquet": { "data_sample": [ @@ -791,9 +948,12 @@ def test_kafka_formats(kafka_cluster): {"id": 0, "blockNo": 0, "val1": str("AM"), "val2": 0.5, "val3": 1}, ), ], - "extra_settings": ", format_avro_schema_registry_url='http://{}:{}'".format( - kafka_cluster.schema_registry_host, kafka_cluster.schema_registry_port - ), + "extra_settings": { + "format_avro_schema_registry_url": "http://{}:{}".format( + kafka_cluster.schema_registry_host, + kafka_cluster.schema_registry_port, + ) + }, "supports_empty_value": True, }, "Avro": { @@ -837,31 +997,25 @@ def test_kafka_formats(kafka_cluster): }, } + topic_postfix = str(hash(create_query_generator)) for format_name, format_opts in list(all_formats.items()): - logging.debug(("Set up {}".format(format_name))) - topic_name = "format_tests_{}".format(format_name) + logging.debug(f"Set up {format_name}") + topic_name = f"format_tests_{format_name}-{topic_postfix}" data_sample = format_opts["data_sample"] data_prefix = [] # prepend empty value when supported if format_opts.get("supports_empty_value", False): data_prefix = data_prefix + [""] kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) + + extra_settings = format_opts.get("extra_settings") or {} + extra_settings["kafka_flush_interval_ms"] = 1000 + instance.query( """ DROP TABLE IF EXISTS test.kafka_{format_name}; - CREATE TABLE test.kafka_{format_name} ( - id Int64, - blockNo UInt16, - val1 String, - val2 Float32, - val3 UInt8 - ) ENGINE = Kafka() - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic_name}', - kafka_group_name = '{topic_name}_group', - kafka_format = '{format_name}', - kafka_flush_interval_ms = 1000 {extra_settings}; + {create_query}; DROP TABLE IF EXISTS test.kafka_{format_name}_mv; @@ -870,7 +1024,14 @@ def test_kafka_formats(kafka_cluster): """.format( topic_name=topic_name, format_name=format_name, - extra_settings=format_opts.get("extra_settings") or "", + create_query=create_query_generator( + f"kafka_{format_name}", + "id Int64, blockNo UInt16, val1 String, val2 Float32, val3 UInt8", + topic_list=f"{topic_name}", + consumer_group=f"{topic_name}_group", + format=format_name, + settings=extra_settings, + ), ) ) raw_expected = """\ @@ -905,13 +1066,16 @@ def test_kafka_formats(kafka_cluster): for format_name, format_opts in list(all_formats.items()): logging.debug(("Checking {}".format(format_name))) - topic_name = f"format_tests_{format_name}" + topic_name = f"format_tests_{format_name}-{topic_postfix}" # shift offsets by 1 if format supports empty value offsets = ( [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] ) - result = instance.query( - "SELECT * FROM test.kafka_{format_name}_mv;".format(format_name=format_name) + result = instance.query_with_retry( + "SELECT * FROM test.kafka_{format_name}_mv;".format( + format_name=format_name + ), + check_callback=lambda x: x.count("\n") == raw_expected.count("\n"), ) expected = raw_expected.format( topic_name=topic_name, @@ -922,7 +1086,7 @@ def test_kafka_formats(kafka_cluster): assert TSV(result) == TSV(expected), "Proper result for format: {}".format( format_name ) - kafka_delete_topic(admin_client, topic_name) + kafka_delete_topic(get_admin_client(kafka_cluster), topic_name) # Since everything is async and shaky when receiving messages from Kafka, @@ -997,8 +1161,29 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): - instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") - # logging.debug("kafka is available - running test") + instance.query("DROP DATABASE IF EXISTS test SYNC; CREATE DATABASE test;") + admin_client = get_admin_client(cluster) + + def get_topics_to_delete(): + return [t for t in admin_client.list_topics() if not t.startswith("_")] + + topics = get_topics_to_delete() + logging.debug(f"Deleting topics: {topics}") + result = admin_client.delete_topics(topics) + for topic, error in result.topic_error_codes: + if error != 0: + logging.warning(f"Received error {error} while deleting topic {topic}") + else: + logging.info(f"Deleted topic {topic}") + + retries = 0 + topics = get_topics_to_delete() + while len(topics) != 0: + logging.info(f"Existing topics: {topics}") + if retries >= 5: + raise Exception(f"Failed to delete topics {topics}") + retries += 1 + time.sleep(0.5) yield # run test @@ -1098,9 +1283,7 @@ def test_kafka_issue4116(kafka_cluster): def test_kafka_consumer_hang(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "consumer_hang" kafka_create_topic(admin_client, topic_name, num_partitions=8) @@ -1180,9 +1363,7 @@ def test_kafka_consumer_hang(kafka_cluster): def test_kafka_consumer_hang2(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "consumer_hang2" kafka_create_topic(admin_client, topic_name) @@ -1243,9 +1424,7 @@ def test_kafka_consumer_hang2(kafka_cluster): # sequential read from different consumers leads to breaking lot of kafka invariants # (first consumer will get all partitions initially, and may have problems in doing polls every 60 sec) def test_kafka_read_consumers_in_parallel(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "read_consumers_in_parallel" kafka_create_topic(admin_client, topic_name, num_partitions=8) @@ -1365,9 +1544,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): def test_kafka_select_empty(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + admin_client = get_admin_client(kafka_cluster) topic_name = "empty" kafka_create_topic(admin_client, topic_name) @@ -1545,13 +1722,13 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): """ ) - kafka_produce_protobuf_messages_no_delimeters( + kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, "pb_no_delimiter", 0, 20 ) - kafka_produce_protobuf_messages_no_delimeters( + kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, "pb_no_delimiter", 20, 1 ) - kafka_produce_protobuf_messages_no_delimeters( + kafka_produce_protobuf_messages_no_delimiters( kafka_cluster, "pb_no_delimiter", 21, 29 ) @@ -1592,18 +1769,20 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): assert TSV(result) == TSV(expected) -def test_kafka_materialized_view(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_materialized_view(kafka_cluster, create_query_generator): + topic_name = "mv" + instance.query( - """ + f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'mv', - kafka_group_name = 'mv', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; + DROP TABLE IF EXISTS test.kafka; + + {create_query_generator("kafka", "key UInt64, value UInt64", topic_list=topic_name, consumer_group="mv")}; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -1615,110 +1794,124 @@ def test_kafka_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "mv", messages) + kafka_produce(kafka_cluster, topic_name, messages) - while True: - result = instance.query("SELECT * FROM test.view") - if kafka_check_result(result): - break + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + result = instance.query_with_retry( + "SELECT * FROM test.view", check_callback=kafka_check_result + ) - instance.query( + kafka_check_result(result, True) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.kafka; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - kafka_check_result(result, True) + ) -def test_kafka_recreate_kafka_table(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + ( + generate_new_create_table_query, + r"kafka.*Saved offset [0-9]+ for topic-partition \[recreate_kafka_table:[0-9]+", + ), + ( + generate_old_create_table_query, + "kafka.*Committed offset [0-9]+.*recreate_kafka_table", + ), + ], +) +def test_kafka_recreate_kafka_table(kafka_cluster, create_query_generator, log_line): """ Checks that materialized view work properly after dropping and recreating the Kafka table. """ - # line for backporting: - # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - topic_name = "recreate_kafka_table" - kafka_create_topic(admin_client, topic_name, num_partitions=6) + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) - instance.query( + with kafka_topic(get_admin_client(kafka_cluster), topic_name, num_partitions=6): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group="recreate_kafka_table_group", + settings={ + "kafka_num_consumers": 4, + "kafka_flush_interval_ms": 1000, + "kafka_skip_broken_messages": 1048577, + "kafka_thread_per_consumer": thread_per_consumer, + }, + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'recreate_kafka_table', - kafka_group_name = 'recreate_kafka_table_group', - kafka_format = 'JSONEachRow', - kafka_num_consumers = 6, - kafka_flush_interval_ms = 1000, - kafka_skip_broken_messages = 1048577; + ) - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) + messages = [] + for i in range(120): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, "recreate_kafka_table", messages) - messages = [] - for i in range(120): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "recreate_kafka_table", messages) + instance.wait_for_log_line( + log_line, + repetitions=6, + look_behind_lines=100, + ) - instance.wait_for_log_line( - "kafka.*Committed offset [0-9]+.*recreate_kafka_table", - repetitions=6, - look_behind_lines=100, - ) - - instance.query( + instance.query( + """ + DROP TABLE test.kafka; """ - DROP TABLE test.kafka; - """ - ) + ) - kafka_produce(kafka_cluster, "recreate_kafka_table", messages) + instance.rotate_logs() - instance.query( + kafka_produce(kafka_cluster, "recreate_kafka_table", messages) + + instance.query(create_query) + + instance.wait_for_log_line( + log_line, + repetitions=6, + look_behind_lines=100, + ) + + # data was not flushed yet (it will be flushed 7.5 sec after creating MV) + assert int(instance.query("SELECT count() FROM test.view")) == 240 + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.kafka; + DROP TABLE test.view; """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'recreate_kafka_table', - kafka_group_name = 'recreate_kafka_table_group', - kafka_format = 'JSONEachRow', - kafka_num_consumers = 6, - kafka_flush_interval_ms = 1000, - kafka_skip_broken_messages = 1048577; - """ - ) - - instance.wait_for_log_line( - "kafka.*Committed offset [0-9]+.*recreate_kafka_table", - repetitions=6, - look_behind_lines=100, - ) - - # data was not flushed yet (it will be flushed 7.5 sec after creating MV) - assert int(instance.query("SELECT count() FROM test.view")) == 240 - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - kafka_delete_topic(admin_client, topic_name) + ) -def test_librdkafka_compression(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_old_create_table_query, "Committed offset {offset}"), + ( + generate_new_create_table_query, + r"kafka.*Saved offset [0-9]+ for topic-partition \[{topic}:[0-9]+\]", + ), + ], +) +def test_librdkafka_compression(kafka_cluster, create_query_generator, log_line): """ Regression for UB in snappy-c (that is used in librdkafka), backport pr is [1]. @@ -1754,58 +1947,68 @@ def test_librdkafka_compression(kafka_cluster): expected = "\n".join(expected) + admin_client = get_admin_client(kafka_cluster) + for compression_type in supported_compression_types: logging.debug(("Check compression {}".format(compression_type))) topic_name = "test_librdkafka_compression_{}".format(compression_type) - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + topic_config = {"compression.type": compression_type} + with kafka_topic(admin_client, topic_name, config=topic_config): + instance.query( + """{create_query}; - kafka_create_topic( - admin_client, topic_name, config={"compression.type": compression_type} - ) + CREATE TABLE test.view (key UInt64, value String) + ENGINE = MergeTree() + ORDER BY key; - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic_name}', - kafka_group_name = '{topic_name}_group', - kafka_format = 'JSONEachRow', - kafka_flush_interval_ms = 1000; - CREATE MATERIALIZED VIEW test.consumer Engine=Log AS - SELECT * FROM test.kafka; - """.format( - topic_name=topic_name + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """.format( + create_query=create_query_generator( + "kafka", + "key UInt64, value String", + topic_list=topic_name, + format="JSONEachRow", + settings={"kafka_flush_interval_ms": 1000}, + ), + ) ) - ) - kafka_produce(kafka_cluster, topic_name, messages) + kafka_produce(kafka_cluster, topic_name, messages) - instance.wait_for_log_line("Committed offset {}".format(number_of_messages)) + instance.wait_for_log_line( + log_line.format(offset=number_of_messages, topic=topic_name) + ) + result = instance.query("SELECT * FROM test.view") + assert TSV(result) == TSV(expected) - result = instance.query("SELECT * FROM test.consumer") - assert TSV(result) == TSV(expected) - - instance.query("DROP TABLE test.kafka SYNC") - instance.query("DROP TABLE test.consumer SYNC") - kafka_delete_topic(admin_client, topic_name) + instance.query("DROP TABLE test.kafka SYNC") + instance.query("DROP TABLE test.consumer SYNC") + instance.query("DROP TABLE test.view SYNC") -def test_kafka_materialized_view_with_subquery(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_new_create_table_query, generate_old_create_table_query], +) +def test_kafka_materialized_view_with_subquery(kafka_cluster, create_query_generator): + topic_name = "mysq" + logging.debug(f"Using topic {topic_name}") + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( - """ + f""" + DROP TABLE IF EXISTS test.kafka; DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'mvsq', - kafka_group_name = 'mvsq', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; + + {create_query}; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -1817,37 +2020,46 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "mvsq", messages) + kafka_produce(kafka_cluster, topic_name, messages) - while True: - result = instance.query("SELECT * FROM test.view") - if kafka_check_result(result): - break + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + result = instance.query_with_retry( + "SELECT * FROM test.view", + check_callback=kafka_check_result, + retry_count=40, + sleep_time=0.75, + ) - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ + ) + + kafka_check_result(result, True) + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_many_materialized_views(kafka_cluster, create_query_generator): + topic_name = f"mmv-{get_topic_postfix(create_query_generator)}" + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}-group", ) - kafka_check_result(result, True) - - -def test_kafka_many_materialized_views(kafka_cluster): instance.query( - """ + f""" DROP TABLE IF EXISTS test.view1; DROP TABLE IF EXISTS test.view2; DROP TABLE IF EXISTS test.consumer1; DROP TABLE IF EXISTS test.consumer2; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'mmv', - kafka_group_name = 'mmv', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; + {create_query}; CREATE TABLE test.view1 (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -1864,237 +2076,258 @@ def test_kafka_many_materialized_views(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "mmv", messages) + kafka_produce(kafka_cluster, topic_name, messages) - while True: - result1 = instance.query("SELECT * FROM test.view1") - result2 = instance.query("SELECT * FROM test.view2") - if kafka_check_result(result1) and kafka_check_result(result2): - break + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + result1 = instance.query_with_retry( + "SELECT * FROM test.view1", check_callback=kafka_check_result + ) + result2 = instance.query_with_retry( + "SELECT * FROM test.view2", check_callback=kafka_check_result + ) - instance.query( + instance.query( + """ + DROP TABLE test.consumer1; + DROP TABLE test.consumer2; + DROP TABLE test.view1; + DROP TABLE test.view2; """ - DROP TABLE test.consumer1; - DROP TABLE test.consumer2; - DROP TABLE test.view1; - DROP TABLE test.view2; - """ - ) + ) - kafka_check_result(result1, True) - kafka_check_result(result2, True) + kafka_check_result(result1, True) + kafka_check_result(result2, True) -def test_kafka_flush_on_big_message(kafka_cluster): - # Create batchs of messages of size ~100Kb +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_flush_on_big_message(kafka_cluster, create_query_generator): + # Create batches of messages of size ~100Kb kafka_messages = 1000 batch_messages = 1000 + topic_name = "flush" + get_topic_postfix(create_query_generator) messages = [ json.dumps({"key": i, "value": "x" * 100}) * batch_messages for i in range(kafka_messages) ] - kafka_produce(kafka_cluster, "flush", messages) + kafka_produce(kafka_cluster, topic_name, messages) - instance.query( + admin_client = get_admin_client(kafka_cluster) + + with existing_kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value String", + topic_list=topic_name, + consumer_group=topic_name, + settings={"kafka_max_block_size": 10}, + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'flush', - kafka_group_name = 'flush', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 10; - CREATE TABLE test.view (key UInt64, value String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) + ) - client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - received = False - while not received: - try: - offsets = client.list_consumer_group_offsets("flush") - for topic, offset in list(offsets.items()): - if topic.topic == "flush" and offset.offset == kafka_messages: - received = True - break - except kafka.errors.GroupCoordinatorNotAvailableError: - continue + received = False + while not received: + try: + offsets = admin_client.list_consumer_group_offsets(topic_name) + for topic, offset in list(offsets.items()): + if topic.topic == topic_name and offset.offset == kafka_messages: + received = True + break + except kafka.errors.GroupCoordinatorNotAvailableError: + continue - while True: - result = instance.query("SELECT count() FROM test.view") - if int(result) == kafka_messages * batch_messages: - break + while True: + result = instance.query("SELECT count() FROM test.view") + if int(result) == kafka_messages * batch_messages: + break - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) + ) - assert ( - int(result) == kafka_messages * batch_messages - ), "ClickHouse lost some messages: {}".format(result) + assert ( + int(result) == kafka_messages * batch_messages + ), "ClickHouse lost some messages: {}".format(result) def test_kafka_virtual_columns(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "virt1", config=topic_config) - - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'virt1', - kafka_group_name = 'virt1', - kafka_commit_on_select = 1, - kafka_format = 'JSONEachRow'; - """ - ) - - messages = "" - for i in range(25): - messages += json.dumps({"key": i, "value": i}) + "\n" - kafka_produce(kafka_cluster, "virt1", [messages], 0) - - messages = "" - for i in range(25, 50): - messages += json.dumps({"key": i, "value": i}) + "\n" - kafka_produce(kafka_cluster, "virt1", [messages], 0) - - result = "" - while True: - result += instance.query( - """SELECT _key, key, _topic, value, _offset, _partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) AS _timestamp FROM test.kafka""", - ignore_error=True, + with kafka_topic(get_admin_client(kafka_cluster), "virt1", config=topic_config): + instance.query( + """ + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'virt1', + kafka_group_name = 'virt1', + kafka_commit_on_select = 1, + kafka_format = 'JSONEachRow'; + """ ) - if kafka_check_result(result, False, "test_kafka_virtual1.reference"): - break - kafka_check_result(result, True, "test_kafka_virtual1.reference") + messages = "" + for i in range(25): + messages += json.dumps({"key": i, "value": i}) + "\n" + kafka_produce(kafka_cluster, "virt1", [messages], 0) + + messages = "" + for i in range(25, 50): + messages += json.dumps({"key": i, "value": i}) + "\n" + kafka_produce(kafka_cluster, "virt1", [messages], 0) + + result = "" + while True: + result += instance.query( + """SELECT _key, key, _topic, value, _offset, _partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) AS _timestamp FROM test.kafka""", + ignore_error=True, + ) + if kafka_check_result(result, False, "test_kafka_virtual1.reference"): + break + + kafka_check_result(result, True, "test_kafka_virtual1.reference") -def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_virtual_columns_with_materialized_view( + kafka_cluster, create_query_generator +): topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "virt2", config=topic_config) - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'virt2', - kafka_group_name = 'virt2', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64, kafka_key String, topic String, offset UInt64, partition UInt64, timestamp Nullable(DateTime('UTC'))) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _key as kafka_key, _topic as topic, _offset as offset, _partition as partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) as timestamp FROM test.kafka; - """ + # the topic name is hardcoded in reference, it doesn't worth to create two reference files to have separate topics, + # as the context manager will always clean up the topic + topic_name = "virt2" + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}-group", ) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "virt2", messages, 0) - - sql = "SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key, key" - result = instance.query(sql) - iterations = 0 - while ( - not kafka_check_result(result, False, "test_kafka_virtual2.reference") - and iterations < 10 - ): - time.sleep(3) - iterations += 1 - result = instance.query(sql) - - kafka_check_result(result, True, "test_kafka_virtual2.reference") - - instance.query( + with kafka_topic(get_admin_client(kafka_cluster), topic_name, config=topic_config): + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64, kafka_key String, topic String, offset UInt64, partition UInt64, timestamp Nullable(DateTime('UTC'))) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT *, _key as kafka_key, _topic as topic, _offset as offset, _partition as partition, _timestamp = 0 ? '0000-00-00 00:00:00' : toString(_timestamp) as timestamp FROM test.kafka; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) + ) + messages = [] + for i in range(50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, topic_name, messages, 0) -def test_kafka_insert(kafka_cluster): - instance.query( + def check_callback(result): + return kafka_check_result(result, False, "test_kafka_virtual2.reference") + + result = instance.query_with_retry( + "SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view ORDER BY kafka_key, key", + check_callback=check_callback, + ) + + kafka_check_result(result, True, "test_kafka_virtual2.reference") + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert1', - kafka_group_name = 'insert1', - kafka_format = 'TSV', - kafka_commit_on_select = 1, - kafka_row_delimiter = '\\n'; - """ - ) + ) - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) +def insert_with_retry(instance, values, table_name="kafka", max_try_count=5): + try_count = 0 while True: + logging.debug(f"Inserting, try_count is {try_count}") try: - instance.query("INSERT INTO test.kafka VALUES {}".format(values)) + try_count += 1 + instance.query(f"INSERT INTO test.{table_name} VALUES {values}") break except QueryRuntimeException as e: - if "Local: Timed out." in str(e): + if "Local: Timed out." in str(e) and try_count < max_try_count: continue else: raise - messages = [] - while True: - messages.extend(kafka_consume(kafka_cluster, "insert1")) - if len(messages) == 50: - break - result = "\n".join(messages) - kafka_check_result(result, True) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_insert(kafka_cluster, create_query_generator): + topic_name = "insert1" + get_topic_postfix(create_query_generator) - -def test_kafka_produce_consume(kafka_cluster): instance.query( - """ + create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV", + ) + ) + + message_count = 50 + values = [] + for i in range(message_count): + values.append("({i}, {i})".format(i=i)) + values = ",".join(values) + + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + insert_with_retry(instance, values) + + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) + result = "\n".join(messages) + kafka_check_result(result, True) + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_produce_consume(kafka_cluster, create_query_generator): + topic_name = "insert2" + get_topic_postfix(create_query_generator) + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV", + ) + instance.query( + f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert2', - kafka_group_name = 'insert2', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; + {create_query}; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; @@ -2111,15 +2344,7 @@ def test_kafka_produce_consume(kafka_cluster): values.append("({i}, {i})".format(i=i)) values = ",".join(values) - while True: - try: - instance.query("INSERT INTO test.kafka VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise + insert_with_retry(instance, values) threads = [] threads_num = 16 @@ -2129,40 +2354,48 @@ def test_kafka_produce_consume(kafka_cluster): time.sleep(random.uniform(0, 1)) thread.start() - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == messages_num * threads_num: - break + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + expected_row_count = messages_num * threads_num + result = instance.query_with_retry( + "SELECT count() FROM test.view", + sleep_time=1, + retry_count=20, + check_callback=lambda result: int(result) == expected_row_count, + ) - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ + ) + + for thread in threads: + thread.join() + + assert ( + int(result) == expected_row_count + ), "ClickHouse lost some messages: {}".format(result) + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): + topic_name = "block" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={"kafka_max_block_size": 100}, ) - - for thread in threads: - thread.join() - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) - - -def test_kafka_commit_on_block_write(kafka_cluster): instance.query( - """ + f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'block', - kafka_group_name = 'block', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_row_delimiter = '\\n'; + {create_query}; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -2173,45 +2406,39 @@ def test_kafka_commit_on_block_write(kafka_cluster): cancel = threading.Event() + # We need to pass i as a reference. Simple integers are passed by value. + # Making an array is probably the easiest way to "force pass by reference". i = [0] - def produce(): + def produce(i): while not cancel.is_set(): messages = [] for _ in range(101): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 - kafka_produce(kafka_cluster, "block", messages) + kafka_produce(kafka_cluster, topic_name, messages) - kafka_thread = threading.Thread(target=produce) + kafka_thread = threading.Thread(target=produce, args=[i]) kafka_thread.start() - while int(instance.query("SELECT count() FROM test.view")) == 0: - time.sleep(1) + instance.query_with_retry( + "SELECT count() FROM test.view", + sleep_time=1, + check_callback=lambda res: int(res) >= 100, + ) cancel.set() - instance.query( - """ - DROP TABLE test.kafka; - """ - ) + instance.query("DROP TABLE test.kafka SYNC") - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'block', - kafka_group_name = 'block', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_row_delimiter = '\\n'; - """ - ) + instance.query(create_query) + kafka_thread.join() - while int(instance.query("SELECT uniqExact(key) FROM test.view")) < i[0]: - time.sleep(1) + instance.query_with_retry( + "SELECT uniqExact(key) FROM test.view", + sleep_time=1, + check_callback=lambda res: int(res) >= i[0], + ) result = int(instance.query("SELECT count() == uniqExact(key) FROM test.view")) @@ -2227,163 +2454,183 @@ def test_kafka_commit_on_block_write(kafka_cluster): assert result == 1, "Messages from kafka get duplicated!" -def test_kafka_virtual_columns2(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"), + ( + generate_new_create_table_query, + r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+", + ), + ], +) +def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line): + admin_client = get_admin_client(kafka_cluster) topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "virt2_0", num_partitions=2, config=topic_config) - kafka_create_topic(admin_client, "virt2_1", num_partitions=2, config=topic_config) + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) + topic_name_0 = "virt2_0" + topic_name_1 = "virt2_1" + consumer_group = "virt2" + get_topic_postfix(create_query_generator) + with kafka_topic(admin_client, topic_name_0, num_partitions=2, config=topic_config): + with kafka_topic( + admin_client, topic_name_1, num_partitions=2, config=topic_config + ): + create_query = create_query_generator( + "kafka", + "value UInt64", + topic_list=f"{topic_name_0},{topic_name_1}", + consumer_group=consumer_group, + settings={ + "kafka_num_consumers": 2, + "kafka_thread_per_consumer": thread_per_consumer, + }, + ) - instance.query( + instance.query( + f""" + {create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; + """ + ) + + producer = KafkaProducer( + bootstrap_servers="localhost:{}".format(cluster.kafka_port), + value_serializer=producer_serializer, + key_serializer=producer_serializer, + ) + + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 1}), + partition=0, + key="k1", + timestamp_ms=1577836801001, + headers=[("content-encoding", b"base64")], + ) + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 2}), + partition=0, + key="k2", + timestamp_ms=1577836802002, + headers=[ + ("empty_value", b""), + ("", b"empty name"), + ("", b""), + ("repetition", b"1"), + ("repetition", b"2"), + ], + ) + producer.flush() + + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 3}), + partition=1, + key="k3", + timestamp_ms=1577836803003, + headers=[("b", b"b"), ("a", b"a")], + ) + producer.send( + topic=topic_name_0, + value=json.dumps({"value": 4}), + partition=1, + key="k4", + timestamp_ms=1577836804004, + headers=[("a", b"a"), ("b", b"b")], + ) + producer.flush() + + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 5}), + partition=0, + key="k5", + timestamp_ms=1577836805005, + ) + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 6}), + partition=0, + key="k6", + timestamp_ms=1577836806006, + ) + producer.flush() + + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 7}), + partition=1, + key="k7", + timestamp_ms=1577836807007, + ) + producer.send( + topic=topic_name_1, + value=json.dumps({"value": 8}), + partition=1, + key="k8", + timestamp_ms=1577836808008, + ) + producer.flush() + + instance.wait_for_log_line(log_line, repetitions=4, look_behind_lines=6000) + + members = describe_consumer_group(kafka_cluster, consumer_group) + # pprint.pprint(members) + # members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' + # members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' + + result = instance.query( + "SELECT * FROM test.view ORDER BY value", ignore_error=True + ) + + expected = f"""\ + 1 k1 {topic_name_0} 0 0 1577836801 1577836801001 ['content-encoding'] ['base64'] + 2 k2 {topic_name_0} 0 1 1577836802 1577836802002 ['empty_value','','','repetition','repetition'] ['','empty name','','1','2'] + 3 k3 {topic_name_0} 1 0 1577836803 1577836803003 ['b','a'] ['b','a'] + 4 k4 {topic_name_0} 1 1 1577836804 1577836804004 ['a','b'] ['a','b'] + 5 k5 {topic_name_1} 0 0 1577836805 1577836805005 [] [] + 6 k6 {topic_name_1} 0 1 1577836806 1577836806006 [] [] + 7 k7 {topic_name_1} 1 0 1577836807 1577836807007 [] [] + 8 k8 {topic_name_1} 1 1 1577836808 1577836808008 [] [] """ - CREATE TABLE test.kafka (value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'virt2_0,virt2_1', - kafka_group_name = 'virt2', - kafka_num_consumers = 2, - kafka_format = 'JSONEachRow'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; - """ - ) + assert TSV(result) == TSV(expected) - producer = KafkaProducer( - bootstrap_servers="localhost:{}".format(cluster.kafka_port), - value_serializer=producer_serializer, - key_serializer=producer_serializer, - ) + instance.query( + """ + DROP TABLE test.kafka; + DROP TABLE test.view; + """ + ) + instance.rotate_logs() - producer.send( - topic="virt2_0", - value=json.dumps({"value": 1}), - partition=0, - key="k1", - timestamp_ms=1577836801001, - headers=[("content-encoding", b"base64")], - ) - producer.send( - topic="virt2_0", - value=json.dumps({"value": 2}), - partition=0, - key="k2", - timestamp_ms=1577836802002, - headers=[ - ("empty_value", b""), - ("", b"empty name"), - ("", b""), - ("repetition", b"1"), - ("repetition", b"2"), - ], - ) - producer.flush() - producer.send( - topic="virt2_0", - value=json.dumps({"value": 3}), - partition=1, - key="k3", - timestamp_ms=1577836803003, - headers=[("b", b"b"), ("a", b"a")], - ) - producer.send( - topic="virt2_0", - value=json.dumps({"value": 4}), - partition=1, - key="k4", - timestamp_ms=1577836804004, - headers=[("a", b"a"), ("b", b"b")], - ) - producer.flush() - - producer.send( - topic="virt2_1", - value=json.dumps({"value": 5}), - partition=0, - key="k5", - timestamp_ms=1577836805005, - ) - producer.send( - topic="virt2_1", - value=json.dumps({"value": 6}), - partition=0, - key="k6", - timestamp_ms=1577836806006, - ) - producer.flush() - - producer.send( - topic="virt2_1", - value=json.dumps({"value": 7}), - partition=1, - key="k7", - timestamp_ms=1577836807007, - ) - producer.send( - topic="virt2_1", - value=json.dumps({"value": 8}), - partition=1, - key="k8", - timestamp_ms=1577836808008, - ) - producer.flush() - - instance.wait_for_log_line( - "kafka.*Committed offset 2.*virt2_[01]", repetitions=4, look_behind_lines=6000 - ) - - members = describe_consumer_group(kafka_cluster, "virt2") - # pprint.pprint(members) - # members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' - # members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' - - result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - - expected = """\ -1 k1 virt2_0 0 0 1577836801 1577836801001 ['content-encoding'] ['base64'] -2 k2 virt2_0 0 1 1577836802 1577836802002 ['empty_value','','','repetition','repetition'] ['','empty name','','1','2'] -3 k3 virt2_0 1 0 1577836803 1577836803003 ['b','a'] ['b','a'] -4 k4 virt2_0 1 1 1577836804 1577836804004 ['a','b'] ['a','b'] -5 k5 virt2_1 0 0 1577836805 1577836805005 [] [] -6 k6 virt2_1 0 1 1577836806 1577836806006 [] [] -7 k7 virt2_1 1 0 1577836807 1577836807007 [] [] -8 k8 virt2_1 1 1 1577836808 1577836808008 [] [] -""" - - assert TSV(result) == TSV(expected) - - instance.query( - """ - DROP TABLE test.kafka; - DROP TABLE test.view; - """ - ) - kafka_delete_topic(admin_client, "virt2_0") - kafka_delete_topic(admin_client, "virt2_1") +@pytest.mark.parametrize( + "create_query_generator, do_direct_read", + [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], +) +def test_kafka_producer_consumer_separate_settings( + kafka_cluster, create_query_generator, do_direct_read +): instance.rotate_logs() - - -def test_kafka_producer_consumer_separate_settings(kafka_cluster): instance.query( - """ - DROP TABLE IF EXISTS test.test_kafka; - CREATE TABLE test.test_kafka (key UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'separate_settings', - kafka_group_name = 'test', - kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\\n'; - """ + create_query_generator( + "test_kafka", + "key UInt64", + topic_list="separate_settings", + consumer_group="test", + ) ) - instance.query("SELECT * FROM test.test_kafka") + if do_direct_read: + instance.query("SELECT * FROM test.test_kafka") instance.query("INSERT INTO test.test_kafka VALUES (1)") assert instance.contains_in_log("Kafka producer created") @@ -2398,11 +2645,11 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): # and producer configurations assert "heartbeat.interval.ms" in warn - kafka_consumer_applyed_properties = instance.grep_in_log("Consumer set property") - kafka_producer_applyed_properties = instance.grep_in_log("Producer set property") + kafka_consumer_applied_properties = instance.grep_in_log("Consumer set property") + kafka_producer_applied_properties = instance.grep_in_log("Producer set property") - assert kafka_consumer_applyed_properties is not None - assert kafka_producer_applyed_properties is not None + assert kafka_consumer_applied_properties is not None + assert kafka_producer_applied_properties is not None # global settings should be applied for consumer and producer global_settings = { @@ -2412,118 +2659,128 @@ def test_kafka_producer_consumer_separate_settings(kafka_cluster): for name, value in global_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log in kafka_consumer_applyed_properties - assert property_in_log in kafka_producer_applyed_properties + assert property_in_log in kafka_consumer_applied_properties + assert property_in_log in kafka_producer_applied_properties settings_topic__separate_settings__consumer = {"session.timeout.ms": "6001"} for name, value in settings_topic__separate_settings__consumer.items(): property_in_log = f"{name}:{value}" - assert property_in_log in kafka_consumer_applyed_properties - assert property_in_log not in kafka_producer_applyed_properties + assert property_in_log in kafka_consumer_applied_properties + assert property_in_log not in kafka_producer_applied_properties producer_settings = {"transaction.timeout.ms": "60001"} for name, value in producer_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log not in kafka_consumer_applyed_properties - assert property_in_log in kafka_producer_applyed_properties + assert property_in_log not in kafka_consumer_applied_properties + assert property_in_log in kafka_producer_applied_properties # Should be ignored, because it is inside producer tag producer_legacy_syntax__topic_separate_settings = {"message.timeout.ms": "300001"} for name, value in producer_legacy_syntax__topic_separate_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log not in kafka_consumer_applyed_properties - assert property_in_log not in kafka_producer_applyed_properties + assert property_in_log not in kafka_consumer_applied_properties + assert property_in_log not in kafka_producer_applied_properties # Old syntax, applied on consumer and producer legacy_syntax__topic_separated_settings = {"heartbeat.interval.ms": "302"} for name, value in legacy_syntax__topic_separated_settings.items(): property_in_log = f"{name}:{value}" - assert property_in_log in kafka_consumer_applyed_properties - assert property_in_log in kafka_producer_applyed_properties + assert property_in_log in kafka_consumer_applied_properties + assert property_in_log in kafka_producer_applied_properties -def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_new_create_table_query, "Saved offset 5"), + (generate_old_create_table_query, "Committed offset 5"), + ], +) +def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_line): topic_name = "insert3" topic_config = { # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, topic_name, config=topic_config) - instance.query( + with kafka_topic(get_admin_client(kafka_cluster), topic_name, config=topic_config): + writer_create_query = create_query_generator( + "kafka_writer", + "key UInt64, value UInt64, _key String, _timestamp DateTime('UTC')", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV", + ) + reader_create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64, inserted_key String, inserted_timestamp DateTime('UTC')", + topic_list=topic_name, + consumer_group=topic_name, + format="TSV", + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {writer_create_query}; + {reader_create_query}; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka_writer (key UInt64, value UInt64, _key String, _timestamp DateTime('UTC')) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert3', - kafka_group_name = 'insert3', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; + ) - CREATE TABLE test.kafka (key UInt64, value UInt64, inserted_key String, inserted_timestamp DateTime('UTC')) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'insert3', - kafka_group_name = 'insert3', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( + 1, 1, "k1", 1577836801 + ) + ) + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( + 2, 2, "k2", 1577836802 + ) + ) + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({})),({},{},'{}',toDateTime({}))".format( + 3, 3, "k3", 1577836803, 4, 4, "k4", 1577836804 + ) + ) + instance.query( + "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( + 5, 5, "k5", 1577836805 + ) + ) - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; + # instance.wait_for_log_line(log_line) + + expected = """\ + 1 1 k1 1577836801 k1 insert3 0 0 1577836801 + 2 2 k2 1577836802 k2 insert3 0 1 1577836802 + 3 3 k3 1577836803 k3 insert3 0 2 1577836803 + 4 4 k4 1577836804 k4 insert3 0 3 1577836804 + 5 5 k5 1577836805 k5 insert3 0 4 1577836805 """ - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( - 1, 1, "k1", 1577836801 + result = instance.query_with_retry( + "SELECT * FROM test.view ORDER BY value", + ignore_error=True, + retry_count=5, + sleep_time=1, + check_callback=lambda res: TSV(res) == TSV(expected), ) - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( - 2, 2, "k2", 1577836802 - ) - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({})),({},{},'{}',toDateTime({}))".format( - 3, 3, "k3", 1577836803, 4, 4, "k4", 1577836804 - ) - ) - instance.query( - "INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format( - 5, 5, "k5", 1577836805 - ) - ) - instance.wait_for_log_line("Committed offset 5") - - result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - - # logging.debug(result) - - expected = """\ -1 1 k1 1577836801 k1 insert3 0 0 1577836801 -2 2 k2 1577836802 k2 insert3 0 1 1577836802 -3 3 k3 1577836803 k3 insert3 0 2 1577836803 -4 4 k4 1577836804 k4 insert3 0 3 1577836804 -5 5 k5 1577836805 k5 insert3 0 4 1577836805 -""" - - assert TSV(result) == TSV(expected) - - kafka_delete_topic(admin_client, topic_name) + assert TSV(result) == TSV(expected) -def test_kafka_insert_avro(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_insert_avro(kafka_cluster, create_query_generator): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2531,296 +2788,327 @@ def test_kafka_insert_avro(kafka_cluster): # default retention, since predefined timestamp_ms is used. "retention.ms": "-1", } - kafka_create_topic(admin_client, "avro1", config=topic_config) - - instance.query( - """ - DROP TABLE IF EXISTS test.kafka; - CREATE TABLE test.kafka (key UInt64, value UInt64, _timestamp DateTime('UTC')) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'avro1', - kafka_group_name = 'avro1', - kafka_commit_on_select = 1, - kafka_format = 'Avro'; - """ - ) - - instance.query( - "INSERT INTO test.kafka select number*10 as key, number*100 as value, 1636505534 as _timestamp from numbers(4) SETTINGS output_format_avro_rows_in_file = 2, output_format_avro_codec = 'deflate'" - ) - - messages = [] - while True: - messages.extend( - kafka_consume( - kafka_cluster, "avro1", needDecode=False, timestamp=1636505534 - ) + topic_name = "avro1" + get_topic_postfix(create_query_generator) + with kafka_topic(admin_client, topic_name, config=topic_config): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64, _timestamp DateTime('UTC')", + topic_list=topic_name, + consumer_group=topic_name, + format="Avro", + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + {create_query} + """ ) - if len(messages) == 2: - break - result = "" - for a_message in messages: - result += decode_avro(a_message) + "\n" + instance.query( + "INSERT INTO test.kafka select number*10 as key, number*100 as value, 1636505534 as _timestamp from numbers(4) SETTINGS output_format_avro_rows_in_file = 2, output_format_avro_codec = 'deflate'" + ) - expected_result = """{'key': 0, 'value': 0, '_timestamp': 1636505534} + message_count = 2 + messages = kafka_consume_with_retry( + kafka_cluster, + topic_name, + message_count, + need_decode=False, + timestamp=1636505534, + ) + + result = "" + for a_message in messages: + result += decode_avro(a_message) + "\n" + + expected_result = """{'key': 0, 'value': 0, '_timestamp': 1636505534} {'key': 10, 'value': 100, '_timestamp': 1636505534} {'key': 20, 'value': 200, '_timestamp': 1636505534} {'key': 30, 'value': 300, '_timestamp': 1636505534} """ - assert result == expected_result + assert result == expected_result -def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): + topic_name = "insert_avro" + get_topic_postfix(create_query_generator) + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + num_rows = 75 - topic_name = "insert_avro" - kafka_create_topic(admin_client, topic_name) - - num_rows = 75 - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; - DROP TABLE IF EXISTS test.kafka_writer; - - CREATE TABLE test.kafka_writer (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'avro', - kafka_group_name = 'avro', - kafka_format = 'Avro'; - - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'avro', - kafka_group_name = 'avro', - kafka_format = 'Avro'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; - """ - ) - - instance.query( - "INSERT INTO test.kafka_writer select number*10 as key, number*100 as value from numbers({num_rows}) SETTINGS output_format_avro_rows_in_file = 7".format( - num_rows=num_rows + writer_create_query = create_query_generator( + "kafka_writer", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="Avro", ) - ) - instance.wait_for_log_line( - "Committed offset {offset}".format(offset=math.ceil(num_rows / 7)) - ) + reader_create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="Avro", + ) - expected_num_rows = instance.query( - "SELECT COUNT(1) FROM test.view", ignore_error=True - ) - assert int(expected_num_rows) == num_rows - - expected_max_key = instance.query( - "SELECT max(key) FROM test.view", ignore_error=True - ) - assert int(expected_max_key) == (num_rows - 1) * 10 - - kafka_delete_topic(admin_client, topic_name) - - -def test_kafka_flush_by_time(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - topic_name = "flush_by_time" - kafka_create_topic(admin_client, topic_name) - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'flush_by_time', - kafka_group_name = 'flush_by_time', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_row_delimiter = '\\n'; - - SELECT * FROM test.kafka; - - CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) - ENGINE = MergeTree() - ORDER BY key; - """ - ) - - cancel = threading.Event() - - def produce(): - while not cancel.is_set(): - messages = [] - messages.append(json.dumps({"key": 0, "value": 0})) - kafka_produce(kafka_cluster, "flush_by_time", messages) - time.sleep(0.8) - - kafka_thread = threading.Thread(target=produce) - kafka_thread.start() - - instance.query( - """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) - - time.sleep(18) - - result = instance.query("SELECT uniqExact(ts) = 2, count() >= 15 FROM test.view") - - cancel.set() - kafka_thread.join() - - # kafka_cluster.open_bash_shell('instance') - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - assert TSV(result) == TSV("1 1") - kafka_delete_topic(admin_client, topic_name) - - -def test_kafka_flush_by_block_size(kafka_cluster): - cancel = threading.Event() - - def produce(): - while not cancel.is_set(): - messages = [] - messages.append(json.dumps({"key": 0, "value": 0})) - kafka_produce(kafka_cluster, "flush_by_block_size", messages) - - kafka_thread = threading.Thread(target=produce) - kafka_thread.start() - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'flush_by_block_size', - kafka_group_name = 'flush_by_block_size', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_poll_max_batch_size = 1, - kafka_flush_interval_ms = 120000, /* should not flush by time during test */ - kafka_row_delimiter = '\\n'; - - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ - ) - - # Wait for Kafka engine to consume this data - while 1 != int( instance.query( - "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.kafka_writer; + + {writer_create_query}; + {reader_create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + """ ) - ): - time.sleep(0.5) - cancel.set() - kafka_thread.join() + instance.query( + "INSERT INTO test.kafka_writer select number*10 as key, number*100 as value from numbers({num_rows}) SETTINGS output_format_avro_rows_in_file = 7".format( + num_rows=num_rows + ) + ) - # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). - result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") - # logging.debug(result) + instance.wait_for_log_line( + "Committed offset {offset}".format(offset=math.ceil(num_rows / 7)) + ) - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ + expected_num_rows = instance.query( + "SELECT COUNT(1) FROM test.view", ignore_error=True + ) + assert int(expected_num_rows) == num_rows + + expected_max_key = instance.query( + "SELECT max(key) FROM test.view", ignore_error=True + ) + assert int(expected_max_key) == (num_rows - 1) * 10 + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_flush_by_time(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) + topic_name = "flush_by_time" + get_topic_postfix(create_query_generator) - # 100 = first poll should return 100 messages (and rows) - # not waiting for stream_flush_interval_ms - assert ( - int(result) == 100 - ), "Messages from kafka should be flushed when block of size kafka_max_block_size is formed!" + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 100, + }, + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + {create_query}; + + CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) + ENGINE = MergeTree() + ORDER BY key; + """ + ) + + cancel = threading.Event() + + def produce(): + while not cancel.is_set(): + messages = [json.dumps({"key": 0, "value": 0})] + kafka_produce(kafka_cluster, topic_name, messages) + time.sleep(0.8) + + kafka_thread = threading.Thread(target=produce) + kafka_thread.start() + + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ + ) + + # By default the flush timeout should be 7.5 seconds => 18 seconds should be enough for 2 flushes, but not for 3 + time.sleep(18) + + result = instance.query("SELECT uniqExact(ts), count() >= 15 FROM test.view") + + cancel.set() + kafka_thread.join() + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) + + assert TSV(result) == TSV("2 1") -def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_flush_by_block_size(kafka_cluster, create_query_generator): + topic_name = "flush_by_block_size" + get_topic_postfix(create_query_generator) + + cancel = threading.Event() + + def produce(): + while not cancel.is_set(): + messages = [] + messages.append(json.dumps({"key": 0, "value": 0})) + kafka_produce(kafka_cluster, topic_name, messages) + + kafka_thread = threading.Thread(target=produce) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + kafka_thread.start() + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 100, + "kafka_poll_max_batch_size": 1, + "kafka_flush_interval_ms": 120000, + }, + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + {create_query}; + + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + """ + ) + + # Wait for Kafka engine to consume this data + while 1 != int( + instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + ) + ): + time.sleep(0.5) + + cancel.set() + kafka_thread.join() + + # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). + result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") + # logging.debug(result) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) + + # 100 = first poll should return 100 messages (and rows) + # not waiting for stream_flush_interval_ms + assert ( + int(result) == 100 + ), "Messages from kafka should be flushed when block of size kafka_max_block_size is formed!" + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_lot_of_partitions_partial_commit_of_bulk( + kafka_cluster, create_query_generator +): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "topic_with_multiple_partitions2" - kafka_create_topic(admin_client, topic_name, num_partitions=10) - - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'topic_with_multiple_partitions2', - kafka_group_name = 'topic_with_multiple_partitions2', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 211, - kafka_flush_interval_ms = 500; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka; - """ + topic_name = "topic_with_multiple_partitions2" + get_topic_postfix( + create_query_generator ) - - messages = [] - count = 0 - for dummy_msg in range(1000): - rows = [] - for dummy_row in range(random.randrange(3, 10)): - count = count + 1 - rows.append(json.dumps({"key": count, "value": count})) - messages.append("\n".join(rows)) - kafka_produce(kafka_cluster, "topic_with_multiple_partitions2", messages) - - instance.wait_for_log_line("kafka.*Stalled", repetitions=5) - - result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") - logging.debug(result) - assert TSV(result) == TSV("{0}\t{0}\t{0}".format(count)) - - instance.query( + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 211, + "kafka_flush_interval_ms": 500, + }, + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + {create_query}; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - kafka_delete_topic(admin_client, topic_name) + ) + + messages = [] + count = 0 + for dummy_msg in range(1000): + rows = [] + for dummy_row in range(random.randrange(3, 10)): + count = count + 1 + rows.append(json.dumps({"key": count, "value": count})) + messages.append("\n".join(rows)) + kafka_produce(kafka_cluster, topic_name, messages) + + instance.wait_for_log_line("kafka.*Stalled", repetitions=5) + + result = instance.query( + "SELECT count(), uniqExact(key), max(key) FROM test.view" + ) + logging.debug(result) + assert TSV(result) == TSV("{0}\t{0}\t{0}".format(count)) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + """ + ) -def test_kafka_rebalance(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator, log_line", + [ + (generate_old_create_table_query, "{}.*Polled offset [0-9]+"), + (generate_new_create_table_query, "{}.*Saved offset"), + ], +) +def test_kafka_rebalance(kafka_cluster, create_query_generator, log_line): NUMBER_OF_CONSURRENT_CONSUMERS = 11 instance.query( @@ -2841,212 +3129,232 @@ def test_kafka_rebalance(kafka_cluster): """ ) - # kafka_cluster.open_bash_shell('instance') - - # time.sleep(2) - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_name = "topic_with_multiple_partitions" - kafka_create_topic(admin_client, topic_name, num_partitions=11) + topic_name = "topic_with_multiple_partitions" + get_topic_postfix( + create_query_generator + ) + table_name_prefix = "kafka_consumer" + keeper_path = f"/clickhouse/{{database}}/{table_name_prefix}" + with kafka_topic(admin_client, topic_name, num_partitions=11): + cancel = threading.Event() - cancel = threading.Event() + msg_index = [0] - msg_index = [0] + def produce(): + while not cancel.is_set(): + messages = [] + for _ in range(59): + messages.append( + json.dumps({"key": msg_index[0], "value": msg_index[0]}) + ) + msg_index[0] += 1 + kafka_produce(kafka_cluster, topic_name, messages) - def produce(): - while not cancel.is_set(): - messages = [] - for _ in range(59): - messages.append( - json.dumps({"key": msg_index[0], "value": msg_index[0]}) - ) - msg_index[0] += 1 - kafka_produce(kafka_cluster, "topic_with_multiple_partitions", messages) + kafka_thread = threading.Thread(target=produce) + kafka_thread.start() - kafka_thread = threading.Thread(target=produce) - kafka_thread.start() + for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): + table_name = f"{table_name_prefix}{consumer_index}" + replica_name = f"r{consumer_index}" + logging.debug(f"Setting up {consumer_index}") - for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - table_name = "kafka_consumer{}".format(consumer_index) - logging.debug(("Setting up {}".format(table_name))) - - instance.query( - """ - DROP TABLE IF EXISTS test.{0}; - DROP TABLE IF EXISTS test.{0}_mv; - CREATE TABLE test.{0} (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'topic_with_multiple_partitions', - kafka_group_name = 'rebalance_test_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 33, - kafka_flush_interval_ms = 500; - CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS - SELECT - key, - value, - _topic, - _key, - _offset, - _partition, - _timestamp, - '{0}' as _consumed_by - FROM test.{0}; - """.format( - table_name + create_query = create_query_generator( + table_name, + "key UInt64, value UInt64", + topic_list=topic_name, + keeper_path=keeper_path, + replica_name=replica_name, + settings={ + "kafka_max_block_size": 33, + "kafka_flush_interval_ms": 500, + }, ) - ) + instance.query( + f""" + DROP TABLE IF EXISTS test.{table_name}; + DROP TABLE IF EXISTS test.{table_name}_mv; + {create_query}; + CREATE MATERIALIZED VIEW test.{table_name}_mv TO test.destination AS + SELECT + key, + value, + _topic, + _key, + _offset, + _partition, + _timestamp, + '{table_name}' as _consumed_by + FROM test.{table_name}; + """ + ) + # kafka_cluster.open_bash_shell('instance') + # Waiting for test.kafka_consumerX to start consume ... + instance.wait_for_log_line(log_line.format(table_name)) + + cancel.set() + + # I leave last one working by intent (to finish consuming after all rebalances) + for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): + logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) + instance.query( + "DROP TABLE IF EXISTS test.kafka_consumer{} SYNC".format(consumer_index) + ) + + # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) # kafka_cluster.open_bash_shell('instance') - # Waiting for test.kafka_consumerX to start consume ... - instance.wait_for_log_line( - "kafka_consumer{}.*Polled offset [0-9]+".format(consumer_index) - ) - cancel.set() + while 1: + messages_consumed = int( + instance.query("SELECT uniqExact(key) FROM test.destination") + ) + if messages_consumed >= msg_index[0]: + break + time.sleep(1) + logging.debug( + ( + "Waiting for finishing consuming (have {}, should be {})".format( + messages_consumed, msg_index[0] + ) + ) + ) - # I leave last one working by intent (to finish consuming after all rebalances) - for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): - logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) - instance.query( - "DROP TABLE IF EXISTS test.kafka_consumer{} SYNC".format(consumer_index) - ) - - # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) - # kafka_cluster.open_bash_shell('instance') - - while 1: - messages_consumed = int( - instance.query("SELECT uniqExact(key) FROM test.destination") - ) - if messages_consumed >= msg_index[0]: - break - time.sleep(1) logging.debug( ( - "Waiting for finishing consuming (have {}, should be {})".format( - messages_consumed, msg_index[0] + instance.query( + "SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination" ) ) ) - logging.debug( - ( - instance.query( - "SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination" - ) + # Some queries to debug... + # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) + # select number + 1 as key from numbers(4141) x left join test.destination using (key) where test.destination.key = 0; + # SELECT * FROM test.destination WHERE key between 2360 and 2370 order by key; + # select _partition from test.destination group by _partition having count() <> max(_offset) + 1; + # select toUInt64(0) as _partition, number + 1 as _offset from numbers(400) x left join test.destination using (_partition,_offset) where test.destination.key = 0 order by _offset; + # SELECT * FROM test.destination WHERE _partition = 0 and _offset between 220 and 240 order by _offset; + + # CREATE TABLE test.reference (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', + # kafka_topic_list = 'topic_with_multiple_partitions', + # kafka_group_name = 'rebalance_test_group_reference', + # kafka_format = 'JSONEachRow', + # kafka_max_block_size = 100000; + # + # CREATE MATERIALIZED VIEW test.reference_mv Engine=Log AS + # SELECT key, value, _topic,_key,_offset, _partition, _timestamp, 'reference' as _consumed_by + # FROM test.reference; + # + # select * from test.reference_mv left join test.destination using (key,_topic,_offset,_partition) where test.destination._consumed_by = ''; + + result = int( + instance.query("SELECT count() == uniqExact(key) FROM test.destination") ) - ) - # Some queries to debug... - # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) - # select number + 1 as key from numbers(4141) x left join test.destination using (key) where test.destination.key = 0; - # SELECT * FROM test.destination WHERE key between 2360 and 2370 order by key; - # select _partition from test.destination group by _partition having count() <> max(_offset) + 1; - # select toUInt64(0) as _partition, number + 1 as _offset from numbers(400) x left join test.destination using (_partition,_offset) where test.destination.key = 0 order by _offset; - # SELECT * FROM test.destination WHERE _partition = 0 and _offset between 220 and 240 order by _offset; + for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): + logging.debug(("kafka_consumer{}".format(consumer_index))) + table_name = "kafka_consumer{}".format(consumer_index) + instance.query( + """ + DROP TABLE IF EXISTS test.{0}; + DROP TABLE IF EXISTS test.{0}_mv; + """.format( + table_name + ) + ) - # CREATE TABLE test.reference (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - # kafka_topic_list = 'topic_with_multiple_partitions', - # kafka_group_name = 'rebalance_test_group_reference', - # kafka_format = 'JSONEachRow', - # kafka_max_block_size = 100000; - # - # CREATE MATERIALIZED VIEW test.reference_mv Engine=Log AS - # SELECT key, value, _topic,_key,_offset, _partition, _timestamp, 'reference' as _consumed_by - # FROM test.reference; - # - # select * from test.reference_mv left join test.destination using (key,_topic,_offset,_partition) where test.destination._consumed_by = ''; - - result = int( - instance.query("SELECT count() == uniqExact(key) FROM test.destination") - ) - - for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - logging.debug(("kafka_consumer{}".format(consumer_index))) - table_name = "kafka_consumer{}".format(consumer_index) instance.query( """ - DROP TABLE IF EXISTS test.{0}; - DROP TABLE IF EXISTS test.{0}_mv; - """.format( - table_name - ) + DROP TABLE IF EXISTS test.destination; + """ ) - instance.query( + kafka_thread.join() + + assert result == 1, "Messages from kafka get duplicated!" + + +# TODO(antaljanosbenjamin): find another way to make insertion fail +@pytest.mark.parametrize( + "create_query_generator", + [ + generate_old_create_table_query, + # generate_new_create_table_query, + ], +) +def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster, create_query_generator): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + topic_name = "no_holes_when_write_suffix_failed" + get_topic_postfix( + create_query_generator + ) + + with kafka_topic(admin_client, topic_name): + messages = [json.dumps({"key": j + 1, "value": "x" * 300}) for j in range(22)] + kafka_produce(kafka_cluster, topic_name, messages) + + create_query = create_query_generator( + "kafka", + "key UInt64, value String", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 20, + "kafka_flush_interval_ms": 2000, + }, + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view SYNC; + DROP TABLE IF EXISTS test.consumer; + + {create_query}; + + CREATE TABLE test.view (key UInt64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/kafkatest/tables/{topic_name}', 'node1') + ORDER BY key; """ - DROP TABLE IF EXISTS test.destination; - """ - ) + ) - kafka_thread.join() + # init PartitionManager (it starts container) earlier + pm = PartitionManager() - assert result == 1, "Messages from kafka get duplicated!" - kafka_delete_topic(admin_client, topic_name) - - -def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): - messages = [json.dumps({"key": j + 1, "value": "x" * 300}) for j in range(22)] - kafka_produce(kafka_cluster, "no_holes_when_write_suffix_failed", messages) - - instance.query( + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka + WHERE NOT sleepEachRow(0.25); """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; + ) - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'no_holes_when_write_suffix_failed', - kafka_group_name = 'no_holes_when_write_suffix_failed', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 20, - kafka_flush_interval_ms = 2000; + instance.wait_for_log_line("Polled batch of 20 messages") + # the tricky part here is that disconnect should happen after write prefix, but before write suffix + # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" + # while materialized view is working to inject zookeeper failure + pm.drop_instance_zk_connections(instance) + instance.wait_for_log_line( + "Error.*(Connection loss|Coordination::Exception).*while pushing to view" + ) + pm.heal_all() + instance.wait_for_log_line("Committed offset 22") - CREATE TABLE test.view (key UInt64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/kafkatest/tables/no_holes_when_write_suffix_failed', 'node1') - ORDER BY key; - """ - ) + result = instance.query( + "SELECT count(), uniqExact(key), max(key) FROM test.view" + ) + logging.debug(result) - # init PartitionManager (it starts container) earlier - pm = PartitionManager() + # kafka_cluster.open_bash_shell('instance') - instance.query( + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view SYNC; """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka - WHERE NOT sleepEachRow(0.25); - """ - ) + ) - instance.wait_for_log_line("Polled batch of 20 messages") - # the tricky part here is that disconnect should happen after write prefix, but before write suffix - # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" - # while materialized view is working to inject zookeeper failure - pm.drop_instance_zk_connections(instance) - instance.wait_for_log_line( - "Error.*(Connection loss|Coordination::Exception).*while pushing to view" - ) - pm.heal_all() - instance.wait_for_log_line("Committed offset 22") - - result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") - logging.debug(result) - - # kafka_cluster.open_bash_shell('instance') - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - """ - ) - - assert TSV(result) == TSV("22\t22\t22") + assert TSV(result) == TSV("22\t22\t22") def test_exception_from_destructor(kafka_cluster): @@ -3092,12 +3400,30 @@ def test_exception_from_destructor(kafka_cluster): assert TSV(instance.query("SELECT 1")) == TSV("1") -def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_commits_of_unprocessed_messages_on_drop(kafka_cluster, create_query_generator): + topic_name = "commits_of_unprocessed_messages_on_drop" + get_topic_postfix( + create_query_generator + ) messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(1)] - kafka_produce(kafka_cluster, "commits_of_unprocessed_messages_on_drop", messages) + kafka_produce(kafka_cluster, topic_name, messages) + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}_test_group", + settings={ + "kafka_max_block_size": 1000, + "kafka_flush_interval_ms": 1000, + }, + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.destination SYNC; CREATE TABLE test.destination ( key UInt64, @@ -3112,14 +3438,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): ENGINE = MergeTree() ORDER BY key; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'commits_of_unprocessed_messages_on_drop', - kafka_group_name = 'commits_of_unprocessed_messages_on_drop_test_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + {create_query}; CREATE MATERIALIZED VIEW test.kafka_consumer TO test.destination AS SELECT @@ -3147,9 +3466,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): for _ in range(113): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 - kafka_produce( - kafka_cluster, "commits_of_unprocessed_messages_on_drop", messages - ) + kafka_produce(kafka_cluster, topic_name, messages) time.sleep(0.5) kafka_thread = threading.Thread(target=produce) @@ -3162,18 +3479,17 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): """ ) - instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'commits_of_unprocessed_messages_on_drop', - kafka_group_name = 'commits_of_unprocessed_messages_on_drop_test_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 10000, - kafka_flush_interval_ms = 1000; - """ + new_create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=f"{topic_name}_test_group", + settings={ + "kafka_max_block_size": 10000, + "kafka_flush_interval_ms": 1000, + }, ) + instance.query(new_create_query) cancel.set() instance.wait_for_log_line("kafka.*Stalled", repetitions=5) @@ -3197,20 +3513,29 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): assert TSV(result) == TSV("{0}\t{0}\t{0}".format(i[0] - 1)), "Missing data!" -def test_bad_reschedule(kafka_cluster): - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(20000)] - kafka_produce(kafka_cluster, "test_bad_reschedule", messages) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_bad_reschedule(kafka_cluster, create_query_generator): + topic_name = "test_bad_reschedule" + get_topic_postfix(create_query_generator) + messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(20000)] + kafka_produce(kafka_cluster, topic_name, messages) + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 1000, + "kafka_flush_interval_ms": 1000, + }, + ) instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_bad_reschedule', - kafka_group_name = 'test_bad_reschedule', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + f""" + {create_query}; CREATE MATERIALIZED VIEW test.destination Engine=Log AS SELECT @@ -3308,21 +3633,26 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): # if we came to partition end we will repeat polling until reaching kafka_max_block_size or flush_interval -# that behavior is a bit quesionable - we can just take a bigger pauses between polls instead - +# that behavior is a bit questionable - we can just take a bigger pauses between polls instead - # to do more job in a single pass, and give more rest for a thread. # But in cases of some peaky loads in kafka topic the current contract sounds more predictable and # easier to understand, so let's keep it as is for now. # also we can came to eof because we drained librdkafka internal queue too fast -def test_premature_flush_on_eof(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_premature_flush_on_eof(kafka_cluster, create_query_generator): + topic_name = "premature_flush_on_eof" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'premature_flush_on_eof', - kafka_group_name = 'premature_flush_on_eof', - kafka_format = 'JSONEachRow'; - SELECT * FROM test.kafka LIMIT 1; + f""" + {create_query}; CREATE TABLE test.destination ( key UInt64, value UInt64, @@ -3338,13 +3668,13 @@ def test_premature_flush_on_eof(kafka_cluster): """ ) - # messages created here will be consumed immedeately after MV creation + # messages created here will be consumed immediately after MV creation # reaching topic EOF. - # But we should not do flush immedeately after reaching EOF, because + # But we should not do flush immediately after reaching EOF, because # next poll can return more data, and we should respect kafka_flush_interval_ms # and try to form bigger block - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(1)] - kafka_produce(kafka_cluster, "premature_flush_on_eof", messages) + messages = [json.dumps({"key": 1, "value": 1})] + kafka_produce(kafka_cluster, topic_name, messages) instance.query( """ @@ -3368,7 +3698,7 @@ def test_premature_flush_on_eof(kafka_cluster): instance.wait_for_log_line("Stalled") # produce more messages after delay - kafka_produce(kafka_cluster, "premature_flush_on_eof", messages) + kafka_produce(kafka_cluster, topic_name, messages) # data was not flushed yet (it will be flushed 7.5 sec after creating MV) assert int(instance.query("SELECT count() FROM test.destination")) == 0 @@ -3389,58 +3719,81 @@ def test_premature_flush_on_eof(kafka_cluster): ) -def test_kafka_unavailable(kafka_cluster): - messages = [json.dumps({"key": j + 1, "value": j + 1}) for j in range(20000)] - kafka_produce(kafka_cluster, "test_bad_reschedule", messages) +@pytest.mark.parametrize( + "create_query_generator, do_direct_read", + [(generate_old_create_table_query, True), (generate_new_create_table_query, False)], +) +def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read): + number_of_messages = 20000 + topic_name = "test_bad_reschedule" + get_topic_postfix(create_query_generator) + messages = [ + json.dumps({"key": j + 1, "value": j + 1}) for j in range(number_of_messages) + ] + kafka_produce(kafka_cluster, topic_name, messages) - kafka_cluster.pause_container("kafka1") + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + kafka_cluster.pause_container("kafka1") - instance.query( + create_query = create_query_generator( + "test_bad_reschedule", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + settings={"kafka_max_block_size": 1000}, + ) + instance.query( + f""" + {create_query}; + + CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS + SELECT + key, + now() as consume_ts, + value, + _topic, + _key, + _offset, + _partition, + _timestamp + FROM test.test_bad_reschedule; """ - CREATE TABLE test.test_bad_reschedule (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_bad_reschedule', - kafka_group_name = 'test_bad_reschedule', - kafka_format = 'JSONEachRow', - kafka_commit_on_select = 1, - kafka_max_block_size = 1000; + ) - CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS - SELECT - key, - now() as consume_ts, - value, - _topic, - _key, - _offset, - _partition, - _timestamp - FROM test.test_bad_reschedule; - """ - ) + if do_direct_read: + instance.query("SELECT * FROM test.test_bad_reschedule") + instance.query("SELECT count() FROM test.destination_unavailable") - instance.query("SELECT * FROM test.test_bad_reschedule") - instance.query("SELECT count() FROM test.destination_unavailable") + # enough to trigger issue + time.sleep(30) + kafka_cluster.unpause_container("kafka1") - # enough to trigger issue - time.sleep(30) - kafka_cluster.unpause_container("kafka1") + result = instance.query_with_retry( + "SELECT count() FROM test.destination_unavailable", + sleep_time=1, + check_callback=lambda res: int(res) == number_of_messages, + ) - while ( - int(instance.query("SELECT count() FROM test.destination_unavailable")) < 20000 - ): - print("Waiting for consume") - time.sleep(1) + assert int(result) == number_of_messages -def test_kafka_issue14202(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_issue14202(kafka_cluster, create_query_generator): """ INSERT INTO Kafka Engine from an empty SELECT sub query was leading to failure """ + topic_name = "issue14202" + get_topic_postfix(create_query_generator) + create_query = create_query_generator( + "kafka_q", + "t UInt64, some_string String", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( - """ + f""" CREATE TABLE test.empty_table ( dt Date, some_string String @@ -3449,12 +3802,7 @@ def test_kafka_issue14202(kafka_cluster): PARTITION BY toYYYYMM(dt) ORDER BY some_string; - CREATE TABLE test.kafka_q (t UInt64, `some_string` String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'issue14202', - kafka_group_name = 'issue14202', - kafka_format = 'JSONEachRow'; + {create_query}; """ ) @@ -3505,20 +3853,30 @@ def random_string(size=8): return "".join(random.choices(string.ascii_uppercase + string.digits, k=size)) -def test_kafka_engine_put_errors_to_stream(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_engine_put_errors_to_stream(kafka_cluster, create_query_generator): + topic_name = "kafka_engine_put_errors_to_stream" + get_topic_postfix( + create_query_generator + ) + create_query = create_query_generator( + "kafka", + "i Int64, s String", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 128, + "kafka_handle_error_mode": "stream", + }, + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.kafka; DROP TABLE IF EXISTS test.kafka_data; DROP TABLE IF EXISTS test.kafka_errors; - CREATE TABLE test.kafka (i Int64, s String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_engine_put_errors_to_stream', - kafka_group_name = 'kafka_engine_put_errors_to_stream', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 128, - kafka_handle_error_mode = 'stream'; + {create_query}; CREATE MATERIALIZED VIEW test.kafka_data (i Int64, s String) ENGINE = MergeTree ORDER BY i @@ -3546,19 +3904,20 @@ def test_kafka_engine_put_errors_to_stream(kafka_cluster): json.dumps({"i": "n_" + random_string(4), "s": random_string(8)}) ) - kafka_produce(kafka_cluster, "kafka_engine_put_errors_to_stream", messages) - instance.wait_for_log_line("Committed offset 128") + kafka_produce(kafka_cluster, topic_name, messages) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + instance.wait_for_log_line("Committed offset 128") - assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("64") - assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") + assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("64") + assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") - instance.query( + instance.query( + """ + DROP TABLE test.kafka; + DROP TABLE test.kafka_data; + DROP TABLE test.kafka_errors; """ - DROP TABLE test.kafka; - DROP TABLE test.kafka_data; - DROP TABLE test.kafka_errors; - """ - ) + ) def gen_normal_json(): @@ -3587,21 +3946,35 @@ def gen_message_with_jsons(jsons=10, malformed=0): return s.getvalue() -def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_engine_put_errors_to_stream_with_random_malformed_json( + kafka_cluster, create_query_generator +): + topic_name = ( + "kafka_engine_put_errors_to_stream_with_random_malformed_json" + + get_topic_postfix(create_query_generator) + ) + create_query = create_query_generator( + "kafka", + "i Int64, s String", + topic_list=topic_name, + consumer_group=topic_name, + settings={ + "kafka_max_block_size": 100, + "kafka_poll_max_batch_size": 1, + "kafka_handle_error_mode": "stream", + }, + ) + instance.query( - """ + f""" DROP TABLE IF EXISTS test.kafka; DROP TABLE IF EXISTS test.kafka_data; DROP TABLE IF EXISTS test.kafka_errors; - CREATE TABLE test.kafka (i Int64, s String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_engine_put_errors_to_stream_with_random_malformed_json', - kafka_group_name = 'kafka_engine_put_errors_to_stream_with_random_malformed_json', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 100, - kafka_poll_max_batch_size = 1, - kafka_handle_error_mode = 'stream'; + {create_query}; CREATE MATERIALIZED VIEW test.kafka_data (i Int64, s String) ENGINE = MergeTree ORDER BY i @@ -3626,28 +3999,28 @@ def test_kafka_engine_put_errors_to_stream_with_random_malformed_json(kafka_clus else: messages.append(gen_message_with_jsons(10, 0)) - kafka_produce( - kafka_cluster, - "kafka_engine_put_errors_to_stream_with_random_malformed_json", - messages, - ) + kafka_produce(kafka_cluster, topic_name, messages) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + instance.wait_for_log_line("Committed offset 128") + # 64 good messages, each containing 10 rows + assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("640") + # 64 bad messages, each containing some broken row + assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") - instance.wait_for_log_line("Committed offset 128") - # 64 good messages, each containing 10 rows - assert TSV(instance.query("SELECT count() FROM test.kafka_data")) == TSV("640") - # 64 bad messages, each containing some broken row - assert TSV(instance.query("SELECT count() FROM test.kafka_errors")) == TSV("64") - - instance.query( + instance.query( + """ + DROP TABLE test.kafka; + DROP TABLE test.kafka_data; + DROP TABLE test.kafka_errors; """ - DROP TABLE test.kafka; - DROP TABLE test.kafka_data; - DROP TABLE test.kafka_errors; - """ - ) + ) -def test_kafka_formats_with_broken_message(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator): # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' admin_client = KafkaAdminClient( @@ -3665,7 +4038,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '{"id":"0","blockNo":"BAD","val1":"AM","val2":0.5,"val3":1}', ], - "expected": """{"raw_message":"{\\"id\\":\\"0\\",\\"blockNo\\":\\"BAD\\",\\"val1\\":\\"AM\\",\\"val2\\":0.5,\\"val3\\":1}","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"val1\\":\\"AM\\",\\"val2\\":0.5,\\"val3\\":1}': (while reading the value of key blockNo)"}""", + "expected": { + "raw_message": '{"id":"0","blockNo":"BAD","val1":"AM","val2":0.5,"val3":1}', + "error": 'Cannot parse input: expected \'"\' before: \'BAD","val1":"AM","val2":0.5,"val3":1}\': (while reading the value of key blockNo)', + }, "supports_empty_value": True, "printable": True, }, @@ -3678,7 +4054,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '["0", "BAD", "AM", 0.5, 1]', ], - "expected": """{"raw_message":"[\\"0\\", \\"BAD\\", \\"AM\\", 0.5, 1]","error":"Cannot parse input: expected '\\"' before: 'BAD\\", \\"AM\\", 0.5, 1]': (while reading the value of key blockNo)"}""", + "expected": { + "raw_message": '["0", "BAD", "AM", 0.5, 1]', + "error": "Cannot parse input: expected '\"' before: 'BAD\", \"AM\", 0.5, 1]': (while reading the value of key blockNo)", + }, "supports_empty_value": True, "printable": True, }, @@ -3690,7 +4069,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '["0", "BAD", "AM", 0.5, 1]', ], - "expected": """{"raw_message":"[\\"0\\", \\"BAD\\", \\"AM\\", 0.5, 1]","error":"Cannot parse JSON string: expected opening quote"}""", + "expected": { + "raw_message": '["0", "BAD", "AM", 0.5, 1]', + "error": "Cannot parse JSON string: expected opening quote", + }, "printable": True, }, "TSKV": { @@ -3701,7 +4083,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", ], - "expected": '{"raw_message":"id=0\\tblockNo=BAD\\tval1=AM\\tval2=0.5\\tval3=1\\n","error":"Found garbage after field in TSKV format: blockNo: (at row 1)\\n"}', + "expected": { + "raw_message": "id=0\tblockNo=BAD\tval1=AM\tval2=0.5\tval3=1\n", + "error": "Found garbage after field in TSKV format: blockNo: (at row 1)\n", + }, "printable": True, }, "CSV": { @@ -3712,7 +4097,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '0,"BAD","AM",0.5,1\n', ], - "expected": """{"raw_message":"0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n'"}""", + "expected": { + "raw_message": '0,"BAD","AM",0.5,1\n', + "error": "Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'", + }, "printable": True, "supports_empty_value": True, }, @@ -3724,7 +4112,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n'"}""", + "expected": { + "raw_message": "0\tBAD\tAM\t0.5\t1\n", + "error": "Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'", + }, "supports_empty_value": True, "printable": True, }, @@ -3736,7 +4127,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message '"id","blockNo","val1","val2","val3"\n0,"BAD","AM",0.5,1\n', ], - "expected": """{"raw_message":"\\"id\\",\\"blockNo\\",\\"val1\\",\\"val2\\",\\"val3\\"\\n0,\\"BAD\\",\\"AM\\",0.5,1\\n","error":"Cannot parse input: expected '\\"' before: 'BAD\\",\\"AM\\",0.5,1\\\\n'"}""", + "expected": { + "raw_message": '"id","blockNo","val1","val2","val3"\n0,"BAD","AM",0.5,1\n', + "error": "Cannot parse input: expected '\"' before: 'BAD\",\"AM\",0.5,1\\n'", + }, "printable": True, }, "Values": { @@ -3747,7 +4141,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "(0,'BAD','AM',0.5,1)", ], - "expected": r"""{"raw_message":"(0,'BAD','AM',0.5,1)","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception"}""", + "expected": { + "raw_message": "(0,'BAD','AM',0.5,1)", + "error": "Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception", + }, "supports_empty_value": True, "printable": True, }, @@ -3759,7 +4156,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n"}""", + "expected": { + "raw_message": "id\tblockNo\tval1\tval2\tval3\n0\tBAD\tAM\t0.5\t1\n", + "error": "Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n", + }, "supports_empty_value": True, "printable": True, }, @@ -3771,7 +4171,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message "id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", ], - "expected": """{"raw_message":"id\\tblockNo\\tval1\\tval2\\tval3\\nInt64\\tUInt16\\tString\\tFloat32\\tUInt8\\n0\\tBAD\\tAM\\t0.5\\t1\\n","error":"Cannot parse input: expected '\\\\t' before: 'BAD\\\\tAM\\\\t0.5\\\\t1\\\\n'"}""", + "expected": { + "raw_message": "id\tblockNo\tval1\tval2\tval3\nInt64\tUInt16\tString\tFloat32\tUInt8\n0\tBAD\tAM\t0.5\t1\n", + "error": "Cannot parse input: expected '\\t' before: 'BAD\\tAM\\t0.5\\t1\\n'", + }, "printable": True, }, "Native": { @@ -3782,7 +4185,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x53\x74\x72\x69\x6e\x67\x03\x42\x41\x44\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01", ], - "expected": """{"raw_message":"050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801","error":"Cannot convert: String to UInt16"}""", + "expected": { + "raw_message": "050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801", + "error": "Cannot convert: String to UInt16", + }, "printable": False, }, "RowBinary": { @@ -3793,7 +4199,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01", ], - "expected": '{"raw_message":"00000000000000000342414402414D0000003F01","error":"Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\\n"}', + "expected": { + "raw_message": "00000000000000000342414402414D0000003F01", + "error": "Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\n", + }, "printable": False, }, "RowBinaryWithNamesAndTypes": { @@ -3804,7 +4213,10 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x53\x74\x72\x69\x6e\x67\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01", ], - "expected": '{"raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01","error":"Type of \'blockNo\' must be UInt16, not String"}', + "expected": { + "raw_message": "0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01", + "error": "Type of 'blockNo' must be UInt16, not String", + }, "printable": False, }, "ORC": { @@ -3815,15 +4227,19 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b"\x4f\x52\x43\x0a\x0b\x0a\x03\x00\x00\x00\x12\x04\x08\x01\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x29\x0a\x04\x00\x00\x00\x00\x12\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\xff\x80\xff\x80\xff\x00\xff\x80\xff\x03\x42\x41\x44\xff\x80\xff\x02\x41\x4d\xff\x80\x00\x00\x00\x3f\xff\x80\xff\x01\x0a\x06\x08\x06\x10\x00\x18\x0d\x0a\x06\x08\x06\x10\x01\x18\x17\x0a\x06\x08\x06\x10\x02\x18\x14\x0a\x06\x08\x06\x10\x03\x18\x14\x0a\x06\x08\x06\x10\x04\x18\x2b\x0a\x06\x08\x06\x10\x05\x18\x17\x0a\x06\x08\x00\x10\x00\x18\x02\x0a\x06\x08\x00\x10\x01\x18\x02\x0a\x06\x08\x01\x10\x01\x18\x02\x0a\x06\x08\x00\x10\x02\x18\x02\x0a\x06\x08\x02\x10\x02\x18\x02\x0a\x06\x08\x01\x10\x02\x18\x03\x0a\x06\x08\x00\x10\x03\x18\x02\x0a\x06\x08\x02\x10\x03\x18\x02\x0a\x06\x08\x01\x10\x03\x18\x02\x0a\x06\x08\x00\x10\x04\x18\x02\x0a\x06\x08\x01\x10\x04\x18\x04\x0a\x06\x08\x00\x10\x05\x18\x02\x0a\x06\x08\x01\x10\x05\x18\x02\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x1a\x03\x47\x4d\x54\x0a\x59\x0a\x04\x08\x01\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x08\x03\x10\xec\x02\x1a\x0c\x08\x03\x10\x8e\x01\x18\x1d\x20\xc1\x01\x28\x01\x22\x2e\x08\x0c\x12\x05\x01\x02\x03\x04\x05\x1a\x02\x69\x64\x1a\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x1a\x04\x76\x61\x6c\x31\x1a\x04\x76\x61\x6c\x32\x1a\x04\x76\x61\x6c\x33\x20\x00\x28\x00\x30\x00\x22\x08\x08\x04\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x05\x20\x00\x28\x00\x30\x00\x22\x08\x08\x01\x20\x00\x28\x00\x30\x00\x30\x01\x3a\x04\x08\x01\x50\x00\x3a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x3a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x3a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x40\x90\x4e\x48\x01\x08\xd5\x01\x10\x00\x18\x80\x80\x04\x22\x02\x00\x0b\x28\x5b\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18", ], - "expected": r"""{"raw_message":"4F52430A0B0A030000001204080150000A150A050000000000120C0801120608001000180050000A120A06000000000000120808014202080650000A120A06000000000000120808014202080450000A290A0400000000122108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A150A050000000000120C080112060802100218025000FF80FF80FF00FF80FF03424144FF80FF02414DFF800000003FFF80FF010A0608061000180D0A060806100118170A060806100218140A060806100318140A0608061004182B0A060806100518170A060800100018020A060800100118020A060801100118020A060800100218020A060802100218020A060801100218030A060800100318020A060802100318020A060801100318020A060800100418020A060801100418040A060800100518020A060801100518021204080010001204080010001204080010001204080010001204080010001204080010001A03474D540A590A04080150000A0C0801120608001000180050000A0808014202080650000A0808014202080450000A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A0C080112060802100218025000080310EC021A0C0803108E01181D20C1012801222E080C120501020304051A0269641A07626C6F636B4E6F1A0476616C311A0476616C321A0476616C33200028003000220808042000280030002208080820002800300022080808200028003000220808052000280030002208080120002800300030013A04080150003A0C0801120608001000180050003A0808014202080650003A0808014202080450003A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50003A0C08011206080210021802500040904E480108D5011000188080042202000B285B300682F403034F524318","error":"Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero\/NULL instead of throwing exception."}""", + "expected": { + "raw_message": "4F52430A0B0A030000001204080150000A150A050000000000120C0801120608001000180050000A120A06000000000000120808014202080650000A120A06000000000000120808014202080450000A290A0400000000122108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A150A050000000000120C080112060802100218025000FF80FF80FF00FF80FF03424144FF80FF02414DFF800000003FFF80FF010A0608061000180D0A060806100118170A060806100218140A060806100318140A0608061004182B0A060806100518170A060800100018020A060800100118020A060801100118020A060800100218020A060802100218020A060801100218030A060800100318020A060802100318020A060801100318020A060800100418020A060801100418040A060800100518020A060801100518021204080010001204080010001204080010001204080010001204080010001204080010001A03474D540A590A04080150000A0C0801120608001000180050000A0808014202080650000A0808014202080450000A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50000A0C080112060802100218025000080310EC021A0C0803108E01181D20C1012801222E080C120501020304051A0269641A07626C6F636B4E6F1A0476616C311A0476616C321A0476616C33200028003000220808042000280030002208080820002800300022080808200028003000220808052000280030002208080120002800300030013A04080150003A0C0801120608001000180050003A0808014202080650003A0808014202080450003A2108011A1B09000000000000E03F11000000000000E03F19000000000000E03F50003A0C08011206080210021802500040904E480108D5011000188080042202000B285B300682F403034F524318", + "error": "Cannot parse string 'BAD' as UInt16: syntax error at begin of string. Note: there are toUInt16OrZero and toUInt16OrNull functions, which returns zero/NULL instead of throwing exception.", + }, "printable": False, }, } topic_name_prefix = "format_tests_4_stream_" + topic_name_postfix = get_topic_postfix(create_query_generator) for format_name, format_opts in list(all_formats.items()): logging.debug(f"Set up {format_name}") - topic_name = f"{topic_name_prefix}{format_name}" + topic_name = f"{topic_name_prefix}{format_name}{topic_name_postfix}" data_sample = format_opts["data_sample"] data_prefix = [] raw_message = "_raw_message" @@ -3833,23 +4249,22 @@ def test_kafka_formats_with_broken_message(kafka_cluster): if format_opts.get("printable", False) == False: raw_message = "hex(_raw_message)" kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) + create_query = create_query_generator( + f"kafka_{format_name}", + "id Int64, blockNo UInt16, val1 String, val2 Float32, val3 UInt8", + topic_list=topic_name, + consumer_group=topic_name, + format=format_name, + settings={ + "kafka_handle_error_mode": "stream", + "kafka_flush_interval_ms": 1000, + }, + ) instance.query( - """ + f""" DROP TABLE IF EXISTS test.kafka_{format_name}; - CREATE TABLE test.kafka_{format_name} ( - id Int64, - blockNo UInt16, - val1 String, - val2 Float32, - val3 UInt8 - ) ENGINE = Kafka() - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic_name}', - kafka_group_name = '{topic_name}', - kafka_format = '{format_name}', - kafka_handle_error_mode = 'stream', - kafka_flush_interval_ms = 1000 {extra_settings}; + {create_query}; DROP TABLE IF EXISTS test.kafka_data_{format_name}_mv; CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv Engine=Log AS @@ -3860,12 +4275,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} WHERE length(_error) > 0; - """.format( - topic_name=topic_name, - format_name=format_name, - raw_message=raw_message, - extra_settings=format_opts.get("extra_settings") or "", - ) + """ ) raw_expected = """\ @@ -3900,7 +4310,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): for format_name, format_opts in list(all_formats.items()): logging.debug(f"Checking {format_name}") - topic_name = f"{topic_name_prefix}{format_name}" + topic_name = f"{topic_name_prefix}{format_name}{topic_name_postfix}" # shift offsets by 1 if format supports empty value offsets = ( [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] @@ -3920,229 +4330,222 @@ def test_kafka_formats_with_broken_message(kafka_cluster): assert TSV(result) == TSV(expected), "Proper result for format: {}".format( format_name ) - errors_result = ast.literal_eval( + errors_result = json.loads( instance.query( "SELECT raw_message, error FROM test.kafka_errors_{format_name}_mv format JSONEachRow".format( format_name=format_name ) ) ) - errors_expected = ast.literal_eval(format_opts["expected"]) # print(errors_result.strip()) # print(errors_expected.strip()) assert ( - errors_result["raw_message"] == errors_expected["raw_message"] + errors_result["raw_message"] == format_opts["expected"]["raw_message"] ), "Proper raw_message for format: {}".format(format_name) # Errors text can change, just checking prefixes assert ( - errors_expected["error"] in errors_result["error"] + format_opts["expected"]["error"] in errors_result["error"] ), "Proper error for format: {}".format(format_name) kafka_delete_topic(admin_client, topic_name) -def wait_for_new_data(table_name, prev_count=0, max_retries=120): - retries = 0 - while True: - new_count = int(instance.query("SELECT count() FROM {}".format(table_name))) - print(new_count) - if new_count > prev_count: - return new_count - else: - retries += 1 - time.sleep(0.5) - if retries > max_retries: - raise Exception("No new data :(") +@pytest.mark.parametrize( + "create_query_generator", + [ + generate_old_create_table_query, + generate_new_create_table_query, + ], +) +def test_kafka_consumer_failover(kafka_cluster, create_query_generator): + topic_name = "kafka_consumer_failover" + get_topic_postfix(create_query_generator) + with kafka_topic(get_admin_client(kafka_cluster), topic_name, num_partitions=2): + consumer_group = f"{topic_name}_group" + create_queries = [] + for counter in range(3): + create_queries.append( + create_query_generator( + f"kafka{counter+1}", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=consumer_group, + settings={ + "kafka_max_block_size": 1, + "kafka_poll_timeout_ms": 200, + }, + ) + ) -def test_kafka_consumer_failover(kafka_cluster): - # for backporting: - # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) + instance.query( + f""" + {create_queries[0]}; + {create_queries[1]}; + {create_queries[2]}; - topic_name = "kafka_consumer_failover" - kafka_create_topic(admin_client, topic_name, num_partitions=2) + CREATE TABLE test.destination ( + key UInt64, + value UInt64, + _consumed_by LowCardinality(String) + ) + ENGINE = MergeTree() + ORDER BY key; - instance.query( - """ - DROP TABLE IF EXISTS test.kafka; - DROP TABLE IF EXISTS test.kafka2; + CREATE MATERIALIZED VIEW test.kafka1_mv TO test.destination AS + SELECT key, value, 'kafka1' as _consumed_by + FROM test.kafka1; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_consumer_failover', - kafka_group_name = 'kafka_consumer_failover_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1, - kafka_poll_timeout_ms = 200; + CREATE MATERIALIZED VIEW test.kafka2_mv TO test.destination AS + SELECT key, value, 'kafka2' as _consumed_by + FROM test.kafka2; - CREATE TABLE test.kafka2 (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_consumer_failover', - kafka_group_name = 'kafka_consumer_failover_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1, - kafka_poll_timeout_ms = 200; - - CREATE TABLE test.kafka3 (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'kafka_consumer_failover', - kafka_group_name = 'kafka_consumer_failover_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1, - kafka_poll_timeout_ms = 200; - - CREATE TABLE test.destination ( - key UInt64, - value UInt64, - _consumed_by LowCardinality(String) + CREATE MATERIALIZED VIEW test.kafka3_mv TO test.destination AS + SELECT key, value, 'kafka3' as _consumed_by + FROM test.kafka3; + """ ) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.kafka_mv TO test.destination AS - SELECT key, value, 'kafka' as _consumed_by - FROM test.kafka; + producer = KafkaProducer( + bootstrap_servers="localhost:{}".format(cluster.kafka_port), + value_serializer=producer_serializer, + key_serializer=producer_serializer, + ) - CREATE MATERIALIZED VIEW test.kafka2_mv TO test.destination AS - SELECT key, value, 'kafka2' as _consumed_by - FROM test.kafka2; + ## all 3 attached, 2 working + producer.send( + topic=topic_name, + value=json.dumps({"key": 1, "value": 1}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 1, "value": 1}), + partition=1, + ) + producer.flush() - CREATE MATERIALIZED VIEW test.kafka3_mv TO test.destination AS - SELECT key, value, 'kafka3' as _consumed_by - FROM test.kafka3; - """ - ) + count_query = "SELECT count() FROM test.destination" + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > 0 + ) - producer = KafkaProducer( - bootstrap_servers="localhost:{}".format(cluster.kafka_port), - value_serializer=producer_serializer, - key_serializer=producer_serializer, - ) + ## 2 attached, 2 working + instance.query("DETACH TABLE test.kafka1") + producer.send( + topic=topic_name, + value=json.dumps({"key": 2, "value": 2}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 2, "value": 2}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) - ## all 3 attached, 2 working - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 1, "value": 1}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 1, "value": 1}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination") + ## 1 attached, 1 working + instance.query("DETACH TABLE test.kafka2") + producer.send( + topic=topic_name, + value=json.dumps({"key": 3, "value": 3}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 3, "value": 3}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) - ## 2 attached, 2 working - instance.query("DETACH TABLE test.kafka") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 2, "value": 2}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 2, "value": 2}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 2 attached, 2 working + instance.query("ATTACH TABLE test.kafka1") + producer.send( + topic=topic_name, + value=json.dumps({"key": 4, "value": 4}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 4, "value": 4}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) - ## 1 attached, 1 working - instance.query("DETACH TABLE test.kafka2") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 3, "value": 3}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 3, "value": 3}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 1 attached, 1 working + instance.query("DETACH TABLE test.kafka3") + producer.send( + topic=topic_name, + value=json.dumps({"key": 5, "value": 5}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 5, "value": 5}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) - ## 2 attached, 2 working - instance.query("ATTACH TABLE test.kafka") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 4, "value": 4}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 4, "value": 4}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 2 attached, 2 working + instance.query("ATTACH TABLE test.kafka2") + producer.send( + topic=topic_name, + value=json.dumps({"key": 6, "value": 6}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 6, "value": 6}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) - ## 1 attached, 1 working - instance.query("DETACH TABLE test.kafka3") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 5, "value": 5}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 5, "value": 5}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) + ## 3 attached, 2 working + instance.query("ATTACH TABLE test.kafka3") + producer.send( + topic=topic_name, + value=json.dumps({"key": 7, "value": 7}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 7, "value": 7}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) - ## 2 attached, 2 working - instance.query("ATTACH TABLE test.kafka2") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 6, "value": 6}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 6, "value": 6}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) - - ## 3 attached, 2 working - instance.query("ATTACH TABLE test.kafka3") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 7, "value": 7}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 7, "value": 7}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) - - ## 2 attached, same 2 working - instance.query("DETACH TABLE test.kafka3") - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 8, "value": 8}), - partition=0, - ) - producer.send( - topic="kafka_consumer_failover", - value=json.dumps({"key": 8, "value": 8}), - partition=1, - ) - producer.flush() - prev_count = wait_for_new_data("test.destination", prev_count) - kafka_delete_topic(admin_client, topic_name) + ## 2 attached, same 2 working + instance.query("DETACH TABLE test.kafka3") + producer.send( + topic=topic_name, + value=json.dumps({"key": 8, "value": 8}), + partition=0, + ) + producer.send( + topic=topic_name, + value=json.dumps({"key": 8, "value": 8}), + partition=1, + ) + producer.flush() + prev_count = instance.query_with_retry( + count_query, check_callback=lambda res: int(res) > prev_count + ) def test_kafka_predefined_configuration(kafka_cluster): @@ -4172,269 +4575,261 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 -def test_issue26643(kafka_cluster): - # for backporting: - # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_issue26643(kafka_cluster, create_query_generator): producer = KafkaProducer( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer, ) + topic_name = "test_issue26643" + get_topic_postfix(create_query_generator) + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) - topic_list = [] - topic_list.append( - NewTopic(name="test_issue26643", num_partitions=4, replication_factor=1) - ) - admin_client.create_topics(new_topics=topic_list, validate_only=False) - - msg = message_with_repeated_pb2.Message( - tnow=1629000000, - server="server1", - clien="host1", - sPort=443, - cPort=50000, - r=[ - message_with_repeated_pb2.dd( - name="1", type=444, ttl=123123, data=b"adsfasd" - ), - message_with_repeated_pb2.dd(name="2"), - ], - method="GET", - ) - - data = b"" - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - - msg = message_with_repeated_pb2.Message(tnow=1629000002) - - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - - producer.send(topic="test_issue26643", value=data) - - data = _VarintBytes(len(serialized_msg)) + serialized_msg - producer.send(topic="test_issue26643", value=data) - producer.flush() - - instance.query( - """ - CREATE TABLE IF NOT EXISTS test.test_queue - ( - `tnow` UInt32, - `server` String, - `client` String, - `sPort` UInt16, - `cPort` UInt16, - `r.name` Array(String), - `r.class` Array(UInt16), - `r.type` Array(UInt16), - `r.ttl` Array(UInt32), - `r.data` Array(String), - `method` String + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + msg = message_with_repeated_pb2.Message( + tnow=1629000000, + server="server1", + clien="host1", + sPort=443, + cPort=50000, + r=[ + message_with_repeated_pb2.dd( + name="1", type=444, ttl=123123, data=b"adsfasd" + ), + message_with_repeated_pb2.dd(name="2"), + ], + method="GET", ) - ENGINE = Kafka - SETTINGS - kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_issue26643', - kafka_group_name = 'test_issue26643_group', - kafka_format = 'Protobuf', - kafka_schema = 'message_with_repeated.proto:Message', - kafka_num_consumers = 4, - kafka_skip_broken_messages = 10000; - SET allow_suspicious_low_cardinality_types=1; + data = b"" + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - CREATE TABLE test.log - ( - `tnow` DateTime('Asia/Istanbul') CODEC(DoubleDelta, LZ4), - `server` LowCardinality(String), - `client` LowCardinality(String), - `sPort` LowCardinality(UInt16), - `cPort` UInt16 CODEC(T64, LZ4), - `r.name` Array(String), - `r.class` Array(LowCardinality(UInt16)), - `r.type` Array(LowCardinality(UInt16)), - `r.ttl` Array(LowCardinality(UInt32)), - `r.data` Array(String), - `method` LowCardinality(String) + msg = message_with_repeated_pb2.Message(tnow=1629000002) + + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + + producer.send(topic_name, value=data) + + data = _VarintBytes(len(serialized_msg)) + serialized_msg + producer.send(topic_name, value=data) + producer.flush() + + create_query = create_query_generator( + "test_queue", + """`tnow` UInt32, + `server` String, + `client` String, + `sPort` UInt16, + `cPort` UInt16, + `r.name` Array(String), + `r.class` Array(UInt16), + `r.type` Array(UInt16), + `r.ttl` Array(UInt32), + `r.data` Array(String), + `method` String""", + topic_list=topic_name, + consumer_group=f"{topic_name}_group", + format="Protobuf", + settings={ + "kafka_schema": "message_with_repeated.proto:Message", + "kafka_skip_broken_messages": 10000, + "kafka_thread_per_consumer": thread_per_consumer, + }, ) - ENGINE = MergeTree - PARTITION BY toYYYYMMDD(tnow) - ORDER BY (tnow, server) - TTL toDate(tnow) + toIntervalMonth(1000) - SETTINGS index_granularity = 16384, merge_with_ttl_timeout = 7200; - CREATE MATERIALIZED VIEW test.test_consumer TO test.log AS - SELECT - toDateTime(a.tnow) AS tnow, - a.server AS server, - a.client AS client, - a.sPort AS sPort, - a.cPort AS cPort, - a.`r.name` AS `r.name`, - a.`r.class` AS `r.class`, - a.`r.type` AS `r.type`, - a.`r.ttl` AS `r.ttl`, - a.`r.data` AS `r.data`, - a.method AS method - FROM test.test_queue AS a; - """ - ) + instance.query( + f""" + {create_query}; - instance.wait_for_log_line("Committed offset") - result = instance.query("SELECT * FROM test.log") + SET allow_suspicious_low_cardinality_types=1; - expected = """\ -2021-08-15 07:00:00 server1 443 50000 ['1','2'] [0,0] [444,0] [123123,0] ['adsfasd',''] GET -2021-08-15 07:00:02 0 0 [] [] [] [] [] -2021-08-15 07:00:02 0 0 [] [] [] [] [] -""" - assert TSV(result) == TSV(expected) + CREATE TABLE test.log + ( + `tnow` DateTime('Asia/Istanbul') CODEC(DoubleDelta, LZ4), + `server` LowCardinality(String), + `client` LowCardinality(String), + `sPort` LowCardinality(UInt16), + `cPort` UInt16 CODEC(T64, LZ4), + `r.name` Array(String), + `r.class` Array(LowCardinality(UInt16)), + `r.type` Array(LowCardinality(UInt16)), + `r.ttl` Array(LowCardinality(UInt32)), + `r.data` Array(String), + `method` LowCardinality(String) + ) + ENGINE = MergeTree + PARTITION BY toYYYYMMDD(tnow) + ORDER BY (tnow, server) + TTL toDate(tnow) + toIntervalMonth(1000) + SETTINGS index_granularity = 16384, merge_with_ttl_timeout = 7200; - # kafka_cluster.open_bash_shell('instance') + CREATE MATERIALIZED VIEW test.test_consumer TO test.log AS + SELECT + toDateTime(a.tnow) AS tnow, + a.server AS server, + a.client AS client, + a.sPort AS sPort, + a.cPort AS cPort, + a.`r.name` AS `r.name`, + a.`r.class` AS `r.class`, + a.`r.type` AS `r.type`, + a.`r.ttl` AS `r.ttl`, + a.`r.data` AS `r.data`, + a.method AS method + FROM test.test_queue AS a; + """ + ) + + instance.wait_for_log_line("Committed offset") + result = instance.query("SELECT * FROM test.log") + + expected = """\ + 2021-08-15 07:00:00 server1 443 50000 ['1','2'] [0,0] [444,0] [123123,0] ['adsfasd',''] GET + 2021-08-15 07:00:02 0 0 [] [] [] [] [] + 2021-08-15 07:00:02 0 0 [] [] [] [] [] + """ + assert TSV(result) == TSV(expected) -def test_num_consumers_limit(kafka_cluster): +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_num_consumers_limit(kafka_cluster, create_query_generator): instance.query("DROP TABLE IF EXISTS test.kafka") - error = instance.query_and_get_error( - """ - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n', '', 100) - SETTINGS kafka_commit_on_select = 1; - """ + thread_per_consumer = must_use_thread_per_consumer(create_query_generator) + + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + settings={ + "kafka_num_consumers": 100, + "kafka_thread_per_consumer": thread_per_consumer, + }, + ) + error = instance.query_and_get_error(create_query) + + assert ( + "BAD_ARGUMENTS" in error + and "The number of consumers can not be bigger than" in error ) - assert "BAD_ARGUMENTS" in error - instance.query( - """ + f""" SET kafka_disable_num_consumers_limit = 1; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n', '', 100) - SETTINGS kafka_commit_on_select = 1; + {create_query}; """ ) instance.query("DROP TABLE test.kafka") -def test_format_with_prefix_and_suffix(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_format_with_prefix_and_suffix(kafka_cluster, create_query_generator): + topic_name = "custom" + get_topic_postfix(create_query_generator) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="CustomSeparated", + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + {create_query}; + """ + ) + + instance.query( + "INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) + + message_count = 2 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) + + assert len(messages) == 2 + + assert ( + "".join(messages) + == "\n0\t0\n\n\n10\t100\n\n" + ) + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_max_rows_per_message(kafka_cluster, create_query_generator): + topic_name = "custom_max_rows_per_message" + get_topic_postfix( + create_query_generator ) - kafka_create_topic(admin_client, "custom") + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + num_rows = 5 - instance.query( + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="CustomSeparated", + settings={ + "format_custom_result_before_delimiter": "\n", + "format_custom_result_after_delimiter": "\n", + "kafka_max_rows_per_message": 3, + }, + ) + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + {create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; """ - DROP TABLE IF EXISTS test.kafka; + ) - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'custom', - kafka_group_name = 'custom', - kafka_format = 'CustomSeparated'; - """ - ) + instance.query( + f"INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" + ) - instance.query( - "INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) + message_count = 2 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) - messages = [] + assert len(messages) == message_count - attempt = 0 - while attempt < 100: - messages.extend(kafka_consume(kafka_cluster, "custom")) - if len(messages) == 2: - break - attempt += 1 + assert ( + "".join(messages) + == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" + ) - assert len(messages) == 2 + instance.query_with_retry( + "SELECT count() FROM test.view", + check_callback=lambda res: int(res) == num_rows, + ) - assert ( - "".join(messages) == "\n0\t0\n\n\n10\t100\n\n" - ) - - kafka_delete_topic(admin_client, "custom") + result = instance.query("SELECT * FROM test.view") + assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" -def test_max_rows_per_message(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic = "custom_max_rows_per_message" - - kafka_create_topic(admin_client, topic) - - num_rows = 5 - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = 'CustomSeparated', - format_custom_result_before_delimiter = '\n', - format_custom_result_after_delimiter = '\n', - kafka_max_rows_per_message = 3; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; - """ - ) - - instance.query( - f"INSERT INTO test.kafka select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - messages = [] - - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, topic)) - if len(messages) == 2: - break - attempt += 1 - - assert len(messages) == 2 - - assert ( - "".join(messages) - == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" - ) - - attempt = 0 - rows = 0 - while attempt < 500: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" - - kafka_delete_topic(admin_client, topic) - - -def test_row_based_formats(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_row_based_formats(kafka_cluster, create_query_generator): + admin_client = get_admin_client(kafka_cluster) for format_name in [ "TSV", @@ -4454,121 +4849,111 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) + logging.debug("Checking {format_name}") - kafka_create_topic(admin_client, format_name) + topic_name = format_name + get_topic_postfix(create_query_generator) + table_name = f"kafka_{format_name}" - num_rows = 10 + with kafka_topic(admin_client, topic_name): + num_rows = 10 + max_rows_per_message = 5 + message_count = num_rows / max_rows_per_message + create_query = create_query_generator( + table_name, + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format=format_name, + settings={"kafka_max_rows_per_message": max_rows_per_message}, + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.{table_name}; + + {create_query}; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.{table_name}; + + INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); + """ + ) + + messages = kafka_consume_with_retry( + kafka_cluster, topic_name, message_count, need_decode=False + ) + + assert len(messages) == message_count + + instance.query_with_retry( + "SELECT count() FROM test.view", + check_callback=lambda res: int(res) == num_rows, + ) + + result = instance.query("SELECT * FROM test.view") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_block_based_formats_1(kafka_cluster, create_query_generator): + topic_name = "pretty_space" + get_topic_postfix(create_query_generator) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + create_query = create_query_generator( + "kafka", + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format="PrettySpace", + ) instance.query( f""" - DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}', - kafka_group_name = '{format_name}', - kafka_format = '{format_name}', - kafka_max_rows_per_message = 5; + {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; - - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1, output_format_pretty_row_numbers=0; """ ) - messages = [] + message_count = 3 + messages = kafka_consume_with_retry(kafka_cluster, topic_name, message_count) + assert len(messages) == 3 - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, format_name, needDecode=False)) - if len(messages) == 2: - break - attempt += 1 + data = [] + for message in messages: + splitted = message.split("\n") + assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" + assert splitted[1] == "" + assert splitted[-1] == "" + data += [line.split() for line in splitted[2:-1]] - assert len(messages) == 2 - - attempt = 0 - rows = 0 - while attempt < 500: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - kafka_delete_topic(admin_client, format_name) + assert data == [ + ["0", "0"], + ["10", "100"], + ["20", "200"], + ["30", "300"], + ["40", "400"], + ] -def test_block_based_formats_1(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic = "pretty_space" - kafka_create_topic(admin_client, topic) - - instance.query( - f""" - DROP TABLE IF EXISTS test.kafka; - - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = 'PrettySpace'; - - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1, output_format_pretty_row_numbers=0; - """ - ) - - messages = [] - - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, topic)) - if len(messages) == 3: - break - attempt += 1 - - assert len(messages) == 3 - - data = [] - for message in messages: - splitted = message.split("\n") - assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" - assert splitted[1] == "" - assert splitted[-1] == "" - data += [line.split() for line in splitted[2:-1]] - - assert data == [ - ["0", "0"], - ["10", "100"], - ["20", "200"], - ["30", "300"], - ["40", "400"], - ] - - kafka_delete_topic(admin_client, topic) - - -def test_block_based_formats_2(kafka_cluster): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_block_based_formats_2(kafka_cluster, create_query_generator): + admin_client = get_admin_client(kafka_cluster) num_rows = 100 + message_count = 9 for format_name in [ "JSONColumns", @@ -4578,55 +4963,50 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) + topic_name = format_name + get_topic_postfix(create_query_generator) + table_name = f"kafka_{format_name}" + logging.debug(f"Checking format {format_name}") + with kafka_topic(admin_client, topic_name): + create_query = create_query_generator( + table_name, + "key UInt64, value UInt64", + topic_list=topic_name, + consumer_group=topic_name, + format=format_name, + ) - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.kafka; + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.{table_name}; - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}', - kafka_group_name = '{format_name}', - kafka_format = '{format_name}'; + {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.kafka; + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.{table_name}; - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; - """ - ) + INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; + """ + ) + messages = kafka_consume_with_retry( + kafka_cluster, topic_name, message_count, need_decode=False + ) + assert len(messages) == message_count - messages = [] + rows = int( + instance.query_with_retry( + "SELECT count() FROM test.view", + check_callback=lambda res: int(res) == num_rows, + ) + ) - attempt = 0 - while attempt < 500: - messages.extend(kafka_consume(kafka_cluster, format_name, needDecode=False)) - if len(messages) == 9: - break - attempt += 1 + assert rows == num_rows - assert len(messages) == 9 - - attempt = 0 - rows = 0 - while attempt < 500: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view ORDER by key") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - kafka_delete_topic(admin_client, format_name) + result = instance.query("SELECT * FROM test.view ORDER by key") + expected = "" + for i in range(num_rows): + expected += str(i * 10) + "\t" + str(i * 100) + "\n" + assert result == expected def test_system_kafka_consumers(kafka_cluster): @@ -4962,137 +5342,186 @@ def test_formats_errors(kafka_cluster): "HiveText", "MySQLDump", ]: - kafka_create_topic(admin_client, format_name) - table_name = f"kafka_{format_name}" + with kafka_topic(admin_client, format_name): + table_name = f"kafka_{format_name}" + instance.query( + f""" + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.{table_name}; + + CREATE TABLE test.{table_name} (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{format_name}', + kafka_group_name = '{format_name}', + kafka_format = '{format_name}', + kafka_max_rows_per_message = 5, + format_template_row='template_row.format', + format_regexp='id: (.+?)', + input_format_with_names_use_header=0, + format_schema='key_value_message:Message'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.{table_name}; + """ + ) + + kafka_produce( + kafka_cluster, + format_name, + ["Broken message\nBroken message\nBroken message\n"], + ) + + num_errors = int( + instance.query_with_retry( + f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'", + check_callback=lambda res: int(res) > 0, + ) + ) + + assert num_errors > 0 + + instance.query(f"DROP TABLE test.{table_name}") + instance.query("DROP TABLE test.view") + + +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_multiple_read_in_materialized_views(kafka_cluster, create_query_generator): + topic_name = "multiple_read_from_mv" + get_topic_postfix(create_query_generator) + + with kafka_topic(get_admin_client(kafka_cluster), topic_name): + create_query = create_query_generator( + "kafka_multiple_read_input", + "id Int64", + topic_list=topic_name, + consumer_group=topic_name, + ) instance.query( f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.{table_name}; + DROP TABLE IF EXISTS test.kafka_multiple_read_input SYNC; + DROP TABLE IF EXISTS test.kafka_multiple_read_table; + DROP TABLE IF EXISTS test.kafka_multiple_read_mv; - CREATE TABLE test.{table_name} (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{format_name}', - kafka_group_name = '{format_name}', - kafka_format = '{format_name}', - kafka_max_rows_per_message = 5, - format_template_row='template_row.format', - format_regexp='id: (.+?)', - input_format_with_names_use_header=0, - format_schema='key_value_message:Message'; + {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.{table_name}; - """ + CREATE TABLE test.kafka_multiple_read_table (id Int64) + ENGINE = MergeTree + ORDER BY id; + + + CREATE MATERIALIZED VIEW test.kafka_multiple_read_mv TO test.kafka_multiple_read_table AS + SELECT id + FROM test.kafka_multiple_read_input + WHERE id NOT IN ( + SELECT id + FROM test.kafka_multiple_read_table + WHERE id IN ( + SELECT id + FROM test.kafka_multiple_read_input + ) + ); + """ ) kafka_produce( - kafka_cluster, - format_name, - ["Broken message\nBroken message\nBroken message\n"], + kafka_cluster, topic_name, [json.dumps({"id": 42}), json.dumps({"id": 43})] ) - attempt = 0 - num_errors = 0 - while attempt < 200: - num_errors = int( - instance.query( - f"SELECT length(exceptions.text) from system.kafka_consumers where database = 'test' and table = '{table_name}'" - ) - ) - if num_errors > 0: - break - attempt += 1 + expected_result = "42\n43\n" + res = instance.query_with_retry( + f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", + check_callback=lambda res: res == expected_result, + ) + assert res == expected_result - assert num_errors > 0 + # Verify that the query deduplicates the records as it meant to be + messages = [] + for _ in range(0, 10): + messages.append(json.dumps({"id": 42})) + messages.append(json.dumps({"id": 43})) - kafka_delete_topic(admin_client, format_name) - instance.query(f"DROP TABLE test.{table_name}") - instance.query("DROP TABLE test.view") + messages.append(json.dumps({"id": 44})) + + kafka_produce(kafka_cluster, topic_name, messages) + + expected_result = "42\n43\n44\n" + res = instance.query_with_retry( + f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", + check_callback=lambda res: res == expected_result, + ) + assert res == expected_result + + instance.query( + f""" + DROP TABLE test.kafka_multiple_read_input; + DROP TABLE test.kafka_multiple_read_table; + DROP TABLE test.kafka_multiple_read_mv; + """ + ) -def test_multiple_read_in_materialized_views(kafka_cluster, max_retries=15): - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic = "multiple_read_from_mv" - kafka_create_topic(admin_client, topic) +@pytest.mark.parametrize( + "create_query_generator", + [generate_old_create_table_query, generate_new_create_table_query], +) +def test_kafka_null_message(kafka_cluster, create_query_generator): + topic_name = "null_message" instance.query( f""" - DROP TABLE IF EXISTS test.kafka_multiple_read_input; - DROP TABLE IF EXISTS test.kafka_multiple_read_table; - DROP TABLE IF EXISTS test.kafka_multiple_read_mv; + DROP TABLE IF EXISTS test.null_message_view; + DROP TABLE IF EXISTS test.null_message_consumer; + DROP TABLE IF EXISTS test.null_message_kafka; - CREATE TABLE test.kafka_multiple_read_input (id Int64) - ENGINE = Kafka - SETTINGS - kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = 'JSONEachRow'; + {create_query_generator("null_message_kafka", "value UInt64", topic_list=topic_name, consumer_group="mv")}; + CREATE TABLE test.null_message_view (value UInt64) + ENGINE = MergeTree() + ORDER BY value; + CREATE MATERIALIZED VIEW test.null_message_consumer TO test.null_message_view AS + SELECT * FROM test.null_message_kafka; + """ + ) - CREATE TABLE test.kafka_multiple_read_table (id Int64) - ENGINE = MergeTree - ORDER BY id; + message_key_values = [] + for i in range(5): + # Here the key is key for Kafka message + message = json.dumps({"value": i}) if i != 3 else None + message_key_values.append({"key": f"{i}".encode(), "message": message}) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer, 15) + for message_kv in message_key_values: + producer.send( + topic=topic_name, key=message_kv["key"], value=message_kv["message"] + ) + producer.flush() - CREATE MATERIALIZED VIEW IF NOT EXISTS test.kafka_multiple_read_mv TO test.kafka_multiple_read_table AS - SELECT id - FROM test.kafka_multiple_read_input - WHERE id NOT IN ( - SELECT id - FROM test.kafka_multiple_read_table - WHERE id IN ( - SELECT id - FROM test.kafka_multiple_read_input - ) - ); + expected = TSV( """ +0 +1 +2 +4 +""" ) + with existing_kafka_topic(get_admin_client(kafka_cluster), topic_name): + result = instance.query_with_retry( + "SELECT * FROM test.null_message_view", + check_callback=lambda res: TSV(res) == expected, + ) - kafka_produce( - kafka_cluster, topic, [json.dumps({"id": 42}), json.dumps({"id": 43})] - ) + assert expected == TSV(result) - expected_result = "42\n43\n" - res = instance.query_with_retry( - f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", - retry_count=30, - sleep_time=0.5, - check_callback=lambda res: res == expected_result, - ) - assert res == expected_result - - # Verify that the query deduplicates the records as it meant to be - messages = [] - for i in range(0, 10): - messages.append(json.dumps({"id": 42})) - messages.append(json.dumps({"id": 43})) - - messages.append(json.dumps({"id": 44})) - - kafka_produce(kafka_cluster, topic, messages) - - expected_result = "42\n43\n44\n" - res = instance.query_with_retry( - f"SELECT id FROM test.kafka_multiple_read_table ORDER BY id", - retry_count=30, - sleep_time=0.5, - check_callback=lambda res: res == expected_result, - ) - assert res == expected_result - - kafka_delete_topic(admin_client, topic) - instance.query( - f""" - DROP TABLE test.kafka_multiple_read_input; - DROP TABLE test.kafka_multiple_read_table; - DROP TABLE test.kafka_multiple_read_mv; + instance.query( + """ + DROP TABLE test.null_message_consumer SYNC; + DROP TABLE test.null_message_view; + DROP TABLE test.null_message_kafka SYNC; """ - ) + ) if __name__ == "__main__": diff --git a/tests/queries/0_stateless/00284_external_aggregation.reference b/tests/queries/0_stateless/00284_external_aggregation.reference index be0db217a97..48e30e781e0 100644 --- a/tests/queries/0_stateless/00284_external_aggregation.reference +++ b/tests/queries/0_stateless/00284_external_aggregation.reference @@ -1,22 +1,2 @@ 49999995000000 10000000 499999500000 1000000 15 -100033 2 -100034 2 -100035 2 -100036 2 -100037 2 -100038 2 -100039 2 -10004 2 -100040 2 -100041 2 -100033 2 -100034 2 -100035 2 -100036 2 -100037 2 -100038 2 -100039 2 -10004 2 -100040 2 -100041 2 diff --git a/tests/queries/0_stateless/00284_external_aggregation.sql b/tests/queries/0_stateless/00284_external_aggregation.sql index c1140faaa28..cdc31ff68c8 100644 --- a/tests/queries/0_stateless/00284_external_aggregation.sql +++ b/tests/queries/0_stateless/00284_external_aggregation.sql @@ -1,5 +1,8 @@ -- Tags: long +-- This test was split in two due to long runtimes in sanitizers. +-- The other part is 00284_external_aggregation_2. + SET max_bytes_before_external_group_by = 100000000; SET max_memory_usage = 410000000; SET group_by_two_level_threshold = 100000; @@ -7,19 +10,3 @@ SET group_by_two_level_threshold_bytes = 50000000; SELECT sum(k), sum(c) FROM (SELECT number AS k, count() AS c FROM (SELECT * FROM system.numbers LIMIT 10000000) GROUP BY k); SELECT sum(k), sum(c), max(u) FROM (SELECT number AS k, count() AS c, uniqArray(range(number % 16)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000000) GROUP BY k); - -SET max_memory_usage = 0; -SET group_by_two_level_threshold = 100000; -SET max_bytes_before_external_group_by = '1Mi'; - --- method: key_string & key_string_two_level -CREATE TABLE t_00284_str(s String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); -INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); -SELECT s, count() FROM t_00284_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; - --- method: low_cardinality_key_string & low_cardinality_key_string_two_level -CREATE TABLE t_00284_lc_str(s LowCardinality(String)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); -INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); -SELECT s, count() FROM t_00284_lc_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; diff --git a/tests/queries/0_stateless/00284_external_aggregation_2.reference b/tests/queries/0_stateless/00284_external_aggregation_2.reference new file mode 100644 index 00000000000..71d2e96d4b0 --- /dev/null +++ b/tests/queries/0_stateless/00284_external_aggregation_2.reference @@ -0,0 +1,20 @@ +100033 2 +100034 2 +100035 2 +100036 2 +100037 2 +100038 2 +100039 2 +10004 2 +100040 2 +100041 2 +100033 2 +100034 2 +100035 2 +100036 2 +100037 2 +100038 2 +100039 2 +10004 2 +100040 2 +100041 2 diff --git a/tests/queries/0_stateless/00284_external_aggregation_2.sql b/tests/queries/0_stateless/00284_external_aggregation_2.sql new file mode 100644 index 00000000000..7960e3894d0 --- /dev/null +++ b/tests/queries/0_stateless/00284_external_aggregation_2.sql @@ -0,0 +1,22 @@ +-- Tags: long + +-- This test was split in two due to long runtimes in sanitizers. +-- The other part is 00284_external_aggregation. + +SET group_by_two_level_threshold_bytes = 50000000; +SET max_memory_usage = 0; +SET group_by_two_level_threshold = 100000; +SET max_bytes_before_external_group_by = '1Mi'; + +-- method: key_string & key_string_two_level +CREATE TABLE t_00284_str(s String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); +INSERT INTO t_00284_str SELECT toString(number) FROM numbers_mt(1e6); +SELECT s, count() FROM t_00284_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; + +-- method: low_cardinality_key_string & low_cardinality_key_string_two_level +CREATE TABLE t_00284_lc_str(s LowCardinality(String)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); +INSERT INTO t_00284_lc_str SELECT toString(number) FROM numbers_mt(1e6); +SELECT s, count() FROM t_00284_lc_str GROUP BY s ORDER BY s LIMIT 10 OFFSET 42; + diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 16ad08deeb2..3e6b339cb57 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: replica, no-debug +# Tags: replica, no-debug, no-fasttest +# no-fasttest: Waiting for failed mutations is slow: https://github.com/ClickHouse/ClickHouse/issues/67936 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.reference b/tests/queries/0_stateless/00945_bloom_filter_index.reference index e6751fe4762..9d9b49b29c9 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.reference +++ b/tests/queries/0_stateless/00945_bloom_filter_index.reference @@ -227,3 +227,5 @@ 1 value1 1 value2 2 value3 +1 +1 diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql index 2b7feacbd98..6e3819e74d3 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -374,3 +374,10 @@ SELECT id, ary[indexOf(ary, 'value2')] FROM test_bf_indexOf WHERE ary[indexOf(ar SELECT id, ary[indexOf(ary, 'value3')] FROM test_bf_indexOf WHERE ary[indexOf(ary, 'value3')] = 'value3' ORDER BY id FORMAT TSV; DROP TABLE IF EXISTS test_bf_indexOf; + +-- Test for bug #65597 +DROP TABLE IF EXISTS test_bf_cast; +CREATE TABLE test_bf_cast (c Int32, INDEX x1 (c) type bloom_filter) ENGINE = MergeTree ORDER BY c AS SELECT 1; +SELECT count() FROM test_bf_cast WHERE cast(c = 1 OR c = 9999 AS Bool) SETTINGS use_skip_indexes=0; +SELECT count() FROM test_bf_cast WHERE cast(c = 1 OR c = 9999 AS Bool) SETTINGS use_skip_indexes=1; +DROP TABLE test_bf_cast; diff --git a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql index 16b62c37d80..f2afb974a06 100644 --- a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql +++ b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql @@ -1,9 +1,10 @@ --- Tags: shard, no-parallel +-- Tags: shard, no-fasttest +-- no-fasttest: Slow timeouts SET prefer_localhost_replica = 1; +SET connections_with_failover_max_tries=1; +SET connect_timeout_with_failover_ms=2000; +SET connect_timeout_with_failover_secure_ms=2000; SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError ALL_CONNECTION_TRIES_FAILED } SELECT count() FROM remote('127.0.0.1|localhos', system.one); - --- Clear cache to avoid future errors in the logs -SYSTEM DROP DNS CACHE diff --git a/tests/queries/0_stateless/01030_storage_url_syntax.sql b/tests/queries/0_stateless/01030_storage_url_syntax.sql index 0eb89af8462..084486b61ee 100644 --- a/tests/queries/0_stateless/01030_storage_url_syntax.sql +++ b/tests/queries/0_stateless/01030_storage_url_syntax.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: Timeout for the first query (CANNOT_DETECT_FORMAT) is too slow: https://github.com/ClickHouse/ClickHouse/issues/67939 + drop table if exists test_table_url_syntax ; create table test_table_url_syntax (id UInt32) ENGINE = URL('') diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index be95e5a7d4b..c5012dabc81 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -27,7 +27,6 @@ RENAME DICTIONARY test_01191.t TO test_01191.dict1; -- {serverError INCORRECT_QU DROP DICTIONARY test_01191.t; -- {serverError INCORRECT_QUERY} DROP TABLE test_01191.t; -DROP DATABASE IF EXISTS dummy_db; CREATE DATABASE dummy_db ENGINE=Atomic; RENAME DICTIONARY test_01191.dict TO dummy_db.dict1; RENAME DICTIONARY dummy_db.dict1 TO test_01191.dict; diff --git a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh index 31e2cc395aa..5cdd6057050 100755 --- a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh +++ b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel +# Tags: zookeeper, no-parallel, no-fasttest +# no-fasttest: Waiting for failed mutations is slow: https://github.com/ClickHouse/ClickHouse/issues/67936 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql b/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql index b7cbfc92c26..a2e4804f12e 100644 --- a/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql +++ b/tests/queries/0_stateless/01456_modify_column_type_via_add_drop_update.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS tbl; CREATE TABLE tbl(a String, b UInt32, c Float64, d Int64, e UInt8) ENGINE=MergeTree ORDER BY tuple(); INSERT INTO tbl SELECT number, number * 2, number * 3, number * 4, number * 5 FROM system.numbers LIMIT 10; diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index 06ec67ee82d..ec957dd7a02 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -79,16 +79,3 @@ iPhone 900 Smartphone 500 500 Kindle Fire 150 Tablet 150 350 Samsung Galaxy Tab 200 Tablet 175 350 iPad 700 Tablet 350 350 ----- Q8 ---- -Lenovo Thinkpad Laptop 700 1 0 -Sony VAIO Laptop 700 1 0 -Dell Vostro Laptop 800 3 0.6666666666666666 -HP Elite Laptop 1200 4 1 -Microsoft Lumia Smartphone 200 1 0 -HTC One Smartphone 400 2 0.3333333333333333 -Nexus Smartphone 500 3 0.6666666666666666 -iPhone Smartphone 900 4 1 -Kindle Fire Tablet 150 1 0 -Samsung Galaxy Tab Tablet 200 2 0.5 -iPad Tablet 700 3 1 -Others Unknow 200 1 0 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index a660fcca7b2..c6bb23bc7cf 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -101,26 +101,7 @@ SELECT FROM products INNER JOIN product_groups USING (group_id)) t order by group_name, product_name, price; -select '---- Q8 ----'; -INSERT INTO product_groups VALUES (4, 'Unknow'); -INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); - -SELECT * -FROM -( - SELECT - product_name, - group_name, - price, - rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, - percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent - FROM products - INNER JOIN product_groups USING (group_id) -) AS t -ORDER BY - group_name ASC, - price ASC, - product_name ASC; - drop table product_groups; drop table products; + + diff --git a/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql b/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql index a7f04921f1f..c6bccde8590 100644 --- a/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql +++ b/tests/queries/0_stateless/01760_ddl_dictionary_use_current_database_name.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS ddl_dictonary_test_source; CREATE TABLE ddl_dictonary_test_source ( diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index 249c3218bcc..63403ce2893 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh index b4b0ee8a023..d70845f52eb 100755 --- a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh +++ b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: distributed +# Tags: distributed, no-fasttest +# no-fasttest: Slow wait and retries CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02099_tsv_raw_format_1.sh b/tests/queries/0_stateless/02099_tsv_raw_format_1.sh index a3468f46ca0..bd1f8731717 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format_1.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format_1.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash # Tags: long +# This test was split in two due to long runtimes in sanitizers. +# The other part is 02099_tsv_raw_format_2.sh. + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02099_tsv_raw_format_2.sh b/tests/queries/0_stateless/02099_tsv_raw_format_2.sh index d6034a0616f..9f57eea42f2 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format_2.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format_2.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash # Tags: long +# This test was split in two due to long runtimes in sanitizers. +# The other part is 02099_tsv_raw_format_1.sh. +# CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh 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 cfae4fee6c2..32e8b2f4312 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -52,6 +52,8 @@ CREATE TABLE system.clusters `database_shard_name` String, `database_replica_name` String, `is_active` Nullable(UInt8), + `replication_lag` Nullable(UInt32), + `recovery_time` Nullable(UInt64), `name` String ALIAS cluster ) ENGINE = SystemClusters diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference index 0e74c0a083e..0bb8966cbe4 100644 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference @@ -1,2 +1,2 @@ -CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\') -CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\') +CREATE TABLE default.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') +CREATE TABLE foo.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh index 3a95e59416a..934d87616ac 100755 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh @@ -4,5 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table' -$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table' +$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' +$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect deleted file mode 100755 index add977c4fce..00000000000 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/expect -f - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -if {[info exists env(CLICKHOUSE_TMP)]} { - set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) -} else { - set CLICKHOUSE_TMP "." -} -exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 - -log_user 0 -set timeout 20 -match_max 100000 - -expect_after { - -i $any_spawn_id eof { exp_continue } - -i $any_spawn_id timeout { exit 1 } -} - -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion" - -expect ":) " -send -- "insert into table function null() format TSV some trash here 123 \n 456\r" -expect "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -expect ":) " - -send -- "" -expect eof diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python new file mode 100644 index 00000000000..4c2df9556a1 --- /dev/null +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.python @@ -0,0 +1,95 @@ +import pty +import os +import shlex +import time +import multiprocessing + +COMPLETION_TIMEOUT_SECONDS = 30 +DEBUG_LOG = os.path.join( + os.environ["CLICKHOUSE_TMP"], + os.path.basename(os.path.abspath(__file__)).strip(".python") + ".debuglog", +) + +STATE_MAP = { + -1: "process did not start", + 0: "all good", + 1: "process started and said ':)'", + 2: "prompt search was started", + 3: "prompt is missing", +} + + +def run_with_timeout(func, args, timeout): + for _ in range(5): + state = multiprocessing.Value("i", -1) + process = multiprocessing.Process( + target=func, args=args, kwargs={"state": state} + ) + process.start() + process.join(timeout) + + if state.value in (0, 3): + return + + if process.is_alive(): + process.terminate() + + if state.value == -1: + continue + + print(f"Timeout, state: {STATE_MAP[state.value]}") + return + + +def expect(text, master, debug_log_fd, output=""): + while not text in output: + output_b = os.read(master, 4096) + output += output_b.decode() + debug_log_fd.write(repr(output_b) + "\n") + debug_log_fd.flush() + + return output + + +def test_completion(program, argv, prompt, state=None): + shell_pid, master = pty.fork() + if shell_pid == 0: + os.execv(program, argv) + else: + try: + debug_log_fd = open(DEBUG_LOG, "a") + + expect(":)", master, debug_log_fd) + + state.value = 1 + os.write(master, bytes(prompt.encode())) + expect(prompt[:-10], master, debug_log_fd) + + time.sleep(0.01) + os.write(master, b"\r") + state.value = 2 + + output = expect("CANNOT_PARSE_INPUT_ASSERTION_FAILED", master, debug_log_fd) + expect(":)", master, debug_log_fd, output) + + print("OK") + state.value = 0 + finally: + os.close(master) + debug_log_fd.close() + + +if __name__ == "__main__": + clickhouse_local = os.environ["CLICKHOUSE_LOCAL"] + args = shlex.split(clickhouse_local) + args.append("--disable_suggestion") + args.append("--highlight=0") + run_with_timeout( + test_completion, + [ + args[0], + args, + "insert into table function null() format TSV some trash here 123 \n 456", + ], + COMPLETION_TIMEOUT_SECONDS, + ) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference index e69de29bb2d..d86bac9de59 100644 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index dca8dae22c3..1548bef857f 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-asan, no-msan, no-tsan, no-debug +# Tags: no-random-settings, no-asan, no-msan, no-tsan, no-debug, no-fasttest +# no-fasttest: The test runs for 40 seconds # shellcheck disable=SC2009 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index ba652013a57..1bc7d4cd1d3 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-ordinary-database +# Tags: no-random-settings, no-ordinary-database, no-fasttest +# no-fasttest: The test is slow (too many small blocks) # shellcheck disable=SC2009 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh b/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh index 4d9048354a1..f716a6aa779 100755 --- a/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh +++ b/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh @@ -1,5 +1,8 @@ #!/usr/bin/env bash +# Tags: no-fasttest +# no-fasttest: Slow timeouts + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index c6bf298f944..abe99398a56 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -1,8 +1,9 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # no-parallel: This test is not parallel because when we execute system-wide SYSTEM DROP REPLICA, # other tests might shut down the storage in parallel and the test will fail. +# no-fasttest: It has several tests with timeouts for inactive replicas CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh index a3e428e75c8..509cd03f6c2 100755 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh @@ -13,13 +13,16 @@ ${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists b ${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();" ${CLICKHOUSE_CLIENT} --query="create table data_02572 (key Int) engine=Memory();" -${CLICKHOUSE_CLIENT} --query="create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, 3, 3, 1, 1e9, 1, 1e9);" +${CLICKHOUSE_CLIENT} --query="create table buffer_02572 (key Int) engine=Buffer(currentDatabase(), data_02572, 1, 8, 8, 1, 1e9, 1, 1e9);" ${CLICKHOUSE_CLIENT} --query="create materialized view mv_02572 to copy_02572 as select * from data_02572;" +start=$(date +%s) ${CLICKHOUSE_CLIENT} --query="insert into buffer_02572 values (1);" -# ensure that the flush was not direct -${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" +if [ $(( $(date +%s) - start )) -gt 6 ]; then # clickhouse test cluster is overloaded, will skip + # ensure that the flush was not direct + ${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" +fi # we cannot use OPTIMIZE, this will attach query context, so let's wait for _ in {1..100}; do diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql index ea1452fc372..ad8bef7fbb7 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks.sql @@ -1,4 +1,5 @@ --- Tags: no-tsan, no-asan, no-ubsan, no-msan +-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-fasttest +-- no-fasttest: Slow test -- no sanitizers: too slow sometimes DROP TABLE IF EXISTS 02581_trips; diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python new file mode 100644 index 00000000000..eb0cab9d56f --- /dev/null +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.python @@ -0,0 +1,63 @@ +#!/usr/bin/env python3 + +import os +import sys +from threading import Thread +from queue import Queue + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from pure_http_client import ClickHouseClient + + +client = ClickHouseClient() + + +client.query("DROP TABLE IF EXISTS test SYNC") +client.query( + """ +CREATE TABLE test +( + c_id String, + p_id String, + d UInt32, +) +Engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') +ORDER BY (c_id, p_id) +""" +) + + +def attempt_mutation(q): + try: + client.query( + "ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync=2", + with_retries=False, + ) + except ValueError as e: + assert "BAD_ARGUMENTS" in str(e) + q.put("OK") + + +client.query("INSERT INTO test SELECT '1', '11', '111' FROM numbers(5)") +client.query("SYSTEM ENABLE FAILPOINT infinite_sleep") +client.query( + "ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1 SETTINGS mutations_sync=0" +) +client.query("ALTER TABLE test ADD COLUMN x UInt32 default 0 SETTINGS mutations_sync=0") +client.query("ALTER TABLE test UPDATE d = x + 1 where 1 SETTINGS mutations_sync=0") + +q = Queue() +t = Thread(target=attempt_mutation, args=(q,)) +t.start() +t.join() +assert not q.empty() +assert q.get() == "OK" + +client.query("SYSTEM DISABLE FAILPOINT infinite_sleep") + +client.query("ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync=2") +client.query("ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync=2") +client.query("SELECT * from test format Null") +client.query("DROP TABLE test") diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.reference b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh new file mode 100755 index 00000000000..5be04d99204 --- /dev/null +++ b/tests/queries/0_stateless/02597_column_update_tricky_expression_and_replication.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: zookeeper, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02597_column_update_tricky_expression_and_replication.python diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql b/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql deleted file mode 100644 index 34f88b19b7e..00000000000 --- a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql +++ /dev/null @@ -1,28 +0,0 @@ -CREATE TABLE test ( - `c_id` String, - `p_id` String, - `d` UInt32 -) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') -ORDER BY (c_id, p_id); - -INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); - -INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); - -set mutations_sync=0; - -ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1; - -ALTER TABLE test ADD COLUMN x UInt32 default 0; -ALTER TABLE test UPDATE d = x + 1 where 1; -ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError BAD_ARGUMENTS} - -ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; - -ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; - -select * from test format Null; - -DROP TABLE test; - diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index 6f43c1ae869..03e0f363d71 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-object-storage, no-random-settings, no-random-merge-tree-settings +# Tags: no-object-storage, no-random-settings, no-random-merge-tree-settings, no-fasttest +# no-fasttest: The test is slow CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql index f6bccc99977..98aeac36243 100644 --- a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: Timeouts are slow create table dist as system.one engine=Distributed(test_shard_localhost, system, one); select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=0, use_hedged_requests=1 format Null; select sleep(8) from dist settings function_sleep_max_microseconds_per_block=8e9, prefer_localhost_replica=0, receive_timeout=7, async_socket_for_remote=1, use_hedged_requests=0 format Null; diff --git a/tests/queries/0_stateless/02864_statistics_bugs.reference b/tests/queries/0_stateless/02864_statistics_bugs.reference new file mode 100644 index 00000000000..a7eeae9def6 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_bugs.reference @@ -0,0 +1,3 @@ +10 +11 +0 diff --git a/tests/queries/0_stateless/02864_statistics_bugs.sql b/tests/queries/0_stateless/02864_statistics_bugs.sql new file mode 100644 index 00000000000..01bbe221b0f --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_bugs.sql @@ -0,0 +1,27 @@ +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < '10'; +DROP TABLE bug_67742; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Int32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < '10.5'; -- { serverError TYPE_MISMATCH } +DROP TABLE bug_67742; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Int32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < 10.5; +DROP TABLE bug_67742; + +DROP TABLE IF EXISTS bug_67742; +CREATE TABLE bug_67742 (a Int16 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); +INSERT INTO bug_67742 SELECT number FROM system.numbers LIMIT 10000; +SELECT count(*) FROM bug_67742 WHERE a < '9999999999999999999999999'; +DROP TABLE bug_67742; diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index a03343c8cb3..39e7aad87e0 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -27,6 +27,7 @@ OK OK 100 100 +OK ===== TestGrants ===== OK OK diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index cc4e76a9ed9..fadbbff7f34 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -199,6 +199,8 @@ ${CLICKHOUSE_CLIENT} --user $user2 --query "INSERT INTO source SELECT * FROM gen ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM destination1" ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM destination2" +(( $(${CLICKHOUSE_CLIENT} --query "ALTER TABLE test_table MODIFY SQL SECURITY INVOKER" 2>&1 | grep -c "is not supported") >= 1 )) && echo "OK" || echo "UNEXPECTED" + echo "===== TestGrants =====" ${CLICKHOUSE_CLIENT} --query "GRANT CREATE ON *.* TO $user1" ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_table TO $user1, $user2" diff --git a/tests/queries/0_stateless/02952_conjunction_optimization.reference b/tests/queries/0_stateless/02952_conjunction_optimization.reference index eeadfaae21d..8af0abefd3a 100644 --- a/tests/queries/0_stateless/02952_conjunction_optimization.reference +++ b/tests/queries/0_stateless/02952_conjunction_optimization.reference @@ -32,7 +32,7 @@ QUERY id: 0 FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: Bool ARGUMENTS LIST id: 6, nodes: 2 - CONSTANT id: 7, constant_value: UInt64_1, constant_value_type: Bool + CONSTANT id: 7, constant_value: Bool_1, constant_value_type: Bool FUNCTION id: 8, function_name: notIn, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 9, nodes: 2 diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index 471b48e0f68..814fe1a0f2c 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -1,6 +1,4 @@ #!/bin/bash -# Tags: no-parallel - CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -8,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) FILE_NAME="data.tmp" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -touch $FORM_DATA +touch $FORM_DATA # Simple tests echo -ne "col1=42&col2=Hello%2C%20World%21" > $FORM_DATA diff --git a/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 b/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 index 1b9be79dbe4..65601cd39be 100644 --- a/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 +++ b/tests/queries/0_stateless/02994_merge_tree_mutations_cleanup.sql.j2 @@ -1,3 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: Slow wait drop table if exists data_rmt; drop table if exists data_mt; diff --git a/tests/queries/0_stateless/02995_forget_partition.sh b/tests/queries/0_stateless/02995_forget_partition.sh index 6fa0b96e90d..e9d4590cce6 100755 --- a/tests/queries/0_stateless/02995_forget_partition.sh +++ b/tests/queries/0_stateless/02995_forget_partition.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database +# Tags: zookeeper, no-replicated-database, no-fasttest +# no-fasttest: Slow wait CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02998_primary_key_skip_columns.sql b/tests/queries/0_stateless/02998_primary_key_skip_columns.sql index ee558996b52..1abe692a7a4 100644 --- a/tests/queries/0_stateless/02998_primary_key_skip_columns.sql +++ b/tests/queries/0_stateless/02998_primary_key_skip_columns.sql @@ -1,4 +1,5 @@ --- Tags: no-asan, no-tsan, no-msan, no-ubsan, no-random-settings, no-debug +-- Tags: no-asan, no-tsan, no-msan, no-ubsan, no-random-settings, no-debug, no-fasttest +-- no-fasttest: Low index granularity and too many parts makes the test slow DROP TABLE IF EXISTS test; diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference new file mode 100644 index 00000000000..202e4557a33 --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference @@ -0,0 +1,2 @@ +query +String diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql new file mode 100644 index 00000000000..b26096f7f0e --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql @@ -0,0 +1,18 @@ + +SELECT * FROM fuzzQuery('SELECT 1', 500, 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; + +SELECT * FROM fuzzQuery('SELECT * +FROM ( + SELECT + ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, + count() + FROM numbers(3) + GROUP BY item_id WITH TOTALS +) AS l FULL JOIN ( + SELECT + ([toString((number % 2) * 2)] :: Array(String)) AS item_id + FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3; +', 500, 8956) LIMIT 10 FORMAT NULL; diff --git a/tests/queries/0_stateless/03037_precent_rank.reference b/tests/queries/0_stateless/03037_precent_rank.reference new file mode 100644 index 00000000000..6a23f3884cd --- /dev/null +++ b/tests/queries/0_stateless/03037_precent_rank.reference @@ -0,0 +1,22 @@ +Lenovo Thinkpad Laptop 700 1 0 +Sony VAIO Laptop 700 1 0 +Dell Vostro Laptop 800 3 0.6666666666666666 +HP Elite Laptop 1200 4 1 +Microsoft Lumia Smartphone 200 1 0 +HTC One Smartphone 400 2 0.3333333333333333 +Nexus Smartphone 500 3 0.6666666666666666 +iPhone Smartphone 900 4 1 +Kindle Fire Tablet 150 1 0 +Samsung Galaxy Tab Tablet 200 2 0.5 +iPad Tablet 700 3 1 +Others Unknow 200 1 0 +0 1 0 +1 2 1 +2 3 2 +3 4 3 +4 5 4 +5 6 5 +6 7 6 +7 8 7 +8 9 8 +9 10 9 diff --git a/tests/queries/0_stateless/03037_precent_rank.sql b/tests/queries/0_stateless/03037_precent_rank.sql new file mode 100644 index 00000000000..b0f83fa3340 --- /dev/null +++ b/tests/queries/0_stateless/03037_precent_rank.sql @@ -0,0 +1,52 @@ +drop table if exists product_groups; +drop table if exists products; + +CREATE TABLE product_groups ( + group_id Int64, + group_name String +) Engine = Memory; + + +CREATE TABLE products ( + product_id Int64, + product_name String, + price DECIMAL(11, 2), + group_id Int64 +) Engine = Memory; + +INSERT INTO product_groups VALUES (1, 'Smartphone'),(2, 'Laptop'),(3, 'Tablet'); + +INSERT INTO products (product_id,product_name, group_id,price) VALUES (1, 'Microsoft Lumia', 1, 200), (2, 'HTC One', 1, 400), (3, 'Nexus', 1, 500), (4, 'iPhone', 1, 900),(5, 'HP Elite', 2, 1200),(6, 'Lenovo Thinkpad', 2, 700),(7, 'Sony VAIO', 2, 700),(8, 'Dell Vostro', 2, 800),(9, 'iPad', 3, 700),(10, 'Kindle Fire', 3, 150),(11, 'Samsung Galaxy Tab', 3, 200); + +INSERT INTO product_groups VALUES (4, 'Unknow'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); + +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +ORDER BY + group_name ASC, + price ASC, + product_name ASC; + +drop table product_groups; +drop table products; + +select number, row_number, cast(percent_rank * 10000 as Int32) as percent_rank +from ( + select number, row_number() over () as row_number, percent_rank() over (order by number) as percent_rank + from numbers(10000) + order by number + limit 10 +) +settings max_block_size=100; + diff --git a/tests/queries/0_stateless/03172_system_detached_tables.reference b/tests/queries/0_stateless/03172_system_detached_tables.reference index 83d1ff13942..2fce89e5980 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.reference +++ b/tests/queries/0_stateless/03172_system_detached_tables.reference @@ -1,11 +1,11 @@ database atomic tests -test03172_system_detached_tables test_table 0 -test03172_system_detached_tables test_table_perm 1 -test03172_system_detached_tables test_table 0 -test03172_system_detached_tables test_table_perm 1 -test03172_system_detached_tables test_table 0 +default_atomic test_table 0 +default_atomic test_table_perm 1 +default_atomic test_table 0 +default_atomic test_table_perm 1 +default_atomic test_table 0 ----------------------- database lazy tests -before attach test03172_system_detached_tables_lazy test_table 0 -before attach test03172_system_detached_tables_lazy test_table_perm 1 +before attach default_lazy test_table 0 +before attach default_lazy test_table_perm 1 DROP TABLE diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sh b/tests/queries/0_stateless/03172_system_detached_tables.sh new file mode 100755 index 00000000000..47775abcc45 --- /dev/null +++ b/tests/queries/0_stateless/03172_system_detached_tables.sh @@ -0,0 +1,68 @@ +#!/bin/bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATABASE_ATOMIC="${CLICKHOUSE_DATABASE}_atomic" +DATABASE_LAZY="${CLICKHOUSE_DATABASE}_lazy" + +$CLICKHOUSE_CLIENT --multiquery " + +SELECT 'database atomic tests'; +DROP DATABASE IF EXISTS ${DATABASE_ATOMIC}; +CREATE DATABASE IF NOT EXISTS ${DATABASE_ATOMIC} ENGINE=Atomic; + +CREATE TABLE ${DATABASE_ATOMIC}.test_table (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +DETACH TABLE ${DATABASE_ATOMIC}.test_table; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +ATTACH TABLE ${DATABASE_ATOMIC}.test_table; + +CREATE TABLE ${DATABASE_ATOMIC}.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +DETACH TABLE ${DATABASE_ATOMIC}.test_table_perm PERMANENTLY; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +DETACH TABLE ${DATABASE_ATOMIC}.test_table SYNC; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}'; + +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_ATOMIC}' AND table='test_table'; + +DROP DATABASE ${DATABASE_ATOMIC} SYNC; + +" + +$CLICKHOUSE_CLIENT --multiquery " + +SELECT '-----------------------'; +SELECT 'database lazy tests'; + +DROP DATABASE IF EXISTS ${DATABASE_LAZY}; +CREATE DATABASE ${DATABASE_LAZY} Engine=Lazy(10); + +CREATE TABLE ${DATABASE_LAZY}.test_table (number UInt64) engine=Log; +INSERT INTO ${DATABASE_LAZY}.test_table SELECT * FROM numbers(100); +DETACH TABLE ${DATABASE_LAZY}.test_table; + +CREATE TABLE ${DATABASE_LAZY}.test_table_perm (number UInt64) engine=Log; +INSERT INTO ${DATABASE_LAZY}.test_table_perm SELECT * FROM numbers(100); +DETACH table ${DATABASE_LAZY}.test_table_perm PERMANENTLY; + +SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_LAZY}'; + +ATTACH TABLE ${DATABASE_LAZY}.test_table; +ATTACH TABLE ${DATABASE_LAZY}.test_table_perm; + +SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='${DATABASE_LAZY}'; + +SELECT 'DROP TABLE'; +DROP TABLE ${DATABASE_LAZY}.test_table SYNC; +DROP TABLE ${DATABASE_LAZY}.test_table_perm SYNC; + +DROP DATABASE ${DATABASE_LAZY} SYNC; + +" diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql deleted file mode 100644 index 1a3c2d7cc0f..00000000000 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ /dev/null @@ -1,53 +0,0 @@ --- Tags: no-parallel - -SELECT 'database atomic tests'; -DROP DATABASE IF EXISTS test03172_system_detached_tables; -CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables ENGINE=Atomic; - -CREATE TABLE test03172_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -DETACH TABLE test03172_system_detached_tables.test_table; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -ATTACH TABLE test03172_system_detached_tables.test_table; - -CREATE TABLE test03172_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -DETACH TABLE test03172_system_detached_tables.test_table_perm PERMANENTLY; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -DETACH TABLE test03172_system_detached_tables.test_table SYNC; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; - -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables' AND table='test_table'; - -DROP DATABASE test03172_system_detached_tables SYNC; - -SELECT '-----------------------'; -SELECT 'database lazy tests'; - -DROP DATABASE IF EXISTS test03172_system_detached_tables_lazy; -CREATE DATABASE test03172_system_detached_tables_lazy Engine=Lazy(10); - -CREATE TABLE test03172_system_detached_tables_lazy.test_table (number UInt64) engine=Log; -INSERT INTO test03172_system_detached_tables_lazy.test_table SELECT * FROM numbers(100); -DETACH TABLE test03172_system_detached_tables_lazy.test_table; - -CREATE TABLE test03172_system_detached_tables_lazy.test_table_perm (number UInt64) engine=Log; -INSERT INTO test03172_system_detached_tables_lazy.test_table_perm SELECT * FROM numbers(100); -DETACH table test03172_system_detached_tables_lazy.test_table_perm PERMANENTLY; - -SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; - -ATTACH TABLE test03172_system_detached_tables_lazy.test_table; -ATTACH TABLE test03172_system_detached_tables_lazy.test_table_perm; - -SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; - -SELECT 'DROP TABLE'; -DROP TABLE test03172_system_detached_tables_lazy.test_table SYNC; -DROP TABLE test03172_system_detached_tables_lazy.test_table_perm SYNC; - -DROP DATABASE test03172_system_detached_tables_lazy SYNC; diff --git a/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference b/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference deleted file mode 100644 index 1975397394b..00000000000 --- a/tests/queries/0_stateless/03199_atomic_clickhouse_local.reference +++ /dev/null @@ -1,6 +0,0 @@ -123 -Hello -['Hello','world'] -Hello -Hello -['Hello','world'] diff --git a/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh b/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh deleted file mode 100755 index edaa83b8f95..00000000000 --- a/tests/queries/0_stateless/03199_atomic_clickhouse_local.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_LOCAL} -n " -CREATE TABLE test (x UInt8) ORDER BY x; -INSERT INTO test VALUES (123); -SELECT * FROM test; -CREATE OR REPLACE TABLE test (s String) ORDER BY s; -INSERT INTO test VALUES ('Hello'); -SELECT * FROM test; -RENAME TABLE test TO test2; -CREATE OR REPLACE TABLE test (s Array(String)) ORDER BY s; -INSERT INTO test VALUES (['Hello', 'world']); -SELECT * FROM test; -SELECT * FROM test2; -EXCHANGE TABLES test AND test2; -SELECT * FROM test; -SELECT * FROM test2; -DROP TABLE test; -DROP TABLE test2; -" diff --git a/tests/queries/0_stateless/03203_drop_detached_partition_all.reference b/tests/queries/0_stateless/03203_drop_detached_partition_all.reference new file mode 100644 index 00000000000..c0f52d1d898 --- /dev/null +++ b/tests/queries/0_stateless/03203_drop_detached_partition_all.reference @@ -0,0 +1,5 @@ +1 1 +2 2 +3 3 +3 +0 diff --git a/tests/queries/0_stateless/03203_drop_detached_partition_all.sql b/tests/queries/0_stateless/03203_drop_detached_partition_all.sql new file mode 100644 index 00000000000..e29eb4ae36b --- /dev/null +++ b/tests/queries/0_stateless/03203_drop_detached_partition_all.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS t_03203; +CREATE TABLE t_03203 (p UInt64, v UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY v; +INSERT INTO t_03203 VALUES (1, 1), (2, 2), (3, 3); +SELECT * FROM t_03203 ORDER BY p, v; +ALTER TABLE t_03203 DETACH PARTITION ALL; +SELECT count() FROM system.detached_parts WHERE database = currentDatabase() AND table = 't_03203'; +ALTER TABLE t_03203 DROP DETACHED PARTITION ALL SETTINGS allow_drop_detached = 1; +SELECT count() FROM system.detached_parts WHERE database = currentDatabase() AND table = 't_03203'; diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.reference b/tests/queries/0_stateless/03206_replication_lag_metric.reference new file mode 100644 index 00000000000..02f4a7264b1 --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.reference @@ -0,0 +1,4 @@ +0 +2 +0 +2 diff --git a/tests/queries/0_stateless/03206_replication_lag_metric.sql b/tests/queries/0_stateless/03206_replication_lag_metric.sql new file mode 100644 index 00000000000..998c332a11c --- /dev/null +++ b/tests/queries/0_stateless/03206_replication_lag_metric.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel + +CREATE DATABASE rdb1 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica1'); +CREATE DATABASE rdb2 ENGINE = Replicated('/test/test_replication_lag_metric', 'shard1', 'replica2'); + +SET distributed_ddl_task_timeout = 0; +CREATE TABLE rdb1.t (id UInt32) ENGINE = ReplicatedMergeTree ORDER BY id; +SELECT replication_lag FROM system.clusters WHERE cluster IN ('rdb1', 'rdb2') ORDER BY cluster ASC, replica_num ASC; + +DROP DATABASE rdb1; +DROP DATABASE rdb2; diff --git a/tests/queries/0_stateless/03215_view_with_recursive.reference b/tests/queries/0_stateless/03215_view_with_recursive.reference new file mode 100644 index 00000000000..c3ca8065a70 --- /dev/null +++ b/tests/queries/0_stateless/03215_view_with_recursive.reference @@ -0,0 +1,2 @@ +5050 +8 diff --git a/tests/queries/0_stateless/03215_view_with_recursive.sql b/tests/queries/0_stateless/03215_view_with_recursive.sql new file mode 100644 index 00000000000..5d93ccc5438 --- /dev/null +++ b/tests/queries/0_stateless/03215_view_with_recursive.sql @@ -0,0 +1,43 @@ +SET allow_experimental_analyzer = 1; + +CREATE VIEW 03215_test_v +AS WITH RECURSIVE test_table AS + ( + SELECT 1 AS number + UNION ALL + SELECT number + 1 + FROM test_table + WHERE number < 100 + ) +SELECT sum(number) +FROM test_table; + +SELECT * FROM 03215_test_v; + +CREATE VIEW 03215_multi_v +AS WITH RECURSIVE + task AS + ( + SELECT + number AS task_id, + number - 1 AS parent_id + FROM numbers(10) + ), + rtq AS + ( + SELECT + task_id, + parent_id + FROM task AS t + WHERE t.parent_id = 1 + UNION ALL + SELECT + t.task_id, + t.parent_id + FROM task AS t, rtq AS r + WHERE t.parent_id = r.task_id + ) +SELECT count() +FROM rtq; + +SELECT * FROM 03215_multi_v; diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index a31a91e0550..c3c4109ce5b 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -19,9 +19,14 @@ class ClickHouseClient: self.host = host def query( - self, query, connection_timeout=500, settings=dict(), binary_result=False + self, + query, + connection_timeout=500, + settings=dict(), + binary_result=False, + with_retries=True, ): - NUMBER_OF_TRIES = 30 + NUMBER_OF_TRIES = 30 if with_retries else 1 DELAY = 10 params = { @@ -40,7 +45,8 @@ class ClickHouseClient: if r.status_code == 200: return r.content if binary_result else r.text else: - print("ATTENTION: try #%d failed" % i) + if with_retries: + print("ATTENTION: try #%d failed" % i) if i != (NUMBER_OF_TRIES - 1): print(query) print(r.text) diff --git a/tests/tsan_ignorelist.txt b/tests/tsan_ignorelist.txt index 96bf6e4251f..2a31fc9bc15 100644 --- a/tests/tsan_ignorelist.txt +++ b/tests/tsan_ignorelist.txt @@ -5,11 +5,9 @@ # # Caveats for generic entry "fun": # - does not work for __attribute__((__always_inline__)) +# - and may not work for functions that had been inlined # - requires asterisk at the beginning *and* end for static functions # [thread] # https://github.com/ClickHouse/ClickHouse/issues/55629 fun:rd_kafka_broker_set_nodename -# https://github.com/ClickHouse/ClickHouse/issues/60443 -fun:*rd_avg_calc* -fun:*rd_avg_rollover* diff --git a/tests/ubsan_ignorelist.txt b/tests/ubsan_ignorelist.txt index 57d6598afa6..b75819b3f4b 100644 --- a/tests/ubsan_ignorelist.txt +++ b/tests/ubsan_ignorelist.txt @@ -9,6 +9,7 @@ # # Caveats for generic entry "fun": # - does not work for __attribute__((__always_inline__)) +# - and may not work for functions that had been inlined # - requires asterisk at the beginning *and* end for static functions # [undefined] diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 37094a1a088..469c0a5fc5f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1674,6 +1674,7 @@ fuzzQuery fuzzer fuzzers gRPC +gaugehistogram gccMurmurHash gcem generateRandom @@ -2558,6 +2559,7 @@ startsWithUTF startswith statbox stateful +stateset stddev stddevPop stddevPopStable @@ -2689,6 +2691,10 @@ themself threadpool throwIf timeDiff +TimeSeries +timeSeriesData +timeSeriesMetrics +timeSeriesTags timeSlot timeSlots timeZone diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e410f31ca5a..f46353277e2 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06