diff --git a/.clang-tidy b/.clang-tidy index 896052915f7..de19059d09e 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -37,7 +37,6 @@ Checks: [ '-cert-oop54-cpp', '-cert-oop57-cpp', - '-clang-analyzer-optin.core.EnumCastOutOfRange', # https://github.com/abseil/abseil-cpp/issues/1667 '-clang-analyzer-optin.performance.Padding', '-clang-analyzer-unix.Malloc', diff --git a/.editorconfig b/.editorconfig index 8ac06debb5a..8ecaf9b0267 100644 --- a/.editorconfig +++ b/.editorconfig @@ -19,3 +19,7 @@ charset = utf-8 indent_style = space indent_size = 4 trim_trailing_whitespace = true + +# Some SQL results have trailing whitespace which is removed by IDEs +[tests/queries/**.reference] +trim_trailing_whitespace = false diff --git a/.github/ISSUE_TEMPLATE/10_question.md b/.github/ISSUE_TEMPLATE/10_question.md deleted file mode 100644 index 08a05a844e0..00000000000 --- a/.github/ISSUE_TEMPLATE/10_question.md +++ /dev/null @@ -1,20 +0,0 @@ ---- -name: Question -about: Ask a question about ClickHouse -title: '' -labels: question -assignees: '' - ---- - -> Make sure to check documentation https://clickhouse.com/docs/en/ first. If the question is concise and probably has a short answer, asking it in [community Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) is probably the fastest way to find the answer. For more complicated questions, consider asking them on StackOverflow with "clickhouse" tag https://stackoverflow.com/questions/tagged/clickhouse - -> If you still prefer GitHub issues, remove all this text and ask your question here. - -**Company or project name** - -Put your company name or project description here - -**Question** - -Your question diff --git a/.github/ISSUE_TEMPLATE/10_question.yaml b/.github/ISSUE_TEMPLATE/10_question.yaml new file mode 100644 index 00000000000..39d4c27807a --- /dev/null +++ b/.github/ISSUE_TEMPLATE/10_question.yaml @@ -0,0 +1,20 @@ +name: Question +description: Ask a question about ClickHouse +labels: ["question"] +body: + - type: markdown + attributes: + value: | + > Make sure to check documentation https://clickhouse.com/docs/en/ first. If the question is concise and probably has a short answer, asking it in [community Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) is probably the fastest way to find the answer. For more complicated questions, consider asking them on StackOverflow with "clickhouse" tag https://stackoverflow.com/questions/tagged/clickhouse + - type: textarea + attributes: + label: Company or project name + description: Put your company name or project description here. + validations: + required: false + - type: textarea + attributes: + label: Question + description: Please put your question here. + validations: + required: true diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 51a1a6e2df8..d9f9e9d6c8b 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -48,21 +48,18 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Allow: Stateful tests - [ ] Allow: Integration Tests - [ ] Allow: Performance tests -- [ ] Allow: All NOT Required Checks +- [ ] Allow: All Builds - [ ] Allow: batch 1, 2 for multi-batch jobs - [ ] Allow: batch 3, 4, 5, 6 for multi-batch jobs --- - [ ] Exclude: Style check - [ ] Exclude: Fast test -- [ ] Exclude: Integration Tests -- [ ] Exclude: Stateless tests -- [ ] Exclude: Stateful tests -- [ ] Exclude: Performance tests - [ ] Exclude: All with ASAN -- [ ] Exclude: All with Aarch64 - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage +- [ ] Exclude: All with aarch64, release, debug --- - [ ] Do not test +- [ ] Woolen Wolfdog - [ ] Upload binaries for special builds - [ ] Disable merge-commit - [ ] Disable CI cache diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index b0380b939bb..e1980ec9ef2 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -70,7 +70,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Compatibility check (amd64) + test_name: Compatibility check (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckAarch64: @@ -194,7 +194,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (amd64) + test_name: Install packages (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -204,7 +204,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (arm64) + test_name: Install packages (aarch64) runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -273,5 +273,5 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} python3 merge_pr.py diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index c2a893a8e99..384bf6825f9 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -104,8 +104,8 @@ jobs: with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} - # stage for jobs that do not prohibit merge Tests_3: + # Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there. needs: [RunConfig, Builds_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} uses: ./.github/workflows/reusable_test_stage.yml @@ -114,25 +114,16 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should be run even if Builds_1/2 failed - put them separately in wf (not in Tests_1/2) - Builds_1_Report: + # Reports should run even if Builds_1/2 fail - run them separately, not in Tests_1/2/3 + Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: [RunConfig, Builds_1] + needs: [RunConfig, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} - Builds_2_Report: - # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: [RunConfig, Builds_2] - uses: ./.github/workflows/reusable_test.yml - with: - test_name: ClickHouse special build check - runner_type: style-checker-aarch64 - data: ${{ needs.RunConfig.outputs.data }} MarkReleaseReady: if: ${{ !failure() && !cancelled() }} @@ -164,7 +155,7 @@ jobs: FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -172,4 +163,4 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index c8b2452829b..cfa01b0e8f3 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -99,7 +99,7 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Tests_1] runs-on: [self-hosted, style-checker-aarch64] steps: @@ -112,4 +112,4 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py ${{ (contains(needs.*.result, 'failure') && github.event_name == 'merge_group') && '--pipeline-failure' || '' }} + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7d22554473e..84dd4222e36 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -126,16 +126,16 @@ jobs: with: stage: Builds_2 data: ${{ needs.RunConfig.outputs.data }} + # stage for running non-required checks without being blocked by required checks (Test_1) if corresponding settings is selected Tests_2: - needs: [RunConfig, Builds_2] + needs: [RunConfig, Builds_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }} uses: ./.github/workflows/reusable_test_stage.yml with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} - # stage for jobs that do not prohibit merge Tests_3: - needs: [RunConfig, Tests_1, Tests_2] + needs: [RunConfig, Builds_1, Tests_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} uses: ./.github/workflows/reusable_test_stage.yml with: @@ -143,29 +143,21 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should by run even if Builds_1/2 fail, so put them separately in wf (not in Tests_1/2) - Builds_1_Report: + # Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3) + Builds_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: [RunConfig, StyleCheck, Builds_1] + if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} + needs: [RunConfig, StyleCheck, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} - Builds_2_Report: - # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: [RunConfig, StyleCheck, Builds_2] - uses: ./.github/workflows/reusable_test.yml - with: - test_name: ClickHouse special build check - runner_type: style-checker-aarch64 - data: ${{ needs.RunConfig.outputs.data }} CheckReadyForMerge: if: ${{ !cancelled() && needs.StyleCheck.result == 'success' }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] + # Test_2 or Test_3 must not have jobs required for Mergeable check + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -181,7 +173,7 @@ jobs: # FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -191,7 +183,7 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} ############################################################################################# ###################################### JEPSEN TESTS ######################################### @@ -204,8 +196,7 @@ jobs: concurrency: group: jepsen if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse Keeper Jepsen') }} - # jepsen needs binary_release build which is in Builds_2 - needs: [RunConfig, Builds_2] + needs: [RunConfig, Builds_1] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse Keeper Jepsen diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 4d45c8d8d4b..45eb7431bb4 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -65,7 +65,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Compatibility check (amd64) + test_name: Compatibility check (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckAarch64: @@ -244,7 +244,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (amd64) + test_name: Install packages (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -254,7 +254,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (arm64) + test_name: Install packages (aarch64) runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -496,4 +496,4 @@ jobs: - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py + python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }} diff --git a/.github/workflows/reusable_test.yml b/.github/workflows/reusable_test.yml index e30ef863a86..c01dd8ca9d4 100644 --- a/.github/workflows/reusable_test.yml +++ b/.github/workflows/reusable_test.yml @@ -58,7 +58,7 @@ jobs: env: GITHUB_JOB_OVERRIDDEN: ${{inputs.test_name}}${{ fromJson(inputs.data).jobs_data.jobs_params[inputs.test_name].num_batches > 1 && format('-{0}',matrix.batch) || '' }} strategy: - fail-fast: false # we always wait for entire matrix + fail-fast: false # we always wait for the entire matrix matrix: batch: ${{ fromJson(inputs.data).jobs_data.jobs_params[inputs.test_name].batches }} steps: diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml index e4fc9f0b1d3..2aa7694bc41 100644 --- a/.github/workflows/tags_stable.yml +++ b/.github/workflows/tags_stable.yml @@ -46,9 +46,10 @@ jobs: ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv ./utils/list-versions/update-docker-version.sh GID=$(id -g "${UID}") - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 \ + # --network=host and CI=1 are required for the S3 access from a container + docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ --volume="${GITHUB_WORKSPACE}:/ClickHouse" clickhouse/style-test \ - /ClickHouse/utils/changelog/changelog.py -v --debug-helpers \ + /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ --gh-user-or-token="$GITHUB_TOKEN" --jobs=5 \ --output="/ClickHouse/docs/changelogs/${GITHUB_TAG}.md" "${GITHUB_TAG}" git add "./docs/changelogs/${GITHUB_TAG}.md" diff --git a/.gitignore b/.gitignore index db3f77d7d1e..4bc162c1b0f 100644 --- a/.gitignore +++ b/.gitignore @@ -21,6 +21,9 @@ *.stderr *.stdout +# llvm-xray logs +xray-log.* + /docs/build /docs/publish /docs/edit diff --git a/.gitmessage b/.gitmessage deleted file mode 100644 index 89ee7d35d23..00000000000 --- a/.gitmessage +++ /dev/null @@ -1,29 +0,0 @@ - - -### CI modificators (add a leading space to apply) ### - -## To avoid a merge commit in CI: -#no_merge_commit - -## To discard CI cache: -#no_ci_cache - -## To not test (only style check): -#do_not_test - -## To run specified set of tests in CI: -#ci_set_ -#ci_set_reduced -#ci_set_arm -#ci_set_integration -#ci_set_old_analyzer - -## To run specified job in CI: -#job_ -#job_stateless_tests_release -#job_package_debug -#job_integration_tests_asan - -## To run only specified batches for multi-batch job(s) -#batch_2 -#batch_1_2_3 diff --git a/.gitmodules b/.gitmodules index 28696428e8c..12d865307d8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -91,13 +91,13 @@ [submodule "contrib/aws"] path = contrib/aws url = https://github.com/ClickHouse/aws-sdk-cpp -[submodule "aws-c-event-stream"] +[submodule "contrib/aws-c-event-stream"] path = contrib/aws-c-event-stream url = https://github.com/awslabs/aws-c-event-stream -[submodule "aws-c-common"] +[submodule "contrib/aws-c-common"] path = contrib/aws-c-common url = https://github.com/awslabs/aws-c-common.git -[submodule "aws-checksums"] +[submodule "contrib/aws-checksums"] path = contrib/aws-checksums url = https://github.com/awslabs/aws-checksums [submodule "contrib/curl"] @@ -163,7 +163,7 @@ url = https://github.com/xz-mirror/xz [submodule "contrib/abseil-cpp"] path = contrib/abseil-cpp - url = https://github.com/abseil/abseil-cpp + url = https://github.com/ClickHouse/abseil-cpp.git [submodule "contrib/dragonbox"] path = contrib/dragonbox url = https://github.com/ClickHouse/dragonbox diff --git a/CHANGELOG.md b/CHANGELOG.md index a089e9e7491..4891b79e4c7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,7 +11,7 @@ ### ClickHouse release 24.5, 2024-05-30 #### Backward Incompatible Change -* Renamed "inverted indexes" to "full-text indexes" which is a less technical / more user-friendly name. This also changes internal table metadata and breaks tables with existing (experimental) inverted indexes. Please make to drop such indexes before upgrade and re-create them after upgrade. [#62884](https://github.com/ClickHouse/ClickHouse/pull/62884) ([Robert Schulze](https://github.com/rschu1ze)). +* Renamed "inverted indexes" to "full-text indexes" which is a less technical / more user-friendly name. This also changes internal table metadata and breaks tables with existing (experimental) inverted indexes. Please make sure to drop such indexes before upgrade and re-create them after upgrade. [#62884](https://github.com/ClickHouse/ClickHouse/pull/62884) ([Robert Schulze](https://github.com/rschu1ze)). * Usage of functions `neighbor`, `runningAccumulate`, `runningDifferenceStartingWithFirstValue`, `runningDifference` deprecated (because it is error-prone). Proper window functions should be used instead. To enable them back, set `allow_deprecated_error_prone_window_functions = 1` or set `compatibility = '24.4'` or lower. [#63132](https://github.com/ClickHouse/ClickHouse/pull/63132) ([Nikita Taranov](https://github.com/nickitat)). * Queries from `system.columns` will work faster if there is a large number of columns, but many databases or tables are not granted for `SHOW TABLES`. Note that in previous versions, if you grant `SHOW COLUMNS` to individual columns without granting `SHOW TABLES` to the corresponding tables, the `system.columns` table will show these columns, but in a new version, it will skip the table entirely. Remove trace log messages "Access granted" and "Access denied" that slowed down queries. [#63439](https://github.com/ClickHouse/ClickHouse/pull/63439) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/CMakeLists.txt b/CMakeLists.txt index 96ba2961d3a..c4f093b1c99 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -122,6 +122,8 @@ add_library(global-libs INTERFACE) include (cmake/sanitize.cmake) +include (cmake/xray_instrumentation.cmake) + option(ENABLE_COLORED_BUILD "Enable colors in compiler output" ON) set (CMAKE_COLOR_MAKEFILE ${ENABLE_COLORED_BUILD}) # works only for the makefile generator @@ -208,8 +210,6 @@ option(OMIT_HEAVY_DEBUG_SYMBOLS "Do not generate debugger info for heavy modules (ClickHouse functions and dictionaries, some contrib)" ${OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT}) -option(USE_DEBUG_HELPERS "Enable debug helpers" ${USE_DEBUG_HELPERS}) - option(BUILD_STANDALONE_KEEPER "Build keeper as small standalone binary" OFF) if (NOT BUILD_STANDALONE_KEEPER) option(CREATE_KEEPER_SYMLINK "Create symlink for clickhouse-keeper to main server binary" ON) @@ -399,7 +399,7 @@ option (ENABLE_GWP_ASAN "Enable Gwp-Asan" ON) # but GWP-ASan also wants to use mmap frequently, # and due to a large number of memory mappings, # it does not work together well. -if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) +if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") OR SANITIZE) set(ENABLE_GWP_ASAN OFF) endif () diff --git a/SECURITY.md b/SECURITY.md index f8511fb42d6..8635951dc0e 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -19,7 +19,10 @@ The following versions of ClickHouse server are currently supported with securit | 24.3 | ✔️ | | 24.2 | ❌ | | 24.1 | ❌ | -| 23.* | ❌ | +| 23.12 | ❌ | +| 23.11 | ❌ | +| 23.10 | ❌ | +| 23.9 | ❌ | | 23.8 | ✔️ | | 23.7 | ❌ | | 23.6 | ❌ | diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 27aa0bd6baf..159502c9735 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -34,15 +34,6 @@ set (SRCS throwError.cpp ) -if (USE_DEBUG_HELPERS) - get_target_property(MAGIC_ENUM_INCLUDE_DIR ch_contrib::magic_enum INTERFACE_INCLUDE_DIRECTORIES) - # CMake generator expression will do insane quoting when it encounters special character like quotes, spaces, etc. - # Prefixing "SHELL:" will force it to use the original text. - set (INCLUDE_DEBUG_HELPERS "SHELL:-I\"${MAGIC_ENUM_INCLUDE_DIR}\" -include \"${ClickHouse_SOURCE_DIR}/base/base/iostream_debug_helpers.h\"") - # Use generator expression as we don't want to pollute CMAKE_CXX_FLAGS, which will interfere with CMake check system. - add_compile_options($<$:${INCLUDE_DEBUG_HELPERS}>) -endif () - add_library (common ${SRCS}) if (WITH_COVERAGE) diff --git a/base/base/EnumReflection.h b/base/base/EnumReflection.h index 4a9de4d17a3..e4e0ef672fd 100644 --- a/base/base/EnumReflection.h +++ b/base/base/EnumReflection.h @@ -32,7 +32,7 @@ constexpr void static_for(F && f) template struct fmt::formatter : fmt::formatter { - constexpr auto format(T value, auto& format_context) + constexpr auto format(T value, auto& format_context) const { return formatter::format(magic_enum::enum_name(value), format_context); } diff --git a/base/base/StringRef.h b/base/base/StringRef.h index 24af84626de..fc0674b8440 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include @@ -376,3 +378,5 @@ namespace PackedZeroTraits std::ostream & operator<<(std::ostream & os, const StringRef & str); + +template<> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/base/base/iostream_debug_helpers.h b/base/base/iostream_debug_helpers.h deleted file mode 100644 index b23d3d9794d..00000000000 --- a/base/base/iostream_debug_helpers.h +++ /dev/null @@ -1,187 +0,0 @@ -#pragma once - -#include "demangle.h" -#include "getThreadId.h" -#include -#include -#include -#include -#include - -/** Usage: - * - * DUMP(variable...) - */ - - -template -Out & dumpValue(Out &, T &&); - - -/// Catch-all case. -template -requires(priority == -1) -Out & dumpImpl(Out & out, T &&) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return out << "{...}"; -} - -/// An object, that could be output with operator <<. -template -requires(priority == 0) -Out & dumpImpl(Out & out, T && x, std::decay_t() << std::declval())> * = nullptr) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return out << x; -} - -/// A pointer-like object. -template -requires(priority == 1 - /// Protect from the case when operator * do effectively nothing (function pointer). - && !std::is_same_v, std::decay_t())>>) -Out & dumpImpl(Out & out, T && x, std::decay_t())> * = nullptr) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - if (!x) - return out << "nullptr"; - return dumpValue(out, *x); -} - -/// Container. -template -requires(priority == 2) -Out & dumpImpl(Out & out, T && x, std::decay_t()))> * = nullptr) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - bool first = true; - out << "{"; - for (const auto & elem : x) - { - if (first) - first = false; - else - out << ", "; - dumpValue(out, elem); - } - return out << "}"; -} - - -template -requires(priority == 3 && std::is_enum_v>) -Out & dumpImpl(Out & out, T && x) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return out << magic_enum::enum_name(x); -} - -/// string and const char * - output not as container or pointer. - -template -requires(priority == 3 && (std::is_same_v, std::string> || std::is_same_v, const char *>)) -Out & dumpImpl(Out & out, T && x) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return out << std::quoted(x); -} - -/// UInt8 - output as number, not char. - -template -requires(priority == 3 && std::is_same_v, unsigned char>) -Out & dumpImpl(Out & out, T && x) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return out << int(x); -} - - -/// Tuple, pair -template -Out & dumpTupleImpl(Out & out, T && x) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - if constexpr (N == 0) - out << "{"; - else - out << ", "; - - dumpValue(out, std::get(x)); - - if constexpr (N + 1 == std::tuple_size_v>) - out << "}"; - else - dumpTupleImpl(out, x); - - return out; -} - -template -requires(priority == 4) -Out & dumpImpl(Out & out, T && x, std::decay_t(std::declval()))> * = nullptr) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return dumpTupleImpl<0>(out, x); -} - - -template -Out & dumpDispatchPriorities(Out & out, T && x, std::decay_t(std::declval(), std::declval()))> *) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return dumpImpl(out, x); -} - -// NOLINTNEXTLINE(google-explicit-constructor) -struct LowPriority { LowPriority(void *) {} }; - -template -Out & dumpDispatchPriorities(Out & out, T && x, LowPriority) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return dumpDispatchPriorities(out, x, nullptr); -} - - -template -Out & dumpValue(Out & out, T && x) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - return dumpDispatchPriorities<5>(out, x, nullptr); -} - - -template -Out & dump(Out & out, const char * name, T && x) // NOLINT(cppcoreguidelines-missing-std-forward) -{ - // Dumping string literal, printing name and demangled type is irrelevant. - if constexpr (std::is_same_v>>) - { - const auto name_len = strlen(name); - const auto value_len = strlen(x); - // `name` is the same as quoted `x` - if (name_len > 2 && value_len > 0 && name[0] == '"' && name[name_len - 1] == '"' - && strncmp(name + 1, x, std::min(value_len, name_len) - 1) == 0) - return out << x; - } - - out << demangle(typeid(x).name()) << " " << name << " = "; - return dumpValue(out, x) << "; "; -} - -#pragma clang diagnostic ignored "-Wgnu-zero-variadic-macro-arguments" - -#define DUMPVAR(VAR) ::dump(std::cerr, #VAR, (VAR)); -#define DUMPHEAD std::cerr << __FILE__ << ':' << __LINE__ << " [ " << getThreadId() << " ] "; -#define DUMPTAIL std::cerr << '\n'; - -#define DUMP1(V1) do { DUMPHEAD DUMPVAR(V1) DUMPTAIL } while(0) -#define DUMP2(V1, V2) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPTAIL } while(0) -#define DUMP3(V1, V2, V3) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPTAIL } while(0) -#define DUMP4(V1, V2, V3, V4) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPTAIL } while(0) -#define DUMP5(V1, V2, V3, V4, V5) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPTAIL } while(0) -#define DUMP6(V1, V2, V3, V4, V5, V6) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) DUMPTAIL } while(0) -#define DUMP7(V1, V2, V3, V4, V5, V6, V7) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) DUMPVAR(V7) DUMPTAIL } while(0) -#define DUMP8(V1, V2, V3, V4, V5, V6, V7, V8) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) DUMPVAR(V7) DUMPVAR(V8) DUMPTAIL } while(0) -#define DUMP9(V1, V2, V3, V4, V5, V6, V7, V8, V9) do { DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) DUMPVAR(V7) DUMPVAR(V8) DUMPVAR(V9) DUMPTAIL } while(0) - -/// https://groups.google.com/forum/#!searchin/kona-dev/variadic$20macro%7Csort:date/kona-dev/XMA-lDOqtlI/GCzdfZsD41sJ - -#define VA_NUM_ARGS_IMPL(x1, x2, x3, x4, x5, x6, x7, x8, x9, N, ...) N -#define VA_NUM_ARGS(...) VA_NUM_ARGS_IMPL(__VA_ARGS__, 9, 8, 7, 6, 5, 4, 3, 2, 1) - -#define MAKE_VAR_MACRO_IMPL_CONCAT(PREFIX, NUM_ARGS) PREFIX ## NUM_ARGS -#define MAKE_VAR_MACRO_IMPL(PREFIX, NUM_ARGS) MAKE_VAR_MACRO_IMPL_CONCAT(PREFIX, NUM_ARGS) -#define MAKE_VAR_MACRO(PREFIX, ...) MAKE_VAR_MACRO_IMPL(PREFIX, VA_NUM_ARGS(__VA_ARGS__)) - -#define DUMP(...) MAKE_VAR_MACRO(DUMP, __VA_ARGS__)(__VA_ARGS__) diff --git a/base/base/tests/CMakeLists.txt b/base/base/tests/CMakeLists.txt index 81db4f3622f..e69de29bb2d 100644 --- a/base/base/tests/CMakeLists.txt +++ b/base/base/tests/CMakeLists.txt @@ -1,2 +0,0 @@ -clickhouse_add_executable (dump_variable dump_variable.cpp) -target_link_libraries (dump_variable PRIVATE clickhouse_common_io) diff --git a/base/base/tests/dump_variable.cpp b/base/base/tests/dump_variable.cpp deleted file mode 100644 index 9addc298ecb..00000000000 --- a/base/base/tests/dump_variable.cpp +++ /dev/null @@ -1,70 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - - -struct S1; -struct S2 {}; - -struct S3 -{ - std::set m1; -}; - -std::ostream & operator<<(std::ostream & stream, const S3 & what) -{ - stream << "S3 {m1="; - dumpValue(stream, what.m1) << "}"; - return stream; -} - -int main(int, char **) -{ - int x = 1; - - DUMP(x); - DUMP(x, 1, &x); - - DUMP(std::make_unique(1)); - DUMP(std::make_shared(1)); - - std::vector vec{1, 2, 3}; - DUMP(vec); - - auto pair = std::make_pair(1, 2); - DUMP(pair); - - auto tuple = std::make_tuple(1, 2, 3); - DUMP(tuple); - - std::map map{{1, "hello"}, {2, "world"}}; - DUMP(map); - - std::initializer_list list{"hello", "world"}; - DUMP(list); - - std::array arr{{"hello", "world"}}; - DUMP(arr); - - //DUMP([]{}); - - S1 * s = nullptr; - DUMP(s); - - DUMP(S2()); - - std::set variants = {"hello", "world"}; - DUMP(variants); - - S3 s3 {{"hello", "world"}}; - DUMP(s3); - - return 0; -} diff --git a/base/base/wide_integer_to_string.h b/base/base/wide_integer_to_string.h index c2cbe8d82e3..f703a722afa 100644 --- a/base/base/wide_integer_to_string.h +++ b/base/base/wide_integer_to_string.h @@ -62,7 +62,7 @@ struct fmt::formatter> } template - auto format(const wide::integer & value, FormatContext & ctx) + auto format(const wide::integer & value, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", to_string(value)); } diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index dfb41a33fb1..324a0170bdd 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -213,6 +213,7 @@ target_compile_definitions (_poco_foundation ) target_include_directories (_poco_foundation SYSTEM PUBLIC "include") +target_link_libraries (_poco_foundation PRIVATE clickhouse_common_io) target_link_libraries (_poco_foundation PRIVATE diff --git a/base/poco/Foundation/include/Poco/ThreadPool.h b/base/poco/Foundation/include/Poco/ThreadPool.h index b9506cc5b7f..e2187bfeb66 100644 --- a/base/poco/Foundation/include/Poco/ThreadPool.h +++ b/base/poco/Foundation/include/Poco/ThreadPool.h @@ -48,7 +48,13 @@ class Foundation_API ThreadPool /// from the pool. { public: - ThreadPool(int minCapacity = 2, int maxCapacity = 16, int idleTime = 60, int stackSize = POCO_THREAD_STACK_SIZE); + explicit ThreadPool( + int minCapacity = 2, + int maxCapacity = 16, + int idleTime = 60, + int stackSize = POCO_THREAD_STACK_SIZE, + size_t global_profiler_real_time_period_ns_ = 0, + size_t global_profiler_cpu_time_period_ns_ = 0); /// Creates a thread pool with minCapacity threads. /// If required, up to maxCapacity threads are created /// a NoThreadAvailableException exception is thrown. @@ -56,8 +62,14 @@ public: /// and more than minCapacity threads are running, the thread /// is killed. Threads are created with given stack size. - ThreadPool( - const std::string & name, int minCapacity = 2, int maxCapacity = 16, int idleTime = 60, int stackSize = POCO_THREAD_STACK_SIZE); + explicit ThreadPool( + const std::string & name, + int minCapacity = 2, + int maxCapacity = 16, + int idleTime = 60, + int stackSize = POCO_THREAD_STACK_SIZE, + size_t global_profiler_real_time_period_ns_ = 0, + size_t global_profiler_cpu_time_period_ns_ = 0); /// Creates a thread pool with the given name and minCapacity threads. /// If required, up to maxCapacity threads are created /// a NoThreadAvailableException exception is thrown. @@ -171,6 +183,8 @@ private: int _serial; int _age; int _stackSize; + size_t _globalProfilerRealTimePeriodNs; + size_t _globalProfilerCPUTimePeriodNs; ThreadVec _threads; mutable FastMutex _mutex; }; diff --git a/base/poco/Foundation/src/ThreadPool.cpp b/base/poco/Foundation/src/ThreadPool.cpp index 6335ee82b47..f57c81e4128 100644 --- a/base/poco/Foundation/src/ThreadPool.cpp +++ b/base/poco/Foundation/src/ThreadPool.cpp @@ -20,6 +20,7 @@ #include "Poco/ErrorHandler.h" #include #include +#include namespace Poco { @@ -28,7 +29,11 @@ namespace Poco { class PooledThread: public Runnable { public: - PooledThread(const std::string& name, int stackSize = POCO_THREAD_STACK_SIZE); + explicit PooledThread( + const std::string& name, + int stackSize = POCO_THREAD_STACK_SIZE, + size_t globalProfilerRealTimePeriodNs_ = 0, + size_t globalProfilerCPUTimePeriodNs_ = 0); ~PooledThread(); void start(); @@ -51,16 +56,24 @@ private: Event _targetCompleted; Event _started; FastMutex _mutex; + size_t _globalProfilerRealTimePeriodNs; + size_t _globalProfilerCPUTimePeriodNs; }; -PooledThread::PooledThread(const std::string& name, int stackSize): - _idle(true), - _idleTime(0), - _pTarget(0), - _name(name), +PooledThread::PooledThread( + const std::string& name, + int stackSize, + size_t globalProfilerRealTimePeriodNs_, + size_t globalProfilerCPUTimePeriodNs_) : + _idle(true), + _idleTime(0), + _pTarget(0), + _name(name), _thread(name), - _targetCompleted(false) + _targetCompleted(false), + _globalProfilerRealTimePeriodNs(globalProfilerRealTimePeriodNs_), + _globalProfilerCPUTimePeriodNs(globalProfilerCPUTimePeriodNs_) { poco_assert_dbg (stackSize >= 0); _thread.setStackSize(stackSize); @@ -83,7 +96,7 @@ void PooledThread::start() void PooledThread::start(Thread::Priority priority, Runnable& target) { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -109,7 +122,7 @@ void PooledThread::start(Thread::Priority priority, Runnable& target, const std: } _thread.setName(fullName); _thread.setPriority(priority); - + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -145,7 +158,7 @@ void PooledThread::join() void PooledThread::activate() { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_idle); _idle = false; _targetCompleted.reset(); @@ -155,7 +168,7 @@ void PooledThread::activate() void PooledThread::release() { const long JOIN_TIMEOUT = 10000; - + _mutex.lock(); _pTarget = 0; _mutex.unlock(); @@ -174,6 +187,10 @@ void PooledThread::release() void PooledThread::run() { + DB::ThreadStatus thread_status; + if (unlikely(_globalProfilerRealTimePeriodNs != 0 || _globalProfilerCPUTimePeriodNs != 0)) + thread_status.initGlobalProfiler(_globalProfilerRealTimePeriodNs, _globalProfilerCPUTimePeriodNs); + _started.set(); for (;;) { @@ -220,13 +237,17 @@ void PooledThread::run() ThreadPool::ThreadPool(int minCapacity, int maxCapacity, int idleTime, - int stackSize): - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + int stackSize, + size_t globalProfilerRealTimePeriodNs_, + size_t globalProfilerCPUTimePeriodNs_) : + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), - _stackSize(stackSize) + _stackSize(stackSize), + _globalProfilerRealTimePeriodNs(globalProfilerRealTimePeriodNs_), + _globalProfilerCPUTimePeriodNs(globalProfilerCPUTimePeriodNs_) { poco_assert (minCapacity >= 1 && maxCapacity >= minCapacity && idleTime > 0); @@ -243,14 +264,18 @@ ThreadPool::ThreadPool(const std::string& name, int minCapacity, int maxCapacity, int idleTime, - int stackSize): + int stackSize, + size_t globalProfilerRealTimePeriodNs_, + size_t globalProfilerCPUTimePeriodNs_) : _name(name), - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), - _stackSize(stackSize) + _stackSize(stackSize), + _globalProfilerRealTimePeriodNs(globalProfilerRealTimePeriodNs_), + _globalProfilerCPUTimePeriodNs(globalProfilerCPUTimePeriodNs_) { poco_assert (minCapacity >= 1 && maxCapacity >= minCapacity && idleTime > 0); @@ -393,15 +418,15 @@ void ThreadPool::housekeep() ThreadVec activeThreads; idleThreads.reserve(_threads.size()); activeThreads.reserve(_threads.size()); - + for (ThreadVec::iterator it = _threads.begin(); it != _threads.end(); ++it) { if ((*it)->idle()) { if ((*it)->idleTime() < _idleTime) idleThreads.push_back(*it); - else - expiredThreads.push_back(*it); + else + expiredThreads.push_back(*it); } else activeThreads.push_back(*it); } @@ -463,7 +488,7 @@ PooledThread* ThreadPool::createThread() { std::ostringstream name; name << _name << "[#" << ++_serial << "]"; - return new PooledThread(name.str(), _stackSize); + return new PooledThread(name.str(), _stackSize, _globalProfilerRealTimePeriodNs, _globalProfilerCPUTimePeriodNs); } @@ -481,7 +506,7 @@ public: ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -490,7 +515,7 @@ public: } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h index c87719b63a4..3178306363c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h +++ b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h @@ -26,7 +26,7 @@ namespace Poco { namespace Net { - constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 8 * 1024; + constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 1024 * 1024; typedef Poco::BasicBufferedStreamBuf> HTTPBasicStreamBuf; diff --git a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp index 7f6cc9abcb2..d404aed4d13 100644 --- a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp @@ -330,27 +330,26 @@ void SSLManager::initDefaultContext(bool server) else _ptrDefaultClientContext->disableProtocols(disabledProtocols); - /// Temporarily disabled during the transition from boringssl to OpenSSL due to tsan issues. - /// bool cacheSessions = config.getBool(prefix + CFG_CACHE_SESSIONS, false); - /// if (server) - /// { - /// std::string sessionIdContext = config.getString(prefix + CFG_SESSION_ID_CONTEXT, config.getString("application.name", "")); - /// _ptrDefaultServerContext->enableSessionCache(cacheSessions, sessionIdContext); - /// if (config.hasProperty(prefix + CFG_SESSION_CACHE_SIZE)) - /// { - /// int cacheSize = config.getInt(prefix + CFG_SESSION_CACHE_SIZE); - /// _ptrDefaultServerContext->setSessionCacheSize(cacheSize); - /// } - /// if (config.hasProperty(prefix + CFG_SESSION_TIMEOUT)) - /// { - /// int timeout = config.getInt(prefix + CFG_SESSION_TIMEOUT); - /// _ptrDefaultServerContext->setSessionTimeout(timeout); - /// } - /// } - /// else - /// { - /// _ptrDefaultClientContext->enableSessionCache(cacheSessions); - /// } + bool cacheSessions = config.getBool(prefix + CFG_CACHE_SESSIONS, false); + if (server) + { + std::string sessionIdContext = config.getString(prefix + CFG_SESSION_ID_CONTEXT, config.getString("application.name", "")); + _ptrDefaultServerContext->enableSessionCache(cacheSessions, sessionIdContext); + if (config.hasProperty(prefix + CFG_SESSION_CACHE_SIZE)) + { + int cacheSize = config.getInt(prefix + CFG_SESSION_CACHE_SIZE); + _ptrDefaultServerContext->setSessionCacheSize(cacheSize); + } + if (config.hasProperty(prefix + CFG_SESSION_TIMEOUT)) + { + int timeout = config.getInt(prefix + CFG_SESSION_TIMEOUT); + _ptrDefaultServerContext->setSessionTimeout(timeout); + } + } + else + { + _ptrDefaultClientContext->enableSessionCache(cacheSessions); + } bool extendedVerification = config.getBool(prefix + CFG_EXTENDED_VERIFICATION, false); if (server) _ptrDefaultServerContext->enableExtendedCertificateVerification(extendedVerification); diff --git a/cmake/xray_instrumentation.cmake b/cmake/xray_instrumentation.cmake new file mode 100644 index 00000000000..661c0575e54 --- /dev/null +++ b/cmake/xray_instrumentation.cmake @@ -0,0 +1,20 @@ +# https://llvm.org/docs/XRay.html + +option (ENABLE_XRAY "Enable LLVM XRay" OFF) + +if (NOT ENABLE_XRAY) + message (STATUS "Not using LLVM XRay") + return() +endif() + +if (NOT (ARCH_AMD64 AND OS_LINUX)) + message (STATUS "Not using LLVM XRay, only amd64 Linux or FreeBSD are supported") + return() +endif() + +# The target clang must support xray, otherwise it should error on invalid option +set (XRAY_FLAGS "-fxray-instrument -DUSE_XRAY") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${XRAY_FLAGS}") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${XRAY_FLAGS}") + +message (STATUS "Using LLVM XRay") diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index 3bd86026c93..a3c4dd3e77f 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit 3bd86026c93da5a40006fd53403dff9d5f5e30e3 +Subproject commit a3c4dd3e77f28b526efbb0eb394b72e29c633936 diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index 7372195bb0d..4137547b736 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -1,6 +1,8 @@ set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") set(ABSL_COMMON_INCLUDE_DIRS "${ABSL_ROOT_DIR}") +# This is a minimized version of the function definition in CMake/AbseilHelpers.cmake + # # Copyright 2017 The Abseil Authors. # @@ -16,7 +18,6 @@ set(ABSL_COMMON_INCLUDE_DIRS "${ABSL_ROOT_DIR}") # See the License for the specific language governing permissions and # limitations under the License. # - function(absl_cc_library) cmake_parse_arguments(ABSL_CC_LIB "DISABLE_INSTALL;PUBLIC;TESTONLY" @@ -76,6 +77,12 @@ function(absl_cc_library) add_library(absl::${ABSL_CC_LIB_NAME} ALIAS ${_NAME}) endfunction() +# The following definitions are an amalgamation of the CMakeLists.txt files in absl/*/ +# To refresh them when upgrading to a new version: +# - copy them over from upstream +# - remove calls of 'absl_cc_test' +# - remove calls of `absl_cc_library` that contain `TESTONLY` +# - append '${DIR}' to the file definitions set(DIR ${ABSL_ROOT_DIR}/absl/algorithm) @@ -102,12 +109,12 @@ absl_cc_library( absl::algorithm absl::core_headers absl::meta + absl::nullability PUBLIC ) set(DIR ${ABSL_ROOT_DIR}/absl/base) -# Internal-only target, do not depend on directly. absl_cc_library( NAME atomic_hook @@ -146,6 +153,18 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} ) +absl_cc_library( + NAME + no_destructor + HDRS + "${DIR}/no_destructor.h" + DEPS + absl::config + absl::nullability + COPTS + ${ABSL_DEFAULT_COPTS} +) + absl_cc_library( NAME nullability @@ -305,6 +324,8 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} LINKOPTS ${ABSL_DEFAULT_LINKOPTS} + $<$:-lrt> + $<$:-ladvapi32> DEPS absl::atomic_hook absl::base_internal @@ -312,6 +333,7 @@ absl_cc_library( absl::core_headers absl::dynamic_annotations absl::log_severity + absl::nullability absl::raw_logging_internal absl::spinlock_wait absl::type_traits @@ -357,6 +379,7 @@ absl_cc_library( absl::base absl::config absl::core_headers + absl::nullability PUBLIC ) @@ -467,10 +490,11 @@ absl_cc_library( LINKOPTS ${ABSL_DEFAULT_LINKOPTS} DEPS - absl::container_common absl::common_policy_traits absl::compare absl::compressed_tuple + absl::config + absl::container_common absl::container_memory absl::cord absl::core_headers @@ -480,7 +504,6 @@ absl_cc_library( absl::strings absl::throw_delegate absl::type_traits - absl::utility ) # Internal-only target, do not depend on directly. @@ -523,7 +546,9 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::base_internal absl::compressed_tuple + absl::config absl::core_headers absl::memory absl::span @@ -548,18 +573,6 @@ absl_cc_library( PUBLIC ) -# Internal-only target, do not depend on directly. -absl_cc_library( - NAME - counting_allocator - HDRS - "${DIR}/internal/counting_allocator.h" - COPTS - ${ABSL_DEFAULT_COPTS} - DEPS - absl::config -) - absl_cc_library( NAME flat_hash_map @@ -570,7 +583,7 @@ absl_cc_library( DEPS absl::container_memory absl::core_headers - absl::hash_function_defaults + absl::hash_container_defaults absl::raw_hash_map absl::algorithm_container absl::memory @@ -586,7 +599,7 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} DEPS absl::container_memory - absl::hash_function_defaults + absl::hash_container_defaults absl::raw_hash_set absl::algorithm_container absl::core_headers @@ -604,7 +617,7 @@ absl_cc_library( DEPS absl::container_memory absl::core_headers - absl::hash_function_defaults + absl::hash_container_defaults absl::node_slot_policy absl::raw_hash_map absl::algorithm_container @@ -620,8 +633,9 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::container_memory absl::core_headers - absl::hash_function_defaults + absl::hash_container_defaults absl::node_slot_policy absl::raw_hash_set absl::algorithm_container @@ -629,6 +643,19 @@ absl_cc_library( PUBLIC ) +absl_cc_library( + NAME + hash_container_defaults + HDRS + "${DIR}/hash_container_defaults.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + absl::hash_function_defaults + PUBLIC +) + # Internal-only target, do not depend on directly. absl_cc_library( NAME @@ -655,9 +682,11 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} DEPS absl::config + absl::container_common absl::cord absl::hash absl::strings + absl::type_traits PUBLIC ) @@ -703,6 +732,7 @@ absl_cc_library( absl::base absl::config absl::exponential_biased + absl::no_destructor absl::raw_logging_internal absl::sample_recorder absl::synchronization @@ -756,7 +786,9 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::config absl::container_memory + absl::core_headers absl::raw_hash_set absl::throw_delegate PUBLIC @@ -817,6 +849,7 @@ absl_cc_library( DEPS absl::config absl::core_headers + absl::debugging_internal absl::meta absl::strings absl::span @@ -931,6 +964,7 @@ absl_cc_library( absl::crc32c absl::config absl::strings + absl::no_destructor ) set(DIR ${ABSL_ROOT_DIR}/absl/debugging) @@ -954,6 +988,8 @@ absl_cc_library( "${DIR}/stacktrace.cc" COPTS ${ABSL_DEFAULT_COPTS} + LINKOPTS + $<$:${EXECINFO_LIBRARY}> DEPS absl::debugging_internal absl::config @@ -980,6 +1016,7 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} LINKOPTS ${ABSL_DEFAULT_LINKOPTS} + $<$:-ldbghelp> DEPS absl::debugging_internal absl::demangle_internal @@ -1058,8 +1095,10 @@ absl_cc_library( demangle_internal HDRS "${DIR}/internal/demangle.h" + "${DIR}/internal/demangle_rust.h" SRCS "${DIR}/internal/demangle.cc" + "${DIR}/internal/demangle_rust.cc" COPTS ${ABSL_DEFAULT_COPTS} DEPS @@ -1252,6 +1291,7 @@ absl_cc_library( absl::strings absl::synchronization absl::flat_hash_map + absl::no_destructor ) # Internal-only target, do not depend on directly. @@ -1283,12 +1323,9 @@ absl_cc_library( absl_cc_library( NAME flags - SRCS - "${DIR}/flag.cc" HDRS "${DIR}/declare.h" "${DIR}/flag.h" - "${DIR}/internal/flag_msvc.inc" COPTS ${ABSL_DEFAULT_COPTS} LINKOPTS @@ -1299,7 +1336,6 @@ absl_cc_library( absl::flags_config absl::flags_internal absl::flags_reflection - absl::base absl::core_headers absl::strings ) @@ -1379,6 +1415,9 @@ absl_cc_library( absl::synchronization ) +############################################################################ +# Unit tests in alphabetical order. + set(DIR ${ABSL_ROOT_DIR}/absl/functional) absl_cc_library( @@ -1431,6 +1470,18 @@ absl_cc_library( PUBLIC ) +absl_cc_library( + NAME + overload + HDRS + "${DIR}/overload.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::meta + PUBLIC +) + set(DIR ${ABSL_ROOT_DIR}/absl/hash) absl_cc_library( @@ -1640,6 +1691,7 @@ absl_cc_library( absl::log_internal_conditions absl::log_internal_message absl::log_internal_strip + absl::absl_vlog_is_on ) absl_cc_library( @@ -1721,6 +1773,7 @@ absl_cc_library( absl::log_entry absl::log_severity absl::log_sink + absl::no_destructor absl::raw_logging_internal absl::synchronization absl::span @@ -1771,6 +1824,7 @@ absl_cc_library( LINKOPTS ${ABSL_DEFAULT_LINKOPTS} DEPS + absl::core_headers absl::log_internal_message absl::log_internal_nullstream absl::log_severity @@ -1876,6 +1930,11 @@ absl_cc_library( PUBLIC ) +# Warning: Many linkers will strip the contents of this library because its +# symbols are only used in a global constructor. A workaround is for clients +# to link this using $ instead of +# the plain absl::log_flags. +# TODO(b/320467376): Implement the equivalent of Bazel's alwayslink=True. absl_cc_library( NAME log_flags @@ -1897,6 +1956,7 @@ absl_cc_library( absl::flags absl::flags_marshalling absl::strings + absl::vlog_config_internal PUBLIC ) @@ -1919,6 +1979,7 @@ absl_cc_library( absl::log_severity absl::raw_logging_internal absl::strings + absl::vlog_config_internal ) absl_cc_library( @@ -1952,6 +2013,7 @@ absl_cc_library( ${ABSL_DEFAULT_LINKOPTS} DEPS absl::log_internal_log_impl + absl::vlog_is_on PUBLIC ) @@ -2064,21 +2126,75 @@ absl_cc_library( ) absl_cc_library( - NAME - log_internal_fnmatch - SRCS - "${DIR}/internal/fnmatch.cc" - HDRS - "${DIR}/internal/fnmatch.h" - COPTS - ${ABSL_DEFAULT_COPTS} - LINKOPTS - ${ABSL_DEFAULT_LINKOPTS} - DEPS - absl::config - absl::strings + NAME + vlog_config_internal + SRCS + "${DIR}/internal/vlog_config.cc" + HDRS + "${DIR}/internal/vlog_config.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::base + absl::config + absl::core_headers + absl::log_internal_fnmatch + absl::memory + absl::no_destructor + absl::strings + absl::synchronization + absl::optional ) +absl_cc_library( + NAME + absl_vlog_is_on + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + HDRS + "${DIR}/absl_vlog_is_on.h" + DEPS + absl::vlog_config_internal + absl::config + absl::core_headers + absl::strings +) + +absl_cc_library( + NAME + vlog_is_on + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + HDRS + "${DIR}/vlog_is_on.h" + DEPS + absl::absl_vlog_is_on +) + +absl_cc_library( + NAME + log_internal_fnmatch + SRCS + "${DIR}/internal/fnmatch.cc" + HDRS + "${DIR}/internal/fnmatch.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::strings +) + +# Test targets + set(DIR ${ABSL_ROOT_DIR}/absl/memory) absl_cc_library( @@ -2147,6 +2263,7 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::compare absl::config absl::core_headers absl::bits @@ -2176,6 +2293,8 @@ absl_cc_library( PUBLIC ) +set(DIR ${ABSL_ROOT_DIR}/absl/profiling) + absl_cc_library( NAME sample_recorder @@ -2188,8 +2307,6 @@ absl_cc_library( absl::synchronization ) -set(DIR ${ABSL_ROOT_DIR}/absl/profiling) - absl_cc_library( NAME exponential_biased @@ -2265,6 +2382,7 @@ absl_cc_library( LINKOPTS ${ABSL_DEFAULT_LINKOPTS} DEPS + absl::config absl::fast_type_id absl::optional ) @@ -2336,11 +2454,13 @@ absl_cc_library( DEPS absl::config absl::inlined_vector + absl::nullability absl::random_internal_pool_urbg absl::random_internal_salted_seed_seq absl::random_internal_seed_material absl::random_seed_gen_exception absl::span + absl::string_view ) # Internal-only target, do not depend on directly. @@ -2399,6 +2519,7 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} LINKOPTS ${ABSL_DEFAULT_LINKOPTS} + $<$:-lbcrypt> DEPS absl::core_headers absl::optional @@ -2658,6 +2779,29 @@ absl_cc_library( absl::config ) +# Internal-only target, do not depend on directly. +absl_cc_library( + NAME + random_internal_distribution_test_util + SRCS + "${DIR}/internal/chi_square.cc" + "${DIR}/internal/distribution_test_util.cc" + HDRS + "${DIR}/internal/chi_square.h" + "${DIR}/internal/distribution_test_util.h" + COPTS + ${ABSL_DEFAULT_COPTS} + LINKOPTS + ${ABSL_DEFAULT_LINKOPTS} + DEPS + absl::config + absl::core_headers + absl::raw_logging_internal + absl::strings + absl::str_format + absl::span +) + # Internal-only target, do not depend on directly. absl_cc_library( NAME @@ -2699,6 +2843,8 @@ absl_cc_library( absl::function_ref absl::inlined_vector absl::memory + absl::no_destructor + absl::nullability absl::optional absl::raw_logging_internal absl::span @@ -2724,8 +2870,11 @@ absl_cc_library( absl::base absl::config absl::core_headers + absl::has_ostream_operator + absl::nullability absl::raw_logging_internal absl::status + absl::str_format absl::strings absl::type_traits absl::utility @@ -2748,6 +2897,7 @@ absl_cc_library( absl::base absl::config absl::core_headers + absl::nullability absl::throw_delegate PUBLIC ) @@ -2762,6 +2912,7 @@ absl_cc_library( "${DIR}/has_absl_stringify.h" "${DIR}/internal/damerau_levenshtein_distance.h" "${DIR}/internal/string_constant.h" + "${DIR}/internal/has_absl_stringify.h" "${DIR}/match.h" "${DIR}/numbers.h" "${DIR}/str_cat.h" @@ -2805,6 +2956,7 @@ absl_cc_library( absl::endian absl::int128 absl::memory + absl::nullability absl::raw_logging_internal absl::throw_delegate absl::type_traits @@ -2824,6 +2976,18 @@ absl_cc_library( PUBLIC ) +absl_cc_library( + NAME + has_ostream_operator + HDRS + "${DIR}/has_ostream_operator.h" + COPTS + ${ABSL_DEFAULT_COPTS} + DEPS + absl::config + PUBLIC +) + # Internal-only target, do not depend on directly. absl_cc_library( NAME @@ -2855,7 +3019,12 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::config + absl::core_headers + absl::nullability + absl::span absl::str_format_internal + absl::string_view PUBLIC ) @@ -2886,6 +3055,7 @@ absl_cc_library( absl::strings absl::config absl::core_headers + absl::fixed_array absl::inlined_vector absl::numeric_representation absl::type_traits @@ -2989,6 +3159,7 @@ absl_cc_library( DEPS absl::base absl::config + absl::no_destructor absl::raw_logging_internal absl::synchronization ) @@ -3079,6 +3250,7 @@ absl_cc_library( absl::endian absl::function_ref absl::inlined_vector + absl::nullability absl::optional absl::raw_logging_internal absl::span @@ -3246,6 +3418,8 @@ absl_cc_library( ${ABSL_DEFAULT_COPTS} DEPS Threads::Threads + # TODO(#1495): Use $ once our + # minimum CMake version >= 3.24 $<$:-Wl,-framework,CoreFoundation> ) @@ -3286,8 +3460,8 @@ absl_cc_library( NAME bad_any_cast_impl SRCS - "${DIR}/bad_any_cast.h" - "${DIR}/bad_any_cast.cc" + "${DIR}/bad_any_cast.h" + "${DIR}/bad_any_cast.cc" COPTS ${ABSL_DEFAULT_COPTS} DEPS @@ -3307,6 +3481,7 @@ absl_cc_library( DEPS absl::algorithm absl::core_headers + absl::nullability absl::throw_delegate absl::type_traits PUBLIC @@ -3327,6 +3502,7 @@ absl_cc_library( absl::config absl::core_headers absl::memory + absl::nullability absl::type_traits absl::utility PUBLIC @@ -3389,6 +3565,7 @@ absl_cc_library( COPTS ${ABSL_DEFAULT_COPTS} DEPS + absl::config absl::core_headers absl::type_traits PUBLIC diff --git a/contrib/aws b/contrib/aws index deeaa9e7c5f..1c2946bfcb7 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit deeaa9e7c5fe690e3dacc4005d7ecfa7a66a32bb +Subproject commit 1c2946bfcb7f1e3ae0a858de0b59d4f1a7b4ccaf diff --git a/contrib/cld2 b/contrib/cld2 index bc6d493a2f6..217ba8b8805 160000 --- a/contrib/cld2 +++ b/contrib/cld2 @@ -1 +1 @@ -Subproject commit bc6d493a2f64ed1fc1c4c4b4294a542a04e04217 +Subproject commit 217ba8b8805b41557faadaa47bb6e99f2242eea3 diff --git a/contrib/fmtlib b/contrib/fmtlib index b6f4ceaed0a..a33701196ad 160000 --- a/contrib/fmtlib +++ b/contrib/fmtlib @@ -1 +1 @@ -Subproject commit b6f4ceaed0a0a24ccf575fab6c56dd50ccf6f1a9 +Subproject commit a33701196adfad74917046096bf5a2aa0ab0bb50 diff --git a/contrib/fmtlib-cmake/CMakeLists.txt b/contrib/fmtlib-cmake/CMakeLists.txt index fe399ddc6e1..6625e411295 100644 --- a/contrib/fmtlib-cmake/CMakeLists.txt +++ b/contrib/fmtlib-cmake/CMakeLists.txt @@ -13,7 +13,6 @@ set (SRCS ${FMT_SOURCE_DIR}/include/fmt/core.h ${FMT_SOURCE_DIR}/include/fmt/format.h ${FMT_SOURCE_DIR}/include/fmt/format-inl.h - ${FMT_SOURCE_DIR}/include/fmt/locale.h ${FMT_SOURCE_DIR}/include/fmt/os.h ${FMT_SOURCE_DIR}/include/fmt/ostream.h ${FMT_SOURCE_DIR}/include/fmt/printf.h diff --git a/contrib/googletest b/contrib/googletest index e47544ad31c..a7f443b80b1 160000 --- a/contrib/googletest +++ b/contrib/googletest @@ -1 +1 @@ -Subproject commit e47544ad31cb3ceecd04cc13e8fe556f8df9fe0b +Subproject commit a7f443b80b105f940225332ed3c31f2790092f47 diff --git a/contrib/openssl b/contrib/openssl index f7b8721dfc6..e0d6ae2bf93 160000 --- a/contrib/openssl +++ b/contrib/openssl @@ -1 +1 @@ -Subproject commit f7b8721dfc66abb147f24ca07b9c9d1d64f40f71 +Subproject commit e0d6ae2bf93cf6dc26bb86aa39992bc6a410869a diff --git a/contrib/orc b/contrib/orc index e24f2c2a3ca..947cebaf943 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit e24f2c2a3ca0769c96704ab20ad6f512a83ea2ad +Subproject commit 947cebaf9432d708253ac08dc3012daa6b4ede6f diff --git a/docker/images.json b/docker/images.json index 7439517379b..716b76ee217 100644 --- a/docker/images.json +++ b/docker/images.json @@ -41,8 +41,7 @@ "docker/test/stateless": { "name": "clickhouse/stateless-test", "dependent": [ - "docker/test/stateful", - "docker/test/unit" + "docker/test/stateful" ] }, "docker/test/stateful": { @@ -122,15 +121,16 @@ "docker/test/base": { "name": "clickhouse/test-base", "dependent": [ + "docker/test/clickbench", "docker/test/fuzzer", - "docker/test/libfuzzer", "docker/test/integration/base", "docker/test/keeper-jepsen", + "docker/test/libfuzzer", "docker/test/server-jepsen", "docker/test/sqllogic", "docker/test/sqltest", - "docker/test/clickbench", - "docker/test/stateless" + "docker/test/stateless", + "docker/test/unit" ] }, "docker/test/integration/kerberized_hadoop": { diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index b3271d94184..24f38740ff5 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.5.1.1763" +ARG VERSION="24.5.3.5" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 3f3b880c8f3..c71319a2a7e 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.5.1.1763" +ARG VERSION="24.5.3.5" 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 5fd22ee9b51..ed8cf3d657d 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.5.1.1763" +ARG VERSION="24.5.3.5" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index b8f967ed9c2..6191aeaf304 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -208,6 +208,7 @@ handle SIGPIPE nostop noprint pass handle SIGTERM nostop noprint pass handle SIGUSR1 nostop noprint pass handle SIGUSR2 nostop noprint pass +handle SIGSEGV nostop pass handle SIG$RTMIN nostop noprint pass info signals continue diff --git a/docker/test/stateless/attach_gdb.lib b/docker/test/stateless/attach_gdb.lib index d288288bb17..eb54f920b98 100644 --- a/docker/test/stateless/attach_gdb.lib +++ b/docker/test/stateless/attach_gdb.lib @@ -20,6 +20,7 @@ handle SIGPIPE nostop noprint pass handle SIGTERM nostop noprint pass handle SIGUSR1 nostop noprint pass handle SIGUSR2 nostop noprint pass +handle SIGSEGV nostop pass handle SIG$RTMIN nostop noprint pass info signals continue diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 4d2c2e6f466..4434a5338a7 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -254,7 +254,7 @@ function run_tests() set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --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 set -e @@ -285,7 +285,7 @@ stop_logs_replication # Try to get logs while server is running failed_to_save_logs=0 -for table in query_log zookeeper_log trace_log transactions_info_log metric_log +for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log do err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes") echo "$err" @@ -339,7 +339,7 @@ if [ $failed_to_save_logs -ne 0 ]; then # directly # - even though ci auto-compress some files (but not *.tsv) it does this only # for files >64MB, we want this files to be compressed explicitly - for table in query_log zookeeper_log trace_log transactions_info_log metric_log + for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log do clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||: if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then @@ -379,6 +379,10 @@ fi tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: +rm -rf /var/lib/clickhouse/data/system/*/ +tar -chf /test_output/store.tar /var/lib/clickhouse/store ||: +tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||: + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: rg -Fa "" /var/log/clickhouse-server/clickhouse-server2.log ||: diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 3b6ad244c82..c069ccbdd8d 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -89,10 +89,6 @@ function configure() # since we run clickhouse from root sudo chown root: /var/lib/clickhouse - # Set more frequent update period of asynchronous metrics to more frequently update information about real memory usage (less chance of OOM). - echo "1" \ - > /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml - local total_mem total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB total_mem=$(( total_mem*1024 )) # bytes diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 172fbce6406..6ad03852b66 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -10,21 +10,33 @@ RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ aspell \ curl \ - git \ - gh \ file \ + gh \ + git \ libxml2-utils \ + locales \ moreutils \ - python3-fuzzywuzzy \ python3-pip \ yamllint \ - locales \ + zstd \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* # python-magic is the same version as in Ubuntu 22.04 -RUN pip3 install black==23.12.0 boto3 codespell==2.2.1 mypy==1.8.0 PyGithub unidiff pylint==3.1.0 \ - python-magic==0.4.24 requests types-requests \ +RUN pip3 install \ + PyGithub \ + black==23.12.0 \ + boto3 \ + codespell==2.2.1 \ + mypy==1.8.0 \ + pylint==3.1.0 \ + python-magic==0.4.24 \ + flake8==4.0.1 \ + requests \ + thefuzz \ + tqdm==4.66.4 \ + types-requests \ + unidiff \ && rm -rf /root/.cache/pip RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index cc6cb292b66..64803191532 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -9,6 +9,8 @@ echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts ./check-black -n |& tee /test_output/black_output.txt +echo "Check python with flake8" | ts +./check-flake8 |& tee /test_output/flake8_output.txt echo "Check python type hinting with mypy" | ts ./check-mypy -n |& tee /test_output/mypy_output.txt echo "Check typos" | ts diff --git a/docker/test/unit/Dockerfile b/docker/test/unit/Dockerfile index cf5ba1eec7f..af44dc930b2 100644 --- a/docker/test/unit/Dockerfile +++ b/docker/test/unit/Dockerfile @@ -1,9 +1,7 @@ # rebuild in #33610 # docker build -t clickhouse/unit-test . ARG FROM_TAG=latest -FROM clickhouse/stateless-test:$FROM_TAG - -RUN apt-get install gdb +FROM clickhouse/test-base:$FROM_TAG COPY run.sh / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 1f2cc9903b2..a4c4c75e5b3 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -25,7 +25,8 @@ azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & ./setup_minio.sh stateless # to have a proper environment echo "Get previous release tag" -previous_release_tag=$(dpkg --info package_folder/clickhouse-client*.deb | grep "Version: " | awk '{print $2}' | cut -f1 -d'+' | get_previous_release_tag) +# shellcheck disable=SC2016 +previous_release_tag=$(dpkg-deb --showformat='${Version}' --show package_folder/clickhouse-client*.deb | get_previous_release_tag) echo $previous_release_tag echo "Clone previous release repository" diff --git a/docs/changelogs/v23.8.15.35-lts.md b/docs/changelogs/v23.8.15.35-lts.md new file mode 100644 index 00000000000..bab5c507fe8 --- /dev/null +++ b/docs/changelogs/v23.8.15.35-lts.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.15.35-lts (060ff8e813a) FIXME as compared to v23.8.14.6-lts (967e51c1d6b) + +#### Build/Testing/Packaging Improvement +* Backported in [#63621](https://github.com/ClickHouse/ClickHouse/issues/63621): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#65153](https://github.com/ClickHouse/ClickHouse/issues/65153): Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64422](https://github.com/ClickHouse/ClickHouse/issues/64422): Fixes [#59989](https://github.com/ClickHouse/ClickHouse/issues/59989): runs init scripts when force-enabled or when no database exists, rather than the inverse. [#59991](https://github.com/ClickHouse/ClickHouse/pull/59991) ([jktng](https://github.com/jktng)). +* Backported in [#64016](https://github.com/ClickHouse/ClickHouse/issues/64016): Fix "Invalid storage definition in metadata file" for parameterized views. [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#63456](https://github.com/ClickHouse/ClickHouse/issues/63456): Fix the issue where the function `addDays` (and similar functions) reports an error when the first parameter is `DateTime64`. [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)). +* Backported in [#63289](https://github.com/ClickHouse/ClickHouse/issues/63289): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63512](https://github.com/ClickHouse/ClickHouse/issues/63512): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63902](https://github.com/ClickHouse/ClickHouse/issues/63902): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#64104](https://github.com/ClickHouse/ClickHouse/issues/64104): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64265](https://github.com/ClickHouse/ClickHouse/issues/64265): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64867](https://github.com/ClickHouse/ClickHouse/issues/64867): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). + +#### NO CL CATEGORY + +* Backported in [#63704](https://github.com/ClickHouse/ClickHouse/issues/63704):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Installation test has wrong check_state'. [#63994](https://github.com/ClickHouse/ClickHouse/pull/63994) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#63343](https://github.com/ClickHouse/ClickHouse/issues/63343): The commit url has different pattern. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63965](https://github.com/ClickHouse/ClickHouse/issues/63965): fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64043](https://github.com/ClickHouse/ClickHouse/issues/64043): Do not create new release in release branch automatically. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Pin requests to fix the integration tests. [#65183](https://github.com/ClickHouse/ClickHouse/pull/65183) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/docs/changelogs/v24.1.6.52-stable.md b/docs/changelogs/v24.1.6.52-stable.md new file mode 100644 index 00000000000..341561e9a64 --- /dev/null +++ b/docs/changelogs/v24.1.6.52-stable.md @@ -0,0 +1,45 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.1.6.52-stable (fa09f677bc9) FIXME as compared to v24.1.5.6-stable (7f67181ff31) + +#### Improvement +* Backported in [#60292](https://github.com/ClickHouse/ClickHouse/issues/60292): Copy S3 file GCP fallback to buffer copy in case GCP returned `Internal Error` with `GATEWAY_TIMEOUT` HTTP error code. [#60164](https://github.com/ClickHouse/ClickHouse/pull/60164) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#60832](https://github.com/ClickHouse/ClickHouse/issues/60832): Update tzdata to 2024a. [#60768](https://github.com/ClickHouse/ClickHouse/pull/60768) ([Raúl Marín](https://github.com/Algunenano)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#60413](https://github.com/ClickHouse/ClickHouse/issues/60413): Fix segmentation fault in KQL parser when the input query exceeds the `max_query_size`. Also re-enable the KQL dialect. Fixes [#59036](https://github.com/ClickHouse/ClickHouse/issues/59036) and [#59037](https://github.com/ClickHouse/ClickHouse/issues/59037). [#59626](https://github.com/ClickHouse/ClickHouse/pull/59626) ([Yong Wang](https://github.com/kashwy)). +* Backported in [#60074](https://github.com/ClickHouse/ClickHouse/issues/60074): Fix error `Read beyond last offset` for `AsynchronousBoundedReadBuffer`. [#59630](https://github.com/ClickHouse/ClickHouse/pull/59630) ([Vitaly Baranov](https://github.com/vitlibar)). +* Backported in [#60299](https://github.com/ClickHouse/ClickHouse/issues/60299): Fix having neigher acked nor nacked messages. If exception happens during read-write phase, messages will be nacked. [#59775](https://github.com/ClickHouse/ClickHouse/pull/59775) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#60066](https://github.com/ClickHouse/ClickHouse/issues/60066): Fix optimize_uniq_to_count removing the column alias. [#60026](https://github.com/ClickHouse/ClickHouse/pull/60026) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60638](https://github.com/ClickHouse/ClickHouse/issues/60638): Fixed a bug in parallel optimization for queries with `FINAL`, which could give an incorrect result in rare cases. [#60041](https://github.com/ClickHouse/ClickHouse/pull/60041) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#60177](https://github.com/ClickHouse/ClickHouse/issues/60177): Fix cosineDistance crash with Nullable. [#60150](https://github.com/ClickHouse/ClickHouse/pull/60150) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60279](https://github.com/ClickHouse/ClickHouse/issues/60279): Hide sensitive info for `S3Queue` table engine. [#60233](https://github.com/ClickHouse/ClickHouse/pull/60233) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#61000](https://github.com/ClickHouse/ClickHouse/issues/61000): Reduce the number of read rows from `system.numbers`. Fixes [#59418](https://github.com/ClickHouse/ClickHouse/issues/59418). [#60546](https://github.com/ClickHouse/ClickHouse/pull/60546) ([JackyWoo](https://github.com/JackyWoo)). +* Backported in [#60791](https://github.com/ClickHouse/ClickHouse/issues/60791): Fix buffer overflow that can happen if the attacker asks the HTTP server to decompress data with a composition of codecs and size triggering numeric overflow. Fix buffer overflow that can happen inside codec NONE on wrong input data. This was submitted by TIANGONG research team through our [Bug Bounty program](https://github.com/ClickHouse/ClickHouse/issues/38986). [#60731](https://github.com/ClickHouse/ClickHouse/pull/60731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#60783](https://github.com/ClickHouse/ClickHouse/issues/60783): Functions for SQL/JSON were able to read uninitialized memory. This closes [#60017](https://github.com/ClickHouse/ClickHouse/issues/60017). Found by Fuzzer. [#60738](https://github.com/ClickHouse/ClickHouse/pull/60738) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#60803](https://github.com/ClickHouse/ClickHouse/issues/60803): Do not set aws custom metadata `x-amz-meta-*` headers on UploadPart & CompleteMultipartUpload calls. [#60748](https://github.com/ClickHouse/ClickHouse/pull/60748) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Backported in [#60820](https://github.com/ClickHouse/ClickHouse/issues/60820): Fix crash in arrayEnumerateRanked. [#60764](https://github.com/ClickHouse/ClickHouse/pull/60764) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60841](https://github.com/ClickHouse/ClickHouse/issues/60841): Fix crash when using input() in INSERT SELECT JOIN. Closes [#60035](https://github.com/ClickHouse/ClickHouse/issues/60035). [#60765](https://github.com/ClickHouse/ClickHouse/pull/60765) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#60904](https://github.com/ClickHouse/ClickHouse/issues/60904): Avoid segfault if too many keys are skipped when reading from S3. [#60849](https://github.com/ClickHouse/ClickHouse/pull/60849) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NO CL CATEGORY + +* Backported in [#60186](https://github.com/ClickHouse/ClickHouse/issues/60186):. [#60181](https://github.com/ClickHouse/ClickHouse/pull/60181) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#60333](https://github.com/ClickHouse/ClickHouse/issues/60333): CI: Fix job failures due to jepsen artifacts. [#59890](https://github.com/ClickHouse/ClickHouse/pull/59890) ([Max K.](https://github.com/maxknv)). +* Backported in [#60034](https://github.com/ClickHouse/ClickHouse/issues/60034): Fix mark release ready. [#59994](https://github.com/ClickHouse/ClickHouse/pull/59994) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#60326](https://github.com/ClickHouse/ClickHouse/issues/60326): Ability to detect undead ZooKeeper sessions. [#60044](https://github.com/ClickHouse/ClickHouse/pull/60044) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#60363](https://github.com/ClickHouse/ClickHouse/issues/60363): CI: hot fix for gh statuses. [#60201](https://github.com/ClickHouse/ClickHouse/pull/60201) ([Max K.](https://github.com/maxknv)). +* Backported in [#60648](https://github.com/ClickHouse/ClickHouse/issues/60648): Detect io_uring in tests. [#60373](https://github.com/ClickHouse/ClickHouse/pull/60373) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#60569](https://github.com/ClickHouse/ClickHouse/issues/60569): Remove broken test while we fix it. [#60547](https://github.com/ClickHouse/ClickHouse/pull/60547) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#60756](https://github.com/ClickHouse/ClickHouse/issues/60756): Update shellcheck. [#60553](https://github.com/ClickHouse/ClickHouse/pull/60553) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#60584](https://github.com/ClickHouse/ClickHouse/issues/60584): CI: fix docker build job name. [#60554](https://github.com/ClickHouse/ClickHouse/pull/60554) ([Max K.](https://github.com/maxknv)). + diff --git a/docs/changelogs/v24.3.4.147-lts.md b/docs/changelogs/v24.3.4.147-lts.md new file mode 100644 index 00000000000..7d77fb29977 --- /dev/null +++ b/docs/changelogs/v24.3.4.147-lts.md @@ -0,0 +1,100 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.4.147-lts (31a7bdc346d) FIXME as compared to v24.3.3.102-lts (7e7f3bdd9be) + +#### Improvement +* Backported in [#63465](https://github.com/ClickHouse/ClickHouse/issues/63465): Make rabbitmq nack broken messages. Closes [#45350](https://github.com/ClickHouse/ClickHouse/issues/45350). [#60312](https://github.com/ClickHouse/ClickHouse/pull/60312) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#64290](https://github.com/ClickHouse/ClickHouse/issues/64290): Fix logical-error when undoing quorum insert transaction. [#61953](https://github.com/ClickHouse/ClickHouse/pull/61953) ([Han Fei](https://github.com/hanfei1991)). + +#### Build/Testing/Packaging Improvement +* Backported in [#63610](https://github.com/ClickHouse/ClickHouse/issues/63610): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#65128](https://github.com/ClickHouse/ClickHouse/issues/65128): Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64277](https://github.com/ClickHouse/ClickHouse/issues/64277): Fix queries with FINAL give wrong result when table does not use adaptive granularity. [#62432](https://github.com/ClickHouse/ClickHouse/pull/62432) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#63716](https://github.com/ClickHouse/ClickHouse/issues/63716): Fix excessive memory usage for queries with nested lambdas. Fixes [#62036](https://github.com/ClickHouse/ClickHouse/issues/62036). [#62462](https://github.com/ClickHouse/ClickHouse/pull/62462) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63247](https://github.com/ClickHouse/ClickHouse/issues/63247): Fix size checks when updating materialized nested columns ( fixes [#62731](https://github.com/ClickHouse/ClickHouse/issues/62731) ). [#62773](https://github.com/ClickHouse/ClickHouse/pull/62773) ([Eliot Hautefeuille](https://github.com/hileef)). +* Backported in [#62984](https://github.com/ClickHouse/ClickHouse/issues/62984): Fix the `Unexpected return type` error for queries that read from `StorageBuffer` with `PREWHERE` when the source table has different types. Fixes [#62545](https://github.com/ClickHouse/ClickHouse/issues/62545). [#62916](https://github.com/ClickHouse/ClickHouse/pull/62916) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63185](https://github.com/ClickHouse/ClickHouse/issues/63185): Sanity check: Clamp values instead of throwing. [#63119](https://github.com/ClickHouse/ClickHouse/pull/63119) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63293](https://github.com/ClickHouse/ClickHouse/issues/63293): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63411](https://github.com/ClickHouse/ClickHouse/issues/63411): Fix a misbehavior when SQL security defaults don't load for old tables during server startup. [#63209](https://github.com/ClickHouse/ClickHouse/pull/63209) ([pufit](https://github.com/pufit)). +* Backported in [#63616](https://github.com/ClickHouse/ClickHouse/issues/63616): Fix bug which could potentially lead to rare LOGICAL_ERROR during SELECT query with message: `Unexpected return type from materialize. Expected type_XXX. Got type_YYY.` Introduced in [#59379](https://github.com/ClickHouse/ClickHouse/issues/59379). [#63353](https://github.com/ClickHouse/ClickHouse/pull/63353) ([alesapin](https://github.com/alesapin)). +* Backported in [#63455](https://github.com/ClickHouse/ClickHouse/issues/63455): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63603](https://github.com/ClickHouse/ClickHouse/issues/63603): Fix backup of projection part in case projection was removed from table metadata, but part still has projection. [#63426](https://github.com/ClickHouse/ClickHouse/pull/63426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#63508](https://github.com/ClickHouse/ClickHouse/issues/63508): Fix 'Every derived table must have its own alias' error for MYSQL dictionary source, close [#63341](https://github.com/ClickHouse/ClickHouse/issues/63341). [#63481](https://github.com/ClickHouse/ClickHouse/pull/63481) ([vdimir](https://github.com/vdimir)). +* Backported in [#63595](https://github.com/ClickHouse/ClickHouse/issues/63595): Avoid segafult in `MergeTreePrefetchedReadPool` while fetching projection parts. [#63513](https://github.com/ClickHouse/ClickHouse/pull/63513) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63748](https://github.com/ClickHouse/ClickHouse/issues/63748): Read only the necessary columns from VIEW (new analyzer). Closes [#62594](https://github.com/ClickHouse/ClickHouse/issues/62594). [#63688](https://github.com/ClickHouse/ClickHouse/pull/63688) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63770](https://github.com/ClickHouse/ClickHouse/issues/63770): Fix [#63539](https://github.com/ClickHouse/ClickHouse/issues/63539). Forbid WINDOW redefinition in new analyzer. [#63694](https://github.com/ClickHouse/ClickHouse/pull/63694) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64189](https://github.com/ClickHouse/ClickHouse/issues/64189): Fix `Not found column` and `CAST AS Map from array requires nested tuple of 2 elements` exceptions for distributed queries which use `Map(Nothing, Nothing)` type. Fixes [#63637](https://github.com/ClickHouse/ClickHouse/issues/63637). [#63753](https://github.com/ClickHouse/ClickHouse/pull/63753) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63845](https://github.com/ClickHouse/ClickHouse/issues/63845): Fix possible `ILLEGAL_COLUMN` error in `partial_merge` join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#63755](https://github.com/ClickHouse/ClickHouse/pull/63755) ([vdimir](https://github.com/vdimir)). +* Backported in [#63906](https://github.com/ClickHouse/ClickHouse/issues/63906): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#63989](https://github.com/ClickHouse/ClickHouse/issues/63989): Fix incorrect select query result when parallel replicas were used to read from a Materialized View. [#63861](https://github.com/ClickHouse/ClickHouse/pull/63861) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#64031](https://github.com/ClickHouse/ClickHouse/issues/64031): Fix a error `Database name is empty` for remote queries with lambdas over the cluster with modified default database. Fixes [#63471](https://github.com/ClickHouse/ClickHouse/issues/63471). [#63864](https://github.com/ClickHouse/ClickHouse/pull/63864) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64559](https://github.com/ClickHouse/ClickHouse/issues/64559): Fix SIGSEGV due to CPU/Real (`query_profiler_real_time_period_ns`/`query_profiler_cpu_time_period_ns`) profiler (has been an issue since 2022, that leads to periodic server crashes, especially if you were using distributed engine). [#63865](https://github.com/ClickHouse/ClickHouse/pull/63865) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64009](https://github.com/ClickHouse/ClickHouse/issues/64009): Fix analyzer - IN function with arbitrary deep sub-selects in materialized view to use insertion block. [#63930](https://github.com/ClickHouse/ClickHouse/pull/63930) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64236](https://github.com/ClickHouse/ClickHouse/issues/64236): Fix resolve of unqualified COLUMNS matcher. Preserve the input columns order and forbid usage of unknown identifiers. [#63962](https://github.com/ClickHouse/ClickHouse/pull/63962) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64106](https://github.com/ClickHouse/ClickHouse/issues/64106): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64168](https://github.com/ClickHouse/ClickHouse/issues/64168): Add missing settings to recoverLostReplica. [#64040](https://github.com/ClickHouse/ClickHouse/pull/64040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64320](https://github.com/ClickHouse/ClickHouse/issues/64320): This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline Closes [#63777](https://github.com/ClickHouse/ClickHouse/issues/63777). [#64079](https://github.com/ClickHouse/ClickHouse/pull/64079) ([pufit](https://github.com/pufit)). +* Backported in [#64380](https://github.com/ClickHouse/ClickHouse/issues/64380): Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. [#64096](https://github.com/ClickHouse/ClickHouse/pull/64096) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64567](https://github.com/ClickHouse/ClickHouse/issues/64567): Fix creating backups to S3 buckets with different credentials from the disk containing the file. [#64153](https://github.com/ClickHouse/ClickHouse/pull/64153) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64270](https://github.com/ClickHouse/ClickHouse/issues/64270): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64339](https://github.com/ClickHouse/ClickHouse/issues/64339): The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. [#64199](https://github.com/ClickHouse/ClickHouse/pull/64199) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64259](https://github.com/ClickHouse/ClickHouse/issues/64259): Ignore `text_log` config when using Keeper. [#64218](https://github.com/ClickHouse/ClickHouse/pull/64218) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64688](https://github.com/ClickHouse/ClickHouse/issues/64688): Fix Query Tree size validation. Closes [#63701](https://github.com/ClickHouse/ClickHouse/issues/63701). [#64377](https://github.com/ClickHouse/ClickHouse/pull/64377) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64725](https://github.com/ClickHouse/ClickHouse/issues/64725): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64621](https://github.com/ClickHouse/ClickHouse/issues/64621): Fix an error `Cannot find column` in distributed queries with constant CTE in the `GROUP BY` key. [#64519](https://github.com/ClickHouse/ClickHouse/pull/64519) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64678](https://github.com/ClickHouse/ClickHouse/issues/64678): Fix [#64612](https://github.com/ClickHouse/ClickHouse/issues/64612). Do not rewrite aggregation if `-If` combinator is already used. [#64638](https://github.com/ClickHouse/ClickHouse/pull/64638) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64831](https://github.com/ClickHouse/ClickHouse/issues/64831): Fix bug which could lead to non-working TTLs with expressions. Fixes [#63700](https://github.com/ClickHouse/ClickHouse/issues/63700). [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Backported in [#64940](https://github.com/ClickHouse/ClickHouse/issues/64940): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64869](https://github.com/ClickHouse/ClickHouse/issues/64869): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64980](https://github.com/ClickHouse/ClickHouse/issues/64980): Fix the `Block structure mismatch` error for queries reading with `PREWHERE` from the materialized view when the materialized view has columns of different types than the source table. Fixes [#64611](https://github.com/ClickHouse/ClickHouse/issues/64611). [#64855](https://github.com/ClickHouse/ClickHouse/pull/64855) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64972](https://github.com/ClickHouse/ClickHouse/issues/64972): Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. [#64858](https://github.com/ClickHouse/ClickHouse/pull/64858) ([alesapin](https://github.com/alesapin)). +* Backported in [#65070](https://github.com/ClickHouse/ClickHouse/issues/65070): Fix `ALTER MODIFY COMMENT` query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. [#65031](https://github.com/ClickHouse/ClickHouse/pull/65031) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#65175](https://github.com/ClickHouse/ClickHouse/issues/65175): Fix the `Unknown expression identifier` error for remote queries with `INTERPOLATE (alias)` (new analyzer). Fixes [#64636](https://github.com/ClickHouse/ClickHouse/issues/64636). [#65090](https://github.com/ClickHouse/ClickHouse/pull/65090) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64587](https://github.com/ClickHouse/ClickHouse/issues/64587): Disabled `enable_vertical_final` setting by default. This feature should not be used because it has a bug: [#64543](https://github.com/ClickHouse/ClickHouse/issues/64543). [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#64878](https://github.com/ClickHouse/ClickHouse/issues/64878): This PR fixes an error when a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). + +#### NO CL CATEGORY + +* Backported in [#63304](https://github.com/ClickHouse/ClickHouse/issues/63304):. [#63297](https://github.com/ClickHouse/ClickHouse/pull/63297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#63708](https://github.com/ClickHouse/ClickHouse/issues/63708):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#64363](https://github.com/ClickHouse/ClickHouse/issues/64363) to 24.3: Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts"'. [#64907](https://github.com/ClickHouse/ClickHouse/pull/64907) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#63751](https://github.com/ClickHouse/ClickHouse/issues/63751): group_by_use_nulls strikes back. [#62922](https://github.com/ClickHouse/ClickHouse/pull/62922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63558](https://github.com/ClickHouse/ClickHouse/issues/63558): Try fix segfault in `MergeTreeReadPoolBase::createTask`. [#63323](https://github.com/ClickHouse/ClickHouse/pull/63323) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63336](https://github.com/ClickHouse/ClickHouse/issues/63336): The commit url has different pattern. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63374](https://github.com/ClickHouse/ClickHouse/issues/63374): Add tags for the test 03000_traverse_shadow_system_data_paths.sql to make it stable. [#63366](https://github.com/ClickHouse/ClickHouse/pull/63366) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#63625](https://github.com/ClickHouse/ClickHouse/issues/63625): Workaround for `oklch()` inside canvas bug for firefox. [#63404](https://github.com/ClickHouse/ClickHouse/pull/63404) ([Sergei Trifonov](https://github.com/serxa)). +* Backported in [#63569](https://github.com/ClickHouse/ClickHouse/issues/63569): Add `jwcrypto` to integration tests runner. [#63551](https://github.com/ClickHouse/ClickHouse/pull/63551) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#63649](https://github.com/ClickHouse/ClickHouse/issues/63649): Fix `02362_part_log_merge_algorithm` flaky test. [#63635](https://github.com/ClickHouse/ClickHouse/pull/63635) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#63762](https://github.com/ClickHouse/ClickHouse/issues/63762): Cancel S3 reads properly when parallel reads are used. [#63687](https://github.com/ClickHouse/ClickHouse/pull/63687) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63741](https://github.com/ClickHouse/ClickHouse/issues/63741): Userspace page cache: don't collect stats if cache is unused. [#63730](https://github.com/ClickHouse/ClickHouse/pull/63730) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#63826](https://github.com/ClickHouse/ClickHouse/issues/63826): Fix `test_odbc_interaction` for arm64 on linux. [#63787](https://github.com/ClickHouse/ClickHouse/pull/63787) ([alesapin](https://github.com/alesapin)). +* Backported in [#63895](https://github.com/ClickHouse/ClickHouse/issues/63895): Fix `test_catboost_evaluate` for aarch64. [#63789](https://github.com/ClickHouse/ClickHouse/pull/63789) ([alesapin](https://github.com/alesapin)). +* Backported in [#63887](https://github.com/ClickHouse/ClickHouse/issues/63887): Fix `test_disk_types` for aarch64. [#63832](https://github.com/ClickHouse/ClickHouse/pull/63832) ([alesapin](https://github.com/alesapin)). +* Backported in [#63879](https://github.com/ClickHouse/ClickHouse/issues/63879): Fix `test_short_strings_aggregation` for arm. [#63836](https://github.com/ClickHouse/ClickHouse/pull/63836) ([alesapin](https://github.com/alesapin)). +* Backported in [#63916](https://github.com/ClickHouse/ClickHouse/issues/63916): Disable `test_non_default_compression/test.py::test_preconfigured_deflateqpl_codec` on arm. [#63839](https://github.com/ClickHouse/ClickHouse/pull/63839) ([alesapin](https://github.com/alesapin)). +* Backported in [#63969](https://github.com/ClickHouse/ClickHouse/issues/63969): fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64047](https://github.com/ClickHouse/ClickHouse/issues/64047): Do not create new release in release branch automatically. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64076](https://github.com/ClickHouse/ClickHouse/issues/64076): Files without shebang have mime 'text/plain' or 'inode/x-empty'. [#64062](https://github.com/ClickHouse/ClickHouse/pull/64062) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64142](https://github.com/ClickHouse/ClickHouse/issues/64142): Fix sanitizers. [#64090](https://github.com/ClickHouse/ClickHouse/pull/64090) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64159](https://github.com/ClickHouse/ClickHouse/issues/64159): Add retries in `git submodule update`. [#64125](https://github.com/ClickHouse/ClickHouse/pull/64125) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#64473](https://github.com/ClickHouse/ClickHouse/issues/64473): Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts. [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65113](https://github.com/ClickHouse/ClickHouse/issues/65113): Adjust the `version_helper` and script to a new release scheme. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64999](https://github.com/ClickHouse/ClickHouse/issues/64999): Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). + diff --git a/docs/changelogs/v24.4.2.141-stable.md b/docs/changelogs/v24.4.2.141-stable.md new file mode 100644 index 00000000000..656d0854392 --- /dev/null +++ b/docs/changelogs/v24.4.2.141-stable.md @@ -0,0 +1,101 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.2.141-stable (9e23d27bd11) FIXME as compared to v24.4.1.2088-stable (6d4b31322d1) + +#### Improvement +* Backported in [#63467](https://github.com/ClickHouse/ClickHouse/issues/63467): Make rabbitmq nack broken messages. Closes [#45350](https://github.com/ClickHouse/ClickHouse/issues/45350). [#60312](https://github.com/ClickHouse/ClickHouse/pull/60312) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Build/Testing/Packaging Improvement +* Backported in [#63612](https://github.com/ClickHouse/ClickHouse/issues/63612): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64279](https://github.com/ClickHouse/ClickHouse/issues/64279): Fix queries with FINAL give wrong result when table does not use adaptive granularity. [#62432](https://github.com/ClickHouse/ClickHouse/pull/62432) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#63295](https://github.com/ClickHouse/ClickHouse/issues/63295): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63978](https://github.com/ClickHouse/ClickHouse/issues/63978): Fix intersect parts when restart after drop range. [#63202](https://github.com/ClickHouse/ClickHouse/pull/63202) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#63413](https://github.com/ClickHouse/ClickHouse/issues/63413): Fix a misbehavior when SQL security defaults don't load for old tables during server startup. [#63209](https://github.com/ClickHouse/ClickHouse/pull/63209) ([pufit](https://github.com/pufit)). +* Backported in [#63388](https://github.com/ClickHouse/ClickHouse/issues/63388): JOIN filter push down filled join fix. Closes [#63228](https://github.com/ClickHouse/ClickHouse/issues/63228). [#63234](https://github.com/ClickHouse/ClickHouse/pull/63234) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63618](https://github.com/ClickHouse/ClickHouse/issues/63618): Fix bug which could potentially lead to rare LOGICAL_ERROR during SELECT query with message: `Unexpected return type from materialize. Expected type_XXX. Got type_YYY.` Introduced in [#59379](https://github.com/ClickHouse/ClickHouse/issues/59379). [#63353](https://github.com/ClickHouse/ClickHouse/pull/63353) ([alesapin](https://github.com/alesapin)). +* Backported in [#63451](https://github.com/ClickHouse/ClickHouse/issues/63451): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63605](https://github.com/ClickHouse/ClickHouse/issues/63605): Fix backup of projection part in case projection was removed from table metadata, but part still has projection. [#63426](https://github.com/ClickHouse/ClickHouse/pull/63426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#63510](https://github.com/ClickHouse/ClickHouse/issues/63510): Fix 'Every derived table must have its own alias' error for MYSQL dictionary source, close [#63341](https://github.com/ClickHouse/ClickHouse/issues/63341). [#63481](https://github.com/ClickHouse/ClickHouse/pull/63481) ([vdimir](https://github.com/vdimir)). +* Backported in [#63592](https://github.com/ClickHouse/ClickHouse/issues/63592): Avoid segafult in `MergeTreePrefetchedReadPool` while fetching projection parts. [#63513](https://github.com/ClickHouse/ClickHouse/pull/63513) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63750](https://github.com/ClickHouse/ClickHouse/issues/63750): Read only the necessary columns from VIEW (new analyzer). Closes [#62594](https://github.com/ClickHouse/ClickHouse/issues/62594). [#63688](https://github.com/ClickHouse/ClickHouse/pull/63688) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63772](https://github.com/ClickHouse/ClickHouse/issues/63772): Fix [#63539](https://github.com/ClickHouse/ClickHouse/issues/63539). Forbid WINDOW redefinition in new analyzer. [#63694](https://github.com/ClickHouse/ClickHouse/pull/63694) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#63872](https://github.com/ClickHouse/ClickHouse/issues/63872): Flatten_nested is broken with replicated database. [#63695](https://github.com/ClickHouse/ClickHouse/pull/63695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63854](https://github.com/ClickHouse/ClickHouse/issues/63854): Fix `Not found column` and `CAST AS Map from array requires nested tuple of 2 elements` exceptions for distributed queries which use `Map(Nothing, Nothing)` type. Fixes [#63637](https://github.com/ClickHouse/ClickHouse/issues/63637). [#63753](https://github.com/ClickHouse/ClickHouse/pull/63753) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63847](https://github.com/ClickHouse/ClickHouse/issues/63847): Fix possible `ILLEGAL_COLUMN` error in `partial_merge` join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#63755](https://github.com/ClickHouse/ClickHouse/pull/63755) ([vdimir](https://github.com/vdimir)). +* Backported in [#63908](https://github.com/ClickHouse/ClickHouse/issues/63908): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#63955](https://github.com/ClickHouse/ClickHouse/issues/63955): Fix possible crash with SYSTEM UNLOAD PRIMARY KEY. [#63778](https://github.com/ClickHouse/ClickHouse/pull/63778) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63938](https://github.com/ClickHouse/ClickHouse/issues/63938): Allow JOIN filter push down to both streams if only single equivalent column is used in query. Closes [#63799](https://github.com/ClickHouse/ClickHouse/issues/63799). [#63819](https://github.com/ClickHouse/ClickHouse/pull/63819) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63991](https://github.com/ClickHouse/ClickHouse/issues/63991): Fix incorrect select query result when parallel replicas were used to read from a Materialized View. [#63861](https://github.com/ClickHouse/ClickHouse/pull/63861) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#64033](https://github.com/ClickHouse/ClickHouse/issues/64033): Fix a error `Database name is empty` for remote queries with lambdas over the cluster with modified default database. Fixes [#63471](https://github.com/ClickHouse/ClickHouse/issues/63471). [#63864](https://github.com/ClickHouse/ClickHouse/pull/63864) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64561](https://github.com/ClickHouse/ClickHouse/issues/64561): Fix SIGSEGV due to CPU/Real (`query_profiler_real_time_period_ns`/`query_profiler_cpu_time_period_ns`) profiler (has been an issue since 2022, that leads to periodic server crashes, especially if you were using distributed engine). [#63865](https://github.com/ClickHouse/ClickHouse/pull/63865) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64011](https://github.com/ClickHouse/ClickHouse/issues/64011): Fix analyzer - IN function with arbitrary deep sub-selects in materialized view to use insertion block. [#63930](https://github.com/ClickHouse/ClickHouse/pull/63930) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64238](https://github.com/ClickHouse/ClickHouse/issues/64238): Fix resolve of unqualified COLUMNS matcher. Preserve the input columns order and forbid usage of unknown identifiers. [#63962](https://github.com/ClickHouse/ClickHouse/pull/63962) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64103](https://github.com/ClickHouse/ClickHouse/issues/64103): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64170](https://github.com/ClickHouse/ClickHouse/issues/64170): Add missing settings to recoverLostReplica. [#64040](https://github.com/ClickHouse/ClickHouse/pull/64040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64322](https://github.com/ClickHouse/ClickHouse/issues/64322): This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline Closes [#63777](https://github.com/ClickHouse/ClickHouse/issues/63777). [#64079](https://github.com/ClickHouse/ClickHouse/pull/64079) ([pufit](https://github.com/pufit)). +* Backported in [#64382](https://github.com/ClickHouse/ClickHouse/issues/64382): Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. [#64096](https://github.com/ClickHouse/ClickHouse/pull/64096) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64568](https://github.com/ClickHouse/ClickHouse/issues/64568): Fix creating backups to S3 buckets with different credentials from the disk containing the file. [#64153](https://github.com/ClickHouse/ClickHouse/pull/64153) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64272](https://github.com/ClickHouse/ClickHouse/issues/64272): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64330](https://github.com/ClickHouse/ClickHouse/issues/64330): The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. [#64199](https://github.com/ClickHouse/ClickHouse/pull/64199) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64254](https://github.com/ClickHouse/ClickHouse/issues/64254): Ignore `text_log` config when using Keeper. [#64218](https://github.com/ClickHouse/ClickHouse/pull/64218) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64690](https://github.com/ClickHouse/ClickHouse/issues/64690): Fix Query Tree size validation. Closes [#63701](https://github.com/ClickHouse/ClickHouse/issues/63701). [#64377](https://github.com/ClickHouse/ClickHouse/pull/64377) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64409](https://github.com/ClickHouse/ClickHouse/issues/64409): Fix `Logical error: Bad cast` for `Buffer` table with `PREWHERE`. Fixes [#64172](https://github.com/ClickHouse/ClickHouse/issues/64172). [#64388](https://github.com/ClickHouse/ClickHouse/pull/64388) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64727](https://github.com/ClickHouse/ClickHouse/issues/64727): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64623](https://github.com/ClickHouse/ClickHouse/issues/64623): Fix an error `Cannot find column` in distributed queries with constant CTE in the `GROUP BY` key. [#64519](https://github.com/ClickHouse/ClickHouse/pull/64519) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64680](https://github.com/ClickHouse/ClickHouse/issues/64680): Fix [#64612](https://github.com/ClickHouse/ClickHouse/issues/64612). Do not rewrite aggregation if `-If` combinator is already used. [#64638](https://github.com/ClickHouse/ClickHouse/pull/64638) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64942](https://github.com/ClickHouse/ClickHouse/issues/64942): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64871](https://github.com/ClickHouse/ClickHouse/issues/64871): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#63364](https://github.com/ClickHouse/ClickHouse/issues/63364): Implement cumulative A Sync status. [#61464](https://github.com/ClickHouse/ClickHouse/pull/61464) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63338](https://github.com/ClickHouse/ClickHouse/issues/63338): Use `/commit/` to have the URLs in [reports](https://play.clickhouse.com/play?user=play#c2VsZWN0IGRpc3RpbmN0IGNvbW1pdF91cmwgZnJvbSBjaGVja3Mgd2hlcmUgY2hlY2tfc3RhcnRfdGltZSA+PSBub3coKSAtIGludGVydmFsIDEgbW9udGggYW5kIHB1bGxfcmVxdWVzdF9udW1iZXI9NjA1MzI=) like https://github.com/ClickHouse/ClickHouse/commit/44f8bc5308b53797bec8cccc3bd29fab8a00235d and not like https://github.com/ClickHouse/ClickHouse/commits/44f8bc5308b53797bec8cccc3bd29fab8a00235d. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63376](https://github.com/ClickHouse/ClickHouse/issues/63376):. [#63366](https://github.com/ClickHouse/ClickHouse/pull/63366) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#63571](https://github.com/ClickHouse/ClickHouse/issues/63571):. [#63551](https://github.com/ClickHouse/ClickHouse/pull/63551) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#63651](https://github.com/ClickHouse/ClickHouse/issues/63651): Fix 02362_part_log_merge_algorithm flaky test. [#63635](https://github.com/ClickHouse/ClickHouse/pull/63635) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#63828](https://github.com/ClickHouse/ClickHouse/issues/63828): Fix test_odbc_interaction from aarch64 [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63787](https://github.com/ClickHouse/ClickHouse/pull/63787) ([alesapin](https://github.com/alesapin)). +* Backported in [#63897](https://github.com/ClickHouse/ClickHouse/issues/63897): Fix test `test_catboost_evaluate` for aarch64. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63789](https://github.com/ClickHouse/ClickHouse/pull/63789) ([alesapin](https://github.com/alesapin)). +* Backported in [#63889](https://github.com/ClickHouse/ClickHouse/issues/63889): Remove HDFS from disks config for one integration test for arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63832](https://github.com/ClickHouse/ClickHouse/pull/63832) ([alesapin](https://github.com/alesapin)). +* Backported in [#63881](https://github.com/ClickHouse/ClickHouse/issues/63881): Bump version for old image in test_short_strings_aggregation to make it work on arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63836](https://github.com/ClickHouse/ClickHouse/pull/63836) ([alesapin](https://github.com/alesapin)). +* Backported in [#63919](https://github.com/ClickHouse/ClickHouse/issues/63919): Disable test `test_non_default_compression/test.py::test_preconfigured_deflateqpl_codec` on arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63839](https://github.com/ClickHouse/ClickHouse/pull/63839) ([alesapin](https://github.com/alesapin)). +* Backported in [#63971](https://github.com/ClickHouse/ClickHouse/issues/63971): Fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64049](https://github.com/ClickHouse/ClickHouse/issues/64049): Add `ClickHouseVersion.copy` method. Create a branch release in advance without spinning out the release to increase the stability. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64078](https://github.com/ClickHouse/ClickHouse/issues/64078): The mime type is not 100% reliable for Python and shell scripts without shebangs; add a check for file extension. [#64062](https://github.com/ClickHouse/ClickHouse/pull/64062) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64161](https://github.com/ClickHouse/ClickHouse/issues/64161): Add retries in git submodule update. [#64125](https://github.com/ClickHouse/ClickHouse/pull/64125) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64589](https://github.com/ClickHouse/ClickHouse/issues/64589): Disabled `enable_vertical_final` setting by default. This feature should not be used because it has a bug: [#64543](https://github.com/ClickHouse/ClickHouse/issues/64543). [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#64880](https://github.com/ClickHouse/ClickHouse/issues/64880): This PR fixes an error when a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). + +#### NO CL CATEGORY + +* Backported in [#63306](https://github.com/ClickHouse/ClickHouse/issues/63306):. [#63297](https://github.com/ClickHouse/ClickHouse/pull/63297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#63710](https://github.com/ClickHouse/ClickHouse/issues/63710):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#64363](https://github.com/ClickHouse/ClickHouse/issues/64363) to 24.4: Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts"'. [#64905](https://github.com/ClickHouse/ClickHouse/pull/64905) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* group_by_use_nulls strikes back [#62922](https://github.com/ClickHouse/ClickHouse/pull/62922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add `FROM` keyword to `TRUNCATE ALL TABLES` [#63241](https://github.com/ClickHouse/ClickHouse/pull/63241) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* More checks for concurrently deleted files and dirs in system.remote_data_paths [#63274](https://github.com/ClickHouse/ClickHouse/pull/63274) ([Alexander Gololobov](https://github.com/davenger)). +* Try fix segfault in `MergeTreeReadPoolBase::createTask` [#63323](https://github.com/ClickHouse/ClickHouse/pull/63323) ([Antonio Andelic](https://github.com/antonio2368)). +* Skip unaccessible table dirs in system.remote_data_paths [#63330](https://github.com/ClickHouse/ClickHouse/pull/63330) ([Alexander Gololobov](https://github.com/davenger)). +* Workaround for `oklch()` inside canvas bug for firefox [#63404](https://github.com/ClickHouse/ClickHouse/pull/63404) ([Sergei Trifonov](https://github.com/serxa)). +* Cancel S3 reads properly when parallel reads are used [#63687](https://github.com/ClickHouse/ClickHouse/pull/63687) ([Antonio Andelic](https://github.com/antonio2368)). +* Userspace page cache: don't collect stats if cache is unused [#63730](https://github.com/ClickHouse/ClickHouse/pull/63730) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix sanitizers [#64090](https://github.com/ClickHouse/ClickHouse/pull/64090) ([Azat Khuzhin](https://github.com/azat)). +* Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). +* CI: Critical bugfix category in PR template [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). + diff --git a/docs/changelogs/v24.5.2.34-stable.md b/docs/changelogs/v24.5.2.34-stable.md new file mode 100644 index 00000000000..2db05a5f5dc --- /dev/null +++ b/docs/changelogs/v24.5.2.34-stable.md @@ -0,0 +1,38 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.2.34-stable (45589aeee49) FIXME as compared to v24.5.1.1763-stable (647c154a94d) + +#### Improvement +* Backported in [#65096](https://github.com/ClickHouse/ClickHouse/issues/65096): The setting `allow_experimental_join_condition` was accidentally marked as important which may prevent distributed queries in a mixed versions cluster from being executed successfully. [#65008](https://github.com/ClickHouse/ClickHouse/pull/65008) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#65132](https://github.com/ClickHouse/ClickHouse/issues/65132): Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64729](https://github.com/ClickHouse/ClickHouse/issues/64729): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#65061](https://github.com/ClickHouse/ClickHouse/issues/65061): Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.` Fixes [#64445](https://github.com/ClickHouse/ClickHouse/issues/64445). [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088): Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). Fixes [#64575](https://github.com/ClickHouse/ClickHouse/issues/64575). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64944](https://github.com/ClickHouse/ClickHouse/issues/64944): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64873](https://github.com/ClickHouse/ClickHouse/issues/64873): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64984](https://github.com/ClickHouse/ClickHouse/issues/64984): Fix the `Block structure mismatch` error for queries reading with `PREWHERE` from the materialized view when the materialized view has columns of different types than the source table. Fixes [#64611](https://github.com/ClickHouse/ClickHouse/issues/64611). [#64855](https://github.com/ClickHouse/ClickHouse/pull/64855) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64976](https://github.com/ClickHouse/ClickHouse/issues/64976): Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. [#64858](https://github.com/ClickHouse/ClickHouse/pull/64858) ([alesapin](https://github.com/alesapin)). +* Backported in [#65074](https://github.com/ClickHouse/ClickHouse/issues/65074): Fix `ALTER MODIFY COMMENT` query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. [#65031](https://github.com/ClickHouse/ClickHouse/pull/65031) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#65179](https://github.com/ClickHouse/ClickHouse/issues/65179): Fix the `Unknown expression identifier` error for remote queries with `INTERPOLATE (alias)` (new analyzer). Fixes [#64636](https://github.com/ClickHouse/ClickHouse/issues/64636). [#65090](https://github.com/ClickHouse/ClickHouse/pull/65090) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65163](https://github.com/ClickHouse/ClickHouse/issues/65163): Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. Part of [#62245](https://github.com/ClickHouse/ClickHouse/issues/62245). [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64882](https://github.com/ClickHouse/ClickHouse/issues/64882): This PR fixes an error when a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65002](https://github.com/ClickHouse/ClickHouse/issues/65002): Be more graceful with existing tables with `inverted` indexes. [#64656](https://github.com/ClickHouse/ClickHouse/pull/64656) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#65115](https://github.com/ClickHouse/ClickHouse/issues/65115): Adjust the `version_helper` and script to a new release scheme. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64796](https://github.com/ClickHouse/ClickHouse/issues/64796): Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). + diff --git a/docs/changelogs/v24.5.3.5-stable.md b/docs/changelogs/v24.5.3.5-stable.md new file mode 100644 index 00000000000..4606e58d0a4 --- /dev/null +++ b/docs/changelogs/v24.5.3.5-stable.md @@ -0,0 +1,14 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.3.5-stable (e0eb66f8e17) FIXME as compared to v24.5.2.34-stable (45589aeee49) + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65227](https://github.com/ClickHouse/ClickHouse/issues/65227): Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#65219](https://github.com/ClickHouse/ClickHouse/issues/65219): Fix false positives leaky memory warnings in OpenSSL. [#65125](https://github.com/ClickHouse/ClickHouse/pull/65125) ([Robert Schulze](https://github.com/rschu1ze)). + diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 91253ca5e44..c283cfbf4c2 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -71,7 +71,7 @@ If it fails, fix the style errors following the [code style guide](style.md). ```sh mkdir -p /tmp/test_output # running all checks -docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test +python3 tests/ci/style_check.py --no-push # run specified check script (e.g.: ./check-mypy) docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-mypy @@ -91,6 +91,9 @@ cd ./utils/check-style # Check python type hinting with mypy ./check-mypy +# Check python with flake8 +./check-flake8 + # Check code with codespell ./check-typos diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index bbc7dac0a2a..8dff6f0ed1d 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -229,6 +229,10 @@ For production builds, clang is used, but we also test make gcc builds. For deve ## Sanitizers {#sanitizers} +:::note +If the process (ClickHouse server or client) crashes at startup when running it locally, you might need to disable address space layout randomization: `sudo sysctl kernel.randomize_va_space=0` +::: + ### Address sanitizer We run functional, integration, stress and unit tests under ASan on per-commit basis. diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 0843ff1ac47..dfc27d6b8cf 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -54,6 +54,7 @@ SELECT * FROM test_table; - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## See also diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 2749fa7e479..c9df713231a 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -235,6 +235,7 @@ libhdfs3 support HDFS namenode HA. - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 9d6395f73ac..21fdbc0b1a5 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -37,7 +37,7 @@ Using named collections: http://test.s3.amazonaws.com/clickhouse-bucket/ - test + test test diff --git a/docs/en/engines/table-engines/integrations/mongodb.md b/docs/en/engines/table-engines/integrations/mongodb.md index f87e8da8b5b..5bb3bc752f5 100644 --- a/docs/en/engines/table-engines/integrations/mongodb.md +++ b/docs/en/engines/table-engines/integrations/mongodb.md @@ -34,10 +34,11 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name - `options` — MongoDB connection string options (optional parameter). :::tip -If you are using the MongoDB Atlas cloud offering please add these options: +If you are using the MongoDB Atlas cloud offering: ``` -'connectTimeoutMS=10000&ssl=true&authSource=admin' +- connection url can be obtained from 'Atlas SQL' option +- use options: 'connectTimeoutMS=10000&ssl=true&authSource=admin' ``` ::: diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index cb1da1c8e68..93f4a187656 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -53,14 +53,14 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da This example uses the [docker compose recipe](https://github.com/ClickHouse/examples/tree/5fdc6ff72f4e5137e23ea075c88d3f44b0202490/docker-compose-recipes/recipes/ch-and-minio-S3), which integrates ClickHouse and MinIO. You should be able to reproduce the same queries using S3 by replacing the endpoint and authentication values. -Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter token `{_partition_id}` as part of the S3 object (filename), and that the SELECT queries select against those resulting object names (e.g., `test_3.csv`). +Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter token `{_partition_id}` as part of the S3 object (filename), and that the SELECT queries select against those resulting object names (e.g., `test_3.csv`). :::note As shown in the example, querying from S3 tables that are partitioned is not directly supported at this time, but can be accomplished by querying the individual partitions using the S3 table function. -The primary use-case for writing +The primary use-case for writing partitioned data in S3 is to enable transferring that data into another ClickHouse system (for example, moving from on-prem systems to ClickHouse Cloud). Because ClickHouse datasets are often very large, and network @@ -78,9 +78,9 @@ CREATE TABLE p ) ENGINE = S3( # highlight-next-line - 'http://minio:10000/clickhouse//test_{_partition_id}.csv', - 'minioadmin', - 'minioadminpassword', + 'http://minio:10000/clickhouse//test_{_partition_id}.csv', + 'minioadmin', + 'minioadminpassword', 'CSV') PARTITION BY column3 ``` @@ -145,6 +145,7 @@ Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Reading fr - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index aa7fa512480..97ca79501a7 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -13,7 +13,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec CREATE TABLE s3_queue_engine_table (name String, value UInt32) ENGINE = S3Queue(path, [NOSIGN, | aws_access_key_id, aws_secret_access_key,] format, [compression]) [SETTINGS] - [mode = 'unordered',] + [mode = '',] [after_processing = 'keep',] [keeper_path = '',] [s3queue_loading_retries = 0,] @@ -75,7 +75,7 @@ Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. - ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. -Default value: `unordered`. +Default value: `ordered` in versions before 24.6. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. ### after_processing {#after_processing} @@ -181,6 +181,10 @@ For 'Ordered' mode. Defines a maximum boundary for reschedule interval for a bac Default value: `30000`. +### s3queue_buckets {#buckets} + +For 'Ordered' mode. Available since `24.6`. If there are several replicas of S3Queue table, each working with the same metadata directory in keeper, the value of `s3queue_buckets` needs to be equal to at least the number of replicas. If `s3queue_processing_threads` setting is used as well, it makes sense to increase the value of `s3queue_buckets` setting even further, as it defines the actual parallelism of `S3Queue` processing. + ## S3-related Settings {#s3-settings} Engine supports all s3 related settings. For more information about S3 settings see [here](../../../engines/table-engines/integrations/s3.md). @@ -267,7 +271,7 @@ For introspection use `system.s3queue` stateless table and `system.s3queue_log` `exception` String ) ENGINE = SystemS3Queue -COMMENT 'SYSTEM TABLE is built on the fly.' │ +COMMENT 'Contains in-memory state of S3Queue metadata and currently processed rows per file.' │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index f58a06464b2..ec4c14b6bf1 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -37,7 +37,7 @@ ways, for example with respect to their DDL/DQL syntax or performance/compressio To use full-text indexes, first enable them in the configuration: ```sql -SET allow_experimental_inverted_index = true; +SET allow_experimental_full_text_index = true; ``` An full-text index can be defined on a string column using the following syntax diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index a009c4a32f3..f0c4e1b0e34 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -6,41 +6,32 @@ sidebar_label: MergeTree # MergeTree -The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most commonly used and most robust ClickHouse table engines. +The `MergeTree` engine and other engines of the `MergeTree` family (e.g. `ReplacingMergeTree`, `AggregatingMergeTree` ) are the most commonly used and most robust table engines in ClickHouse. -Engines in the `MergeTree` family are designed for inserting a very large amount of data into a table. The data is quickly written to the table part by part, then rules are applied for merging the parts in the background. This method is much more efficient than continually rewriting the data in storage during insert. +`MergeTree`-family table engines are designed for high data ingest rates and huge data volumes. +Insert operations create table parts which are merged by a background process with other table parts. -Main features: +Main features of `MergeTree`-family table engines. -- Stores data sorted by primary key. +- The table's primary key determines the sort order within each table part (clustered index). The primary key also does not reference individual rows but blocks of 8192 rows called granules. This makes primary keys of huge data sets small enough to remain loaded in main memory, while still providing fast access to on-disk data. - This allows you to create a small sparse index that helps find data faster. +- Tables can be partitioned using an arbitrary partition expression. Partition pruning ensures partitions are omitted from reading when the query allows it. -- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. +- Data can be replicated across multiple cluster nodes for high availability, failover, and zero downtime upgrades. See [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. +- `MergeTree` table engines support various statistics kinds and sampling methods to help query optimization. -- Data replication support. - - The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - -- Data sampling support. - - If necessary, you can set the data sampling method in the table. - -:::info -The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. +:::note +Despite a similar name, the [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine is different from `*MergeTree` engines. ::: -If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) table engine. Using `ALTER TABLE my_table UPDATE` to update rows triggers a mutation, which causes parts to be re-written and uses IO/resources. With `ReplacingMergeTree`, you can simply insert the updated rows and the old rows will be replaced according to the table sorting key. - -## Creating a Table {#table_engine-mergetree-creating-a-table} +## Creating Tables {#table_engine-mergetree-creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)], - name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)], + name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTICS(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)], + name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTICS(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)], ... INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1], INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2], @@ -59,23 +50,24 @@ ORDER BY expr [SETTINGS name = value, ...] ``` -For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). +For a detailed description of the parameters, see the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md) statement ### Query Clauses {#mergetree-query-clauses} #### ENGINE -`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. +`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine has no parameters. #### ORDER_BY `ORDER BY` — The sorting key. -A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. +A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID + 1, EventDate)`. -ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause. +If no primary key is defined (i.e. `PRIMARY KEY` was not specified), ClickHouse uses the the sorting key as primary key. -Use the `ORDER BY tuple()` syntax, if you do not need sorting, or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). +If no sorting is required, you can use syntax `ORDER BY tuple()`. +Alternatively, if setting `create_table_empty_primary_key_by_default` is enabled, `ORDER BY tuple()` is implicitly added to `CREATE TABLE` statements. See [Selecting a Primary Key](#selecting-a-primary-key). #### PARTITION BY @@ -87,96 +79,32 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional. -By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. +Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key. +It is usually not necessary to specify the primary key in addition to the primary key. #### SAMPLE BY -`SAMPLE BY` — An expression for sampling. Optional. +`SAMPLE BY` — A sampling expression. Optional. -If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. +If specified, it must be contained in the primary key. +The sampling expression must result in an unsigned integer. + +Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. #### TTL -`TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. +`TTL` — A list of rules that specify the storage duration of rows and the logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. -Expression must have one `Date` or `DateTime` column as a result. Example: -``` -TTL date + INTERVAL 1 DAY -``` +Expression must result in a `Date` or `DateTime`, e.g. `TTL date + INTERVAL 1 DAY`. Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`), or aggregating values in expired rows. Default type of the rule is removal (`DELETE`). List of multiple rules can be specified, but there should be no more than one `DELETE` rule. + For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -### SETTINGS -Additional parameters that control the behavior of the `MergeTree` (optional): +#### SETTINGS -#### index_granularity - -`index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage). - -#### index_granularity_bytes - -`index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage). - -#### min_index_granularity_bytes - -`min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage). - -#### enable_mixed_granularity_parts - -`enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. - -#### use_minimalistic_part_header_in_zookeeper - -`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. - -#### min_merge_bytes_to_use_direct_io - -`min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes. - -#### merge_with_ttl_timeout - -`merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours). -#### merge_with_recompression_ttl_timeout - -`merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours). - -#### try_fetch_recompressed_part_timeout - -`try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours). - -#### write_final_mark - -`write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don’t turn it off. - -#### merge_max_block_size - -`merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192. - -#### storage_policy - -`storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes). - -#### min_bytes_for_wide_part - -`min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage). - -#### max_parts_in_total - -`max_parts_in_total` — Maximum number of parts in all partitions. - -#### max_compress_block_size - -`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### min_compress_block_size - -`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### max_partitions_to_read - -`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. +See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). **Example of Sections Setting** @@ -266,7 +194,7 @@ ClickHouse does not require a unique primary key. You can insert multiple rows w You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](/docs/en/operations/settings/settings.md/#allow-nullable-key) setting. The [NULLS_LAST](/docs/en/sql-reference/statements/select/order-by.md/#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. -### Selecting the Primary Key {#selecting-the-primary-key} +### Selecting a Primary Key {#selecting-a-primary-key} The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may: @@ -1039,12 +967,12 @@ ClickHouse versions 22.3 through 22.7 use a different cache configuration, see [ ## Column Statistics (Experimental) {#column-statistics} -The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`. +The statistics declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistics = 1`. ``` sql CREATE TABLE tab ( - a Int64 STATISTIC(tdigest), + a Int64 STATISTICS(TDigest, Uniq), b Float64 ) ENGINE = MergeTree @@ -1054,19 +982,23 @@ ORDER BY a We can also manipulate statistics with `ALTER` statements. ```sql -ALTER TABLE tab ADD STATISTIC b TYPE tdigest; -ALTER TABLE tab DROP STATISTIC a TYPE tdigest; +ALTER TABLE tab ADD STATISTICS b TYPE TDigest, Uniq; +ALTER TABLE tab DROP STATISTICS a; ``` -These lightweight statistics aggregate information about distribution of values in columns. -They can be used for query optimization when we enable `set allow_statistic_optimize = 1`. +These lightweight statistics aggregate information about distribution of values in columns. Statistics are stored in every part and updated when every insert comes. +They can be used for prewhere optimization only if we enable `set allow_statistics_optimize = 1`. #### Available Types of Column Statistics {#available-types-of-column-statistics} -- `tdigest` +- `TDigest` Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. +- `Uniq` + + Estimate the number of distinct values of a column by HyperLogLog. + ## Column-level Settings {#column-level-settings} Certain MergeTree settings can be override at column level: diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 0d422f64762..957b18b5305 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -102,6 +102,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Settings {#settings} diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index f6183a779ae..c906830d0e9 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -108,6 +108,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 66d5bd2e574..ffdd7e2ca25 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -480,7 +480,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. -- [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. +- [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - allow variable number of columns in CSV format, ignore extra columns and use default values on missing columns. Default value - `false`. - [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. - [input_format_csv_try_infer_numbers_from_strings](/docs/en/operations/settings/settings-formats.md/#input_format_csv_try_infer_numbers_from_strings) - Try to infer numbers from string fields while schema inference. Default value - `false`. @@ -1490,6 +1490,8 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000 - [output_format_pretty_color](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`. - [output_format_pretty_grid_charset](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`. - [output_format_pretty_row_numbers](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `true`. +- [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) - Display column names in the footer if table contains many rows. Default value - `true`. +- [output_format_pretty_display_footer_column_names_min_rows](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names_min_rows) - Sets the minimum number of rows for which a footer will be displayed if [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) is enabled. Default value - 50. ## RowBinary {#rowbinary} @@ -2165,6 +2167,8 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`. - [output_format_parquet_version](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_version) - The version of Parquet format used in output format. Default value - `2.latest`. - [output_format_parquet_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_compression_method) - compression method used in output Parquet format. Default value - `lz4`. +- [input_format_parquet_max_block_size](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_max_block_size) - Max block row size for parquet reader. Default value - `65409`. +- [input_format_parquet_prefer_block_bytes](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_prefer_block_bytes) - Average block bytes output by parquet reader. Default value - `16744704`. ## ParquetMetadata {data-format-parquet-metadata} diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index eb1a3ba1dbc..f5b6326fa96 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -508,7 +508,7 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`: - `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request. -- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`. +- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `http_response_headers`, `response_content`, `query`, `query_param_name`. `type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static). - `query` — use with `predefined_query_handler` type, executes query when the handler is called. @@ -519,6 +519,8 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`: - `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type). + - `http_response_headers` — use with any type, response headers map. Could be used to set content type as well. + - `response_content` — use with `static` type, response content sent to client, when using the prefix ‘file://’ or ‘config://’, find the content from the file or configuration sends to client. Next are the configuration methods for different `type`. @@ -616,6 +618,33 @@ Return a message. static 402 text/html; charset=UTF-8 + + en + 43 + + Say Hi! + + + + +``` + +`http_response_headers` could be used to set content type instead of `content_type`. + +``` xml + + + GET + xxx + /hi + + static + 402 + + text/html; charset=UTF-8 + en + 43 + Say Hi! @@ -696,6 +725,9 @@ Find the content from the file send to client. static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file:///absolute_path_file.html @@ -706,6 +738,9 @@ Find the content from the file send to client. static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file://./relative_path_file.html diff --git a/docs/en/operations/allocation-profiling.md b/docs/en/operations/allocation-profiling.md index 64b4106a7e1..574e1ae2ff3 100644 --- a/docs/en/operations/allocation-profiling.md +++ b/docs/en/operations/allocation-profiling.md @@ -59,10 +59,10 @@ For that, we need to use `jemalloc`'s tool called [jeprof](https://github.com/je If that’s the case, we recommend installing an [alternative implementation](https://github.com/gimli-rs/addr2line) of the tool. ``` -git clone https://github.com/gimli-rs/addr2line +git clone https://github.com/gimli-rs/addr2line.git --depth=1 --branch=0.23.0 cd addr2line -cargo b --examples -r -cp ./target/release/examples/addr2line path/to/current/addr2line +cargo build --features bin --release +cp ./target/release/addr2line path/to/current/addr2line ``` ::: diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md new file mode 100644 index 00000000000..298c6dacd06 --- /dev/null +++ b/docs/en/operations/analyzer.md @@ -0,0 +1,194 @@ +--- +slug: /en/operations/analyzer +sidebar_label: Analyzer +title: Analyzer +description: Details about ClickHouse's query analyzer +keywords: [analyzer] +--- + +# Analyzer + + + +## Known incompatibilities + +In ClickHouse version `24.3`, the new query analyzer was enabled by default. +Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. Please read the following changes to determine how to rewrite your queries for the new analyzer. + +### Invalid queries are no longer optimized + +The previous query planning infrastructure applied AST-level optimizations before the query validation step. +Optimizations could rewrite the initial query so it becomes valid and can be executed. + +In the new analyzer, query validation takes place before the optimization step. +This means that invalid queries that were possible to execute before are now unsupported. +In such cases, the query must be fixed manually. + +**Example 1:** + +```sql +SELECT number +FROM numbers(1) +GROUP BY toString(number) +``` + +The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. +In the old analyzer, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. + +**Example 2:** + +```sql +SELECT + number % 2 AS n, + sum(number) +FROM numbers(10) +GROUP BY n +HAVING number > 5 +``` + +The same problem occurs in this query: column `number` is used after aggregation with another key. +The previous query analyzer fixed this query by moving the `number > 5` filter from the `HAVING` clause to the `WHERE` clause. + +To fix the query, you should move all conditions that apply to non-aggregated columns to the `WHERE` section to conform to standard SQL syntax: +```sql +SELECT + number % 2 AS n, + sum(number) +FROM numbers(10) +WHERE number > 5 +GROUP BY n +``` + +### CREATE VIEW with invalid query + +The new analyzer always performs type-checking. +Previously, it was possible to create a `VIEW` with an invalid `SELECT` query. It would then fail during the first `SELECT` or `INSERT` (in the case of `MATERIALIZED VIEW`). + +Now, it's not possible to create such `VIEW`s anymore. + +**Example:** + +```sql +CREATE TABLE source (data String) ENGINE=MergeTree ORDER BY tuple(); + +CREATE VIEW some_view +AS SELECT JSONExtract(data, 'test', 'DateTime64(3)') +FROM source; +``` + +### Known incompatibilities of the `JOIN` clause + +#### Join using column from projection + +Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. + +A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the `SELECT` query, rather than using the columns from left table directly. + +**Example:** + +```sql +SELECT a + 1 AS b, t2.s +FROM Values('a UInt64, b UInt64', (1, 1)) AS t1 +JOIN Values('b UInt64, s String', (1, 'one'), (2, 'two')) t2 +USING (b); +``` + +With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'`. +When the setting is `false`, the join condition defaults to `t1.b = t2.b`, and the query will return `2, 'one'`. +If `b` is not present in `t1`, the query will fail with an error. + +#### Changes in behavior with `JOIN USING` and `ALIAS`/`MATERIALIZED` columns + +In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include those columns in the result set by default. + +**Example:** + +```sql +CREATE TABLE t1 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; +INSERT INTO t1 VALUES (1), (2); + +CREATE TABLE t2 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; +INSERT INTO t2 VALUES (2), (3); + +SELECT * FROM t1 +FULL JOIN t2 USING (payload); +``` + +In the new analyzer, the result of this query will include the `payload` column along with `id` from both tables. In contrast, the previous analyzer would only include these `ALIAS` columns if specific settings (`asterisk_include_alias_columns` or `asterisk_include_materialized_columns`) were enabled, and the columns might appear in a different order. + +To ensure consistent and expected results, especially when migrating old queries to the new analyzer, it is advisable to specify columns explicitly in the `SELECT` clause rather than using `*`. + +#### Handling of Type Modifiers for columns in `USING` Clause + +In the new version of the analyzer, the rules for determining the common supertype for columns specified in the `USING` clause have been standardized to produce more predictable outcomes, especially when dealing with type modifiers like `LowCardinality` and `Nullable`. + +- `LowCardinality(T)` and `T`: When a column of type `LowCardinality(T)` is joined with a column of type `T`, the resulting common supertype will be `T`, effectively discarding the `LowCardinality` modifier. + +- `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved. + +**Example:** + +```sql +SELECT id, toTypeName(id) FROM Values('id LowCardinality(String)', ('a')) AS t1 +FULL OUTER JOIN Values('id String', ('b')) AS t2 +USING (id); +``` + +In this query, the common supertype for `id` is determined as `String`, discarding the `LowCardinality` modifier from `t1`. + +### Projection column names changes + +During projection names computation, aliases are not substituted. + +```sql +SELECT + 1 + 1 AS x, + x + 1 +SETTINGS allow_experimental_analyzer = 0 +FORMAT PrettyCompact + + ┌─x─┬─plus(plus(1, 1), 1)─┐ +1. │ 2 │ 3 │ + └───┴─────────────────────┘ + +SELECT + 1 + 1 AS x, + x + 1 +SETTINGS allow_experimental_analyzer = 1 +FORMAT PrettyCompact + + ┌─x─┬─plus(x, 1)─┐ +1. │ 2 │ 3 │ + └───┴────────────┘ +``` + +### Incompatible function arguments types + +In the new analyzer, type inference happens during initial query analysis. +This change means that type checks are done before short-circuit evaluation; thus, `if` function arguments must always have a common supertype. + +**Example:** + +The following query fails with `There is no supertype for types Array(UInt8), String because some of them are Array and some of them are not`: + +```sql +SELECT toTypeName(if(0, [2, 3, 4], 'String')) +``` + +### Heterogeneous clusters + +The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values. + +### Mutations are interpreted by previous analyzer + +Mutations are still using the old analyzer. +This means some new ClickHouse SQL features can't be used in mutations. For example, the `QUALIFY` clause. +Status can be checked [here](https://github.com/ClickHouse/ClickHouse/issues/61563). + +### Unsupported features + +The list of features new analyzer currently doesn't support: + +- Annoy index. +- Hypothesis index. Work in progress [here](https://github.com/ClickHouse/ClickHouse/pull/48381). +- Window view is not supported. There are no plans to support it in the future. diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index c9d94dd95ee..91438cfb675 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -67,6 +67,23 @@ To manage named collections with DDL a user must have the `named_control_collect In the above example the `password_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plain text and sha256 hex passwords set for a user. ::: +### Storage for named collections + +Named collections can either be stored on local disk or in zookeeper/keeper. By default local storage is used. + +To configure named collections storage in keeper and a `type` (equal to either `keeper` or `zookeeper`) and `path` (path in keeper, where named collections will be stored) to `named_collections_storage` section in configuration file: +``` + + + zookeeper + /named_collections_path/ + 1000 + + +``` + +An optional configuration parameter `update_timeout_ms` by default is equal to `5000`. + ## Storing named collections in configuration files ### XML example @@ -443,3 +460,59 @@ SELECT dictGet('dict', 'b', 1); │ a │ └─────────────────────────┘ ``` + +## Named collections for accessing Kafka + +The description of parameters see [Kafka](../engines/table-engines/integrations/kafka.md). + +### DDL example + +```sql +CREATE NAMED COLLECTION my_kafka_cluster AS +kafka_broker_list = 'localhost:9092', +kafka_topic_list = 'kafka_topic', +kafka_group_name = 'consumer_group', +kafka_format = 'JSONEachRow', +kafka_max_block_size = '1048576'; + +``` +### XML example + +```xml + + + + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 + + + +``` + +### Example of using named collections with a Kafka table + +Both of the following examples use the same named collection `my_kafka_cluster`: + + +```sql +CREATE TABLE queue +( + timestamp UInt64, + level String, + message String +) +ENGINE = Kafka(my_kafka_cluster) + +CREATE TABLE queue +( + timestamp UInt64, + level String, + message String +) +ENGINE = Kafka(my_kafka_cluster) +SETTINGS kafka_num_consumers = 4, + kafka_thread_per_consumer = 1; +``` diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a5fe74fd0c6..0c007f1ac79 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -591,6 +591,22 @@ Default value: 100000 400 ``` +## max\_table\_num\_to\_throw {#max-table-num-to-throw} +If number of tables is greater than this value, server will throw an exception. 0 means no limitation. View, remote tables, dictionary, system tables are not counted. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.Default value: 0 + +**Example** +```xml +400 +``` + +## max\_database\_num\_to\_throw {#max-table-num-to-throw} +If number of _database is greater than this value, server will throw an exception. 0 means no limitation. +Default value: 0 + +**Example** +```xml +400 +``` ## max_temporary_data_on_disk_size @@ -1206,6 +1222,16 @@ Expired time for HSTS in seconds. The default value is 0 means clickhouse disabl 600000 ``` +## mlock_executable {#mlock_executable} + +Perform mlockall after startup to lower first queries latency and to prevent clickhouse executable from being paged out under high IO load. Enabling this option is recommended but will lead to increased startup time for up to a few seconds. +Keep in mind that this parameter would not work without "CAP_IPC_LOCK" capability. +**Example** + +``` xml +false +``` + ## include_from {#include_from} The path to the file with substitutions. Both XML and YAML formats are supported. @@ -1353,6 +1379,26 @@ Examples: 127.0.0.1 ``` +## listen_try {#listen_try} + +The server will not exit if IPv6 or IPv4 networks are unavailable while trying to listen. + +Examples: + +``` xml +0 +``` + +## listen_reuse_port {#listen_reuse_port} + +Allow multiple servers to listen on the same address:port. Requests will be routed to a random server by the operating system. Enabling this setting is not recommended. + +Examples: + +``` xml +0 +``` + ## listen_backlog {#listen_backlog} Backlog (queue size of pending connections) of the listen socket. @@ -2894,6 +2940,8 @@ Define proxy servers for HTTP and HTTPS requests, currently supported by S3 stor There are three ways to define proxy servers: environment variables, proxy lists, and remote proxy resolvers. +Bypassing proxy servers for specific hosts is also supported with the use of `no_proxy`. + ### Environment variables The `http_proxy` and `https_proxy` environment variables allow you to specify a @@ -3003,6 +3051,29 @@ This also allows a mix of resolver types can be used. By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests over `HTTP` proxy. This setting can be used to disable it. +### no_proxy +By default, all requests will go through the proxy. In order to disable it for specific hosts, the `no_proxy` variable must be set. +It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. +It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass. Leading dots are stripped just like curl does. + +Example: + +The below configuration bypasses proxy requests to `clickhouse.cloud` and all of its subdomains (e.g, `auth.clickhouse.cloud`). +The same applies to GitLab, even though it has a leading dot. Both `gitlab.com` and `about.gitlab.com` would bypass the proxy. + +``` xml + + clickhouse.cloud,.gitlab.com + + http://proxy1 + http://proxy2:3128 + + + http://proxy1:3128 + + +``` + ## max_materialized_views_count_for_table {#max_materialized_views_count_for_table} A limit on the number of materialized views attached to a table. @@ -3029,3 +3100,21 @@ This setting is only necessary for the migration period and will become obsolete Type: Bool Default: 1 + +## merge_workload {#merge_workload} + +Used to regulate how resources are utilized and shared between merges and other workloads. Specified value is used as `workload` setting value for all background merges. Can be overridden by a merge tree setting. + +Default value: "default" + +**See Also** +- [Workload Scheduling](/docs/en/operations/workload-scheduling.md) + +## mutation_workload {#mutation_workload} + +Used to regulate how resources are utilized and shared between mutations and other workloads. Specified value is used as `workload` setting value for all background mutations. Can be overridden by a merge tree setting. + +Default value: "default" + +**See Also** +- [Workload Scheduling](/docs/en/operations/workload-scheduling.md) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 76250b80476..9879ee35612 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -3,9 +3,126 @@ slug: /en/operations/settings/merge-tree-settings title: "MergeTree tables settings" --- -The values of `merge_tree` settings (for all MergeTree tables) can be viewed in the table `system.merge_tree_settings`, they can be overridden in `config.xml` in the `merge_tree` section, or set in the `SETTINGS` section of each table. +System table `system.merge_tree_settings` shows the globally set MergeTree settings. -These are example overrides for `max_suspicious_broken_parts`: +MergeTree settings can be set in the `merge_tree` section of the server config file, or specified for each `MergeTree` table individually in +the `SETTINGS` clause of the `CREATE TABLE` statement. + +Example for customizing setting `max_suspicious_broken_parts`: + +Configure the default for all `MergeTree` tables in the server configuration file: + +``` text + + 5 + +``` + +Set for a particular table: + +``` sql +CREATE TABLE tab +( + `A` Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS max_suspicious_broken_parts = 500; +``` + +Change the settings for a particular table using `ALTER TABLE ... MODIFY SETTING`: + +```sql +ALTER TABLE tab MODIFY SETTING max_suspicious_broken_parts = 100; + +-- reset to global default (value from system.merge_tree_settings) +ALTER TABLE tab RESET SETTING max_suspicious_broken_parts; +``` + +## index_granularity + +Maximum number of data rows between the marks of an index. + +Default value: 8192. + +## index_granularity_bytes + +Maximum size of data granules in bytes. + +Default value: 10Mb. + +To restrict the granule size only by number of rows, set to 0 (not recommended). + +## min_index_granularity_bytes + +Min allowed size of data granules in bytes. + +Default value: 1024b. + +To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. + +## enable_mixed_granularity_parts + +Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. + +## use_minimalistic_part_header_in_zookeeper + +Storage method of the data parts headers in ZooKeeper. If enabled, ZooKeeper stores less data. For details, see [here](../server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper). + +## min_merge_bytes_to_use_direct_io + +The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. +When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. +If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). +If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. + +Default value: `10 * 1024 * 1024 * 1024` bytes. + +## merge_with_ttl_timeout + +Minimum delay in seconds before repeating a merge with delete TTL. + +Default value: `14400` seconds (4 hours). + +## merge_with_recompression_ttl_timeout + +Minimum delay in seconds before repeating a merge with recompression TTL. + +Default value: `14400` seconds (4 hours). + +## write_final_mark + +Enables or disables writing the final index mark at the end of data part (after the last byte). + +Default value: 1. + +Don’t change or bad things will happen. + +## storage_policy + +Storage policy. + +## min_bytes_for_wide_part + +Minimum number of bytes/rows in a data part that can be stored in `Wide` format. +You can set one, both or none of these settings. + +## max_compress_block_size + +Maximum size of blocks of uncompressed data before compressing for writing to a table. +You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## min_compress_block_size + +Minimum size of blocks of uncompressed data required for compression when writing the next mark. +You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## max_partitions_to_read + +Limits the maximum number of partitions that can be accessed in one query. +You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. ## max_suspicious_broken_parts @@ -17,37 +134,6 @@ Possible values: Default value: 100. -Override example in `config.xml`: - -``` text - - 5 - -``` - -An example to set in `SETTINGS` for a particular table: - -``` sql -CREATE TABLE foo -( - `A` Int64 -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS max_suspicious_broken_parts = 500; -``` - -An example of changing the settings for a specific table with the `ALTER TABLE ... MODIFY SETTING` command: - -``` sql -ALTER TABLE foo - MODIFY SETTING max_suspicious_broken_parts = 100; - --- reset to default (use value from system.merge_tree_settings) -ALTER TABLE foo - RESET SETTING max_suspicious_broken_parts; -``` - ## parts_to_throw_insert {#parts-to-throw-insert} If the number of active parts in a single partition exceeds the `parts_to_throw_insert` value, `INSERT` is interrupted with the `Too many parts (N). Merges are processing significantly slower than inserts` exception. @@ -301,6 +387,8 @@ Default value: 10800 ## try_fetch_recompressed_part_timeout +Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. + Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression. Possible values: @@ -885,3 +973,67 @@ Default value: false **See Also** - [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting + +## merge_workload + +Used to regulate how resources are utilized and shared between merges and other workloads. Specified value is used as `workload` setting value for background merges of this table. If not specified (empty string), then server setting `merge_workload` is used instead. + +Default value: an empty string + +**See Also** +- [Workload Scheduling](/docs/en/operations/workload-scheduling.md) + +## mutation_workload + +Used to regulate how resources are utilized and shared between mutations and other workloads. Specified value is used as `workload` setting value for background mutations of this table. If not specified (empty string), then server setting `mutation_workload` is used instead. + +Default value: an empty string + +**See Also** +- [Workload Scheduling](/docs/en/operations/workload-scheduling.md) + +### optimize_row_order + +Controls if the row order should be optimized during inserts to improve the compressability of the newly inserted table part. + +Only has an effect for ordinary MergeTree-engine tables. Does nothing for specialized MergeTree engine tables (e.g. CollapsingMergeTree). + +MergeTree tables are (optionally) compressed using [compression codecs](../../sql-reference/statements/create/table.md#column_compression_codec). +Generic compression codecs such as LZ4 and ZSTD achieve maximum compression rates if the data exposes patterns. +Long runs of the same value typically compress very well. + +If this setting is enabled, ClickHouse attempts to store the data in newly inserted parts in a row order that minimizes the number of equal-value runs across the columns of the new table part. +In other words, a small number of equal-value runs mean that individual runs are long and compress well. + +Finding the optimal row order is computationally infeasible (NP hard). +Therefore, ClickHouse uses a heuristics to quickly find a row order which still improves compression rates over the original row order. + +
+ +Heuristics for finding a row order + +It is generally possible to shuffle the rows of a table (or table part) freely as SQL considers the same table (table part) in different row order equivalent. + +This freedom of shuffling rows is restricted when a primary key is defined for the table. +In ClickHouse, a primary key `C1, C2, ..., CN` enforces that the table rows are sorted by columns `C1`, `C2`, ... `Cn` ([clustered index](https://en.wikipedia.org/wiki/Database_index#Clustered)). +As a result, rows can only be shuffled within "equivalence classes" of row, i.e. rows which have the same values in their primary key columns. +The intuition is that primary keys with high-cardinality, e.g. primary keys involving a `DateTime64` timestamp column, lead to many small equivalence classes. +Likewise, tables with a low-cardinality primary key, create few and large equivalence classes. +A table with no primary key represents the extreme case of a single equivalence class which spans all rows. + +The fewer and the larger the equivalence classes are, the higher the degree of freedom when re-shuffling rows. + +The heuristics applied to find the best row order within each equivalence class is suggested by D. Lemir, O. Kaser in [Reordering columns for smaller indexes](https://doi.org/10.1016/j.ins.2011.02.002) and based on sorting the rows within each equivalence class by ascending cardinality of the non-primary key columns. +It performs three steps: +1. Find all equivalence classes based on the row values in primary key columns. +2. For each equivalence class, calculate (usually estimate) the cardinalities of the non-primary-key columns. +3. For each equivalence class, sort the rows in order of ascending non-primary-key column cardinality. + +
+ +If enabled, insert operations incur additional CPU costs to analyze and optimize the row order of the new data. +INSERTs are expected to take 30-50% longer depending on the data characteristics. +Compression rates of LZ4 or ZSTD improve on average by 20-40%. + +This setting works best for tables with no primary key or a low-cardinality primary key, i.e. a table with only few distinct primary key values. +High-cardinality primary keys, e.g. involving timestamp columns of type `DateTime64`, are not expected to benefit from this setting. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1a27b350652..670c9c6cbf1 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1417,6 +1417,17 @@ Compression method used in output Parquet format. Supported codecs: `snappy`, `l Default value: `lz4`. +### input_format_parquet_max_block_size {#input_format_parquet_max_block_size} +Max block row size for parquet reader. By controlling the number of rows in each block, you can control the memory usage, +and in some operators that cache blocks, you can improve the accuracy of the operator's memory control。 + +Default value: `65409`. + +### input_format_parquet_prefer_block_bytes {#input_format_parquet_prefer_block_bytes} +Average block bytes output by parquet reader. Lowering the configuration in the case of reading some high compression parquet relieves the memory pressure. + +Default value: `65409 * 256 = 16744704` + ## Hive format settings {#hive-format-settings} ### input_format_hive_text_fields_delimiter {#input_format_hive_text_fields_delimiter} @@ -1695,6 +1706,43 @@ Result: └────────────┘ ``` +## output_format_pretty_display_footer_column_names + +Display column names in the footer if there are many table rows. + +Possible values: + +- 0 — No column names are displayed in the footer. +- 1 — Column names are displayed in the footer if row count is greater than or equal to the threshold value set by [output_format_pretty_display_footer_column_names_min_rows](#output_format_pretty_display_footer_column_names_min_rows) (50 by default). + +Default value: `1`. + +**Example** + +Query: + +```sql +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 1000); +``` + +Result: + +```response + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + ... + 999. │ 998 │ UInt64 │ +1000. │ 999 │ UInt64 │ + └─number─┴─toTypeName(number)─┘ +``` +## output_format_pretty_display_footer_column_names_min_rows + +Sets the minimum number of rows for which a footer with column names will be displayed if setting [output_format_pretty_display_footer_column_names](#output_format_pretty_display_footer_column_names) is enabled. + +Default value: `50`. + ## Template format settings {#template-format-settings} ### format_template_resultset {#format_template_resultset} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0b905df21d4..59dd92f0fcd 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1590,6 +1590,22 @@ Possible values: Default value: `default`. +## parallel_replicas_custom_key_range_lower {#parallel_replicas_custom_key_range_lower} + +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. + +When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. + +## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper} + +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. + +When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. + ## allow_experimental_parallel_reading_from_replicas Enables or disables sending SELECT queries to all replicas of a table (up to `max_parallel_replicas`). Reading is parallelized and coordinated dynamically. It will work for any kind of MergeTree table. @@ -3170,6 +3186,18 @@ Possible values: Default value: `0`. +## lightweight_deletes_sync {#lightweight_deletes_sync} + +The same as 'mutation_sync', but controls only execution of lightweight deletes. + +Possible values: + +- 0 - Mutations execute asynchronously. +- 1 - The query waits for the lightweight deletes to complete on the current server. +- 2 - The query waits for the lightweight deletes to complete on all replicas (if they exist). + +Default value: `2`. + **See Also** - [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) @@ -3850,6 +3878,10 @@ Possible values: Default value: 30. +:::note +It's applicable only to the default profile. A server reboot is required for the changes to take effect. +::: + ## http_receive_timeout {#http_receive_timeout} HTTP receive timeout (in seconds). @@ -5108,7 +5140,7 @@ a Tuple( ) ``` -## allow_experimental_statistic {#allow_experimental_statistic} +## allow_experimental_statistics {#allow_experimental_statistics} Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). @@ -5386,6 +5418,12 @@ When set to `false` than all attempts are made with identical timeouts. Default value: `true`. +## uniform_snowflake_conversion_functions {#uniform_snowflake_conversion_functions} + +If set to `true`, then functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` are enabled, and functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` are disabled (and vice versa if set to `false`). + +Default value: `true` + ## allow_experimental_variant_type {#allow_experimental_variant_type} Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index 81725b97e41..762d187917c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -639,6 +639,10 @@ An internal metric of the low-level memory allocator (jemalloc). See https://jem An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +### jemalloc.prof.active + +An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html + **See Also** - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 75b855966a3..47094eec3f0 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -113,6 +113,8 @@ Columns: - `used_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `functions`, which were used during query execution. - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. +- `used_privileges` ([Array(String)](../../sql-reference/data-types/array.md)) - Privileges which were successfully checked during query execution. +- `missing_privileges` ([Array(String)](../../sql-reference/data-types/array.md)) - Privileges that are missing during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - `'Unknown'` = Status unknown. - `'None'` = The query result was neither written into nor read from the query cache. @@ -194,6 +196,8 @@ used_formats: [] used_functions: [] used_storages: [] used_table_functions: [] +used_privileges: [] +missing_privileges: [] query_cache_usage: None ``` diff --git a/docs/en/operations/utilities/odbc-bridge.md b/docs/en/operations/utilities/odbc-bridge.md index abb8860880e..eb849c6b6ae 100644 --- a/docs/en/operations/utilities/odbc-bridge.md +++ b/docs/en/operations/utilities/odbc-bridge.md @@ -18,7 +18,7 @@ This tool works via HTTP, not via pipes, shared memory, or TCP because: However it can be used as standalone tool from command line with the following parameters in POST-request URL: - `connection_string` -- ODBC connection string. -- `columns` -- columns in ClickHouse NamesAndTypesList format, name in backticks, +- `sample_block` -- columns description in ClickHouse NamesAndTypesList format, name in backticks, type as string. Name and type are space separated, rows separated with newline. - `max_block_size` -- optional parameter, sets maximum size of single block. diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 24149099892..08629492ec6 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -47,6 +47,8 @@ Example: Queries can be marked with setting `workload` to distinguish different workloads. If `workload` is not set, than value "default" is used. Note that you are able to specify the other value using settings profiles. Setting constraints can be used to make `workload` constant if you want all queries from the user to be marked with fixed value of `workload` setting. +It is possible to assign a `workload` setting for background activities. Merges and mutations are using `merge_workload` and `mutation_workload` server settings correspondingly. These values can also be overridden for specific tables using `merge_workload` and `mutation_workload` merge tree settings + Let's consider an example of a system with two different workloads: "production" and "development". ```sql @@ -151,6 +153,9 @@ Example: ``` - ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) + - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting + - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting + - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting + - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 8ccc5e292b5..e30aa66b3b3 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -106,8 +106,8 @@ To work with these states, use: - [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engine. - [finalizeAggregation](../../sql-reference/functions/other-functions.md#function-finalizeaggregation) function. - [runningAccumulate](../../sql-reference/functions/other-functions.md#runningaccumulate) function. -- [-Merge](#aggregate_functions_combinators-merge) combinator. -- [-MergeState](#aggregate_functions_combinators-mergestate) combinator. +- [-Merge](#-merge) combinator. +- [-MergeState](#-mergestate) combinator. ## -Merge diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 1dc89b8dcf9..093d88f939f 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -82,10 +82,12 @@ FROM In this case, you should remember that you do not know the histogram bin borders. -## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) +## sequenceMatch Checks whether the sequence contains an event chain that matches the pattern. +**Syntax** + ``` sql sequenceMatch(pattern)(timestamp, cond1, cond2, ...) ``` @@ -102,7 +104,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). **Returned values** @@ -170,9 +172,9 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM **See Also** -- [sequenceCount](#function-sequencecount) +- [sequenceCount](#sequencecount) -## sequenceCount(pattern)(time, cond1, cond2, ...) +## sequenceCount Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. @@ -180,6 +182,8 @@ Counts the number of event chains that matched the pattern. The function searche Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: +**Syntax** + ``` sql sequenceCount(pattern)(timestamp, cond1, cond2, ...) ``` @@ -192,7 +196,7 @@ sequenceCount(pattern)(timestamp, cond1, cond2, ...) **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). **Returned values** @@ -229,7 +233,7 @@ SELECT sequenceCount('(?1).*(?2)')(time, number = 1, number = 2) FROM t **See Also** -- [sequenceMatch](#function-sequencematch) +- [sequenceMatch](#sequencematch) ## windowFunnel diff --git a/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md new file mode 100644 index 00000000000..ae17153085c --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/flame_graph.md @@ -0,0 +1,95 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/flamegraph +sidebar_position: 110 +--- + +# flameGraph + +Aggregate function which builds a [flamegraph](https://www.brendangregg.com/flamegraphs.html) using the list of stacktraces. Outputs an array of strings which can be used by [flamegraph.pl utility](https://github.com/brendangregg/FlameGraph) to render an SVG of the flamegraph. + +## Syntax + +```sql +flameGraph(traces, [size], [ptr]) +``` + +## Parameters + +- `traces` — a stacktrace. [Array](../../data-types/array.md)([UInt64](../../data-types/int-uint.md)). +- `size` — an allocation size for memory profiling. (optional - default `1`). [UInt64](../../data-types/int-uint.md). +- `ptr` — an allocation address. (optional - default `0`). [UInt64](../../data-types/int-uint.md). + +:::note +In the case where `ptr != 0`, a flameGraph will map allocations (size > 0) and deallocations (size < 0) with the same size and ptr. +Only allocations which were not freed are shown. Non mapped deallocations are ignored. +::: + +## Returned value + +- An array of strings for use with [flamegraph.pl utility](https://github.com/brendangregg/FlameGraph). [Array](../../data-types/array.md)([String](../../data-types/string.md)). + +## Examples + +### Building a flamegraph based on a CPU query profiler + +```sql +SET query_profiler_cpu_time_period_ns=10000000; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +```text +clickhouse client --allow_introspection_functions=1 -q "select arrayJoin(flameGraph(arrayReverse(trace))) from system.trace_log where trace_type = 'CPU' and query_id = 'xxx'" | ~/dev/FlameGraph/flamegraph.pl > flame_cpu.svg +``` + +### Building a flamegraph based on a memory query profiler, showing all allocations + +```sql +SET memory_profiler_sample_probability=1, max_untracked_memory=1; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +```text +clickhouse client --allow_introspection_functions=1 -q "select arrayJoin(flameGraph(trace, size)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem.svg +``` + +### Building a flamegraph based on a memory query profiler, showing allocations which were not deallocated in query context + +```sql +SET memory_profiler_sample_probability=1, max_untracked_memory=1, use_uncompressed_cache=1, merge_tree_max_rows_to_use_cache=100000000000, merge_tree_max_bytes_to_use_cache=1000000000000; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +```text +clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, size, ptr)) FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx'" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_untracked.svg +``` + +### Build a flamegraph based on memory query profiler, showing active allocations at the fixed point of time + +```sql +SET memory_profiler_sample_probability=1, max_untracked_memory=1; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +``` + +- 1 - Memory usage per second + +```sql +SELECT event_time, m, formatReadableSize(max(s) as m) FROM (SELECT event_time, sum(size) OVER (ORDER BY event_time) AS s FROM system.trace_log WHERE query_id = 'xxx' AND trace_type = 'MemorySample') GROUP BY event_time ORDER BY event_time; +``` + +- 2 - Find a time point with maximal memory usage + +```sql +SELECT argMax(event_time, s), max(s) FROM (SELECT event_time, sum(size) OVER (ORDER BY event_time) AS s FROM system.trace_log WHERE query_id = 'xxx' AND trace_type = 'MemorySample'); +``` + +- 3 - Fix active allocations at fixed point of time + +```text +clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, size, ptr)) FROM (SELECT * FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx' AND event_time <= 'yyy' ORDER BY event_time)" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_pos.svg +``` + +- 4 - Find deallocations at fixed point of time + +```text +clickhouse client --allow_introspection_functions=1 -q "SELECT arrayJoin(flameGraph(trace, -size, ptr)) FROM (SELECT * FROM system.trace_log WHERE trace_type = 'MemorySample' AND query_id = 'xxx' AND event_time > 'yyy' ORDER BY event_time desc)" | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_neg.svg +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md new file mode 100644 index 00000000000..072252de8c9 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -0,0 +1,90 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/groupconcat +sidebar_position: 363 +sidebar_label: groupConcat +title: groupConcat +--- + +Calculates a concatenated string from a group of strings, optionally separated by a delimiter, and optionally limited by a maximum number of elements. + +**Syntax** + +``` sql +groupConcat(expression [, delimiter] [, limit]); +``` + +**Arguments** + +- `expression` — The expression or column name that outputs strings to be concatenated.. +- `delimiter` — A [string](../../../sql-reference/data-types/string.md) that will be used to separate concatenated values. This parameter is optional and defaults to an empty string if not specified. +- `limit` — A positive [integer](../../../sql-reference/data-types/int-uint.md) specifying the maximum number of elements to concatenate. If more elements are present, excess elements are ignored. This parameter is optional. + +:::note +If delimiter is specified without limit, it must be the first parameter following the expression. If both delimiter and limit are specified, delimiter must precede limit. +::: + +**Returned value** + +- Returns a [string](../../../sql-reference/data-types/string.md) consisting of the concatenated values of the column or expression. If the group has no elements or only null elements, and the function does not specify a handling for only null values, the result is a nullable string with a null value. + +**Examples** + +Input table: + +``` text +┌─id─┬─name─┐ +│ 1 │ John│ +│ 2 │ Jane│ +│ 3 │ Bob│ +└────┴──────┘ +``` + +1. Basic usage without a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name) FROM Employees; +``` + +Result: + +``` text +JohnJaneBob +``` + +This concatenates all names into one continuous string without any separator. + + +2. Using comma as a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane, Bob +``` + +This output shows the names separated by a comma followed by a space. + + +3. Limiting the number of concatenated elements + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane +``` + +This query limits the output to the first two names, even though there are more names in the table. diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index a56b1c97681..e3725b6a430 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -58,6 +58,7 @@ ClickHouse-specific aggregate functions: - [topKWeighted](../reference/topkweighted.md) - [deltaSum](../reference/deltasum.md) - [deltaSumTimestamp](../reference/deltasumtimestamp.md) +- [flameGraph](../reference/flame_graph.md) - [groupArray](../reference/grouparray.md) - [groupArrayLast](../reference/grouparraylast.md) - [groupUniqArray](../reference/groupuniqarray.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md b/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md index 06443994dd9..4f73aadb8da 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md +++ b/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md @@ -24,6 +24,8 @@ Alias: `lttb`. - `x` — x coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md). - `y` — y coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md). +NaNs are ignored in the provided series, meaning that any NaN values will be excluded from the analysis. This ensures that the function operates only on valid numerical data. + **Parameters** - `n` — number of points in the resulting series. [UInt64](../../../sql-reference/data-types/int-uint.md). @@ -61,7 +63,7 @@ Result: ``` text ┌────────largestTriangleThreeBuckets(4)(x, y)───────────┐ -│ [(1,10),(3,15),(5,40),(10,70)] │ +│ [(1,10),(3,15),(9,55),(10,70)] │ └───────────────────────────────────────────────────────┘ ``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index ddac82a0977..7ab9e1d3256 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -3,7 +3,7 @@ slug: /en/sql-reference/aggregate-functions/reference/stochasticlinearregression sidebar_position: 221 --- -# stochasticLinearRegression +# stochasticLinearRegression {#agg_functions_stochasticlinearregression_parameters} This function implements stochastic linear regression. It supports custom parameters for learning rate, L2 regularization coefficient, mini-batch size, and has a few methods for updating weights ([Adam](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Adam) (used by default), [simple SGD](https://en.wikipedia.org/wiki/Stochastic_gradient_descent), [Momentum](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Momentum), and [Nesterov](https://mipt.ru/upload/medialibrary/d7e/41-91.pdf)). @@ -72,5 +72,5 @@ The query will return a column of predicted values. Note that first argument of **See Also** -- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) +- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#stochasticlogisticregression) - [Difference between linear and logistic regressions](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md index 0a040689681..4bf5529ddcb 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md @@ -11,7 +11,7 @@ This function implements stochastic logistic regression. It can be used for bina Parameters are exactly the same as in stochasticLinearRegression: `learning rate`, `l2 regularization coefficient`, `mini-batch size`, `method for updating weights`. -For more information see [parameters](#agg_functions-stochasticlinearregression-parameters). +For more information see [parameters](../reference/stochasticlinearregression.md/#parameters). ``` text stochasticLogisticRegression(1.0, 1.0, 10, 'SGD') diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index fcabeb4c6a8..4e010248f6e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -27,7 +27,7 @@ Returns an integer of type `Float64`. **Implementation details** -This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable` function](#varPopStable). +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable`](#varpopstable) function. **Example** @@ -76,7 +76,7 @@ Returns an integer of type `Float64`. **Implementation details** -Unlike [`varPop()`](#varPop), this function uses a stable, numerically accurate algorithm to calculate the population variance to avoid issues like catastrophic cancellation or loss of precision. This function also handles `NaN` and `Inf` values correctly, excluding them from calculations. +Unlike [`varPop`](#varpop), this function uses a stable, numerically accurate algorithm to calculate the population variance to avoid issues like catastrophic cancellation or loss of precision. This function also handles `NaN` and `Inf` values correctly, excluding them from calculations. **Example** diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md index be669a16ae8..bd1cfa5742a 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md @@ -40,7 +40,7 @@ Where: The function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPop()` function](./varpop#varpop) instead. -This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable` function](#varSampStable). +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable`](#varsampstable) function. **Example** @@ -82,11 +82,11 @@ varSampStable(expr) **Returned value** -The `varSampStable()` function returns a Float64 value representing the sample variance of the input data set. +The `varSampStable` function returns a Float64 value representing the sample variance of the input data set. **Implementation details** -The `varSampStable()` function calculates the sample variance using the same formula as the [`varSamp()`](#varSamp function): +The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](#varsamp) function: ```plaintext ∑(x - mean(x))^2 / (n - 1) @@ -97,9 +97,9 @@ Where: - `mean(x)` is the arithmetic mean of the data set. - `n` is the number of data points in the data set. -The difference between `varSampStable()` and `varSamp()` is that `varSampStable()` is designed to provide a more deterministic and stable result when dealing with floating-point arithmetic. It uses an algorithm that minimizes the accumulation of rounding errors, which can be particularly important when dealing with large data sets or data with a wide range of values. +The difference between `varSampStable` and `varSamp` is that `varSampStable` is designed to provide a more deterministic and stable result when dealing with floating-point arithmetic. It uses an algorithm that minimizes the accumulation of rounding errors, which can be particularly important when dealing with large data sets or data with a wide range of values. -Like `varSamp()`, the `varSampStable()` function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPopStable()` function](./varpop#varpopstable) instead. +Like `varSamp`, the `varSampStable` function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPopStable`](./varpop#varpopstable) function instead. **Example** @@ -125,4 +125,4 @@ Response: 0.865 ``` -This query calculates the sample variance of the `value` column in the `example_table` using the `varSampStable()` function. The result shows that the sample variance of the values `[10.5, 12.3, 9.8, 11.2, 10.7]` is approximately 0.865, which may differ slightly from the result of `varSamp()` due to the more precise handling of floating-point arithmetic. +This query calculates the sample variance of the `value` column in the `example_table` using the `varSampStable()` function. The result shows that the sample variance of the values `[10.5, 12.3, 9.8, 11.2, 10.7]` is approximately 0.865, which may differ slightly from the result of `varSamp` due to the more precise handling of floating-point arithmetic. diff --git a/docs/en/sql-reference/data-types/boolean.md b/docs/en/sql-reference/data-types/boolean.md index 4c59bd947de..6fcbc218c5d 100644 --- a/docs/en/sql-reference/data-types/boolean.md +++ b/docs/en/sql-reference/data-types/boolean.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/data-types/boolean sidebar_position: 22 -sidebar_label: Boolean +sidebar_label: Bool --- # Bool diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index ac9a72c2641..250e766f2b7 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -137,7 +137,7 @@ If the time transition (due to daylight saving time or for other reasons) was pe Non-monotonic calendar dates. For example, in Happy Valley - Goose Bay, the time was transitioned one hour backwards at 00:01:00 7 Nov 2010 (one minute after midnight). So after 6th Nov has ended, people observed a whole one minute of 7th Nov, then time was changed back to 23:01 6th Nov and after another 59 minutes the 7th Nov started again. ClickHouse does not (yet) support this kind of fun. During these days the results of time processing functions may be slightly incorrect. -Similar issue exists for Casey Antarctic station in year 2010. They changed time three hours back at 5 Mar, 02:00. If you are working in antarctic station, please don't afraid to use ClickHouse. Just make sure you set timezone to UTC or be aware of inaccuracies. +Similar issue exists for Casey Antarctic station in year 2010. They changed time three hours back at 5 Mar, 02:00. If you are working in antarctic station, please don't be afraid to use ClickHouse. Just make sure you set timezone to UTC or be aware of inaccuracies. Time shifts for multiple days. Some pacific islands changed their timezone offset from UTC+14 to UTC-12. That's alright but some inaccuracies may present if you do calculations with their timezone for historical time points at the days of conversion. diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 23131d5b4fe..3c789076c1e 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -7,33 +7,43 @@ sidebar_label: Float32, Float64 # Float32, Float64 :::note -If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below: +If you need accurate calculations, in particular if you work with financial or business data requiring a high precision, you should consider using [Decimal](../data-types/decimal.md) instead. -``` +[Floating Point Numbers](https://en.wikipedia.org/wiki/IEEE_754) might lead to inaccurate results as illustrated below: + +```sql CREATE TABLE IF NOT EXISTS float_vs_decimal ( my_float Float64, my_decimal Decimal64(3) -)Engine=MergeTree ORDER BY tuple() - -INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +) +Engine=MergeTree +ORDER BY tuple(); +# Generate 1 000 000 random numbers with 2 decimal places and store them as a float and as a decimal +INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; +``` +``` SELECT sum(my_float), sum(my_decimal) FROM float_vs_decimal; -> 500279.56300000014 500279.563 + +┌──────sum(my_float)─┬─sum(my_decimal)─┐ +│ 499693.60500000004 │ 499693.605 │ +└────────────────────┴─────────────────┘ SELECT sumKahan(my_float), sumKahan(my_decimal) FROM float_vs_decimal; -> 500279.563 500279.563 + +┌─sumKahan(my_float)─┬─sumKahan(my_decimal)─┐ +│ 499693.605 │ 499693.605 │ +└────────────────────┴──────────────────────┘ ``` ::: -[Floating point numbers](https://en.wikipedia.org/wiki/IEEE_754). - -Types are equivalent to types of C: +The equivalent types in ClickHouse and in C are given below: - `Float32` — `float`. - `Float64` — `double`. -Aliases: +Float types in ClickHouse have the following aliases: - `Float32` — `FLOAT`, `REAL`, `SINGLE`. - `Float64` — `DOUBLE`, `DOUBLE PRECISION`. diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md index 7e3c32b3451..7ffc7447d96 100644 --- a/docs/en/sql-reference/data-types/geo.md +++ b/docs/en/sql-reference/data-types/geo.md @@ -33,7 +33,7 @@ Result: ## Ring -`Ring` is a simple polygon without holes stored as an array of points: [Array](array.md)([Point](#point-data-type)). +`Ring` is a simple polygon without holes stored as an array of points: [Array](array.md)([Point](#point)). **Example** @@ -54,7 +54,7 @@ Result: ## Polygon -`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring-data-type)). First element of outer array is the outer shape of polygon and all the following elements are holes. +`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring)). First element of outer array is the outer shape of polygon and all the following elements are holes. **Example** @@ -76,7 +76,7 @@ Result: ## MultiPolygon -`MultiPolygon` consists of multiple polygons and is stored as an array of polygons: [Array](array.md)([Polygon](#polygon-data-type)). +`MultiPolygon` consists of multiple polygons and is stored as an array of polygons: [Array](array.md)([Polygon](#polygon)). **Example** diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index 18c7816f811..9f82c2f093a 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -6,101 +6,106 @@ sidebar_label: Map(K, V) # Map(K, V) -`Map(K, V)` data type stores `key:value` pairs. -The Map datatype is implemented as `Array(Tuple(key T1, value T2))`, which means that the order of keys in each map does not change, i.e., this data type maintains insertion order. +Data type `Map(K, V)` stores key-value pairs. + +Unlike other databases, maps are not unique in ClickHouse, i.e. a map can contain two elements with the same key. +(The reason for that is that maps are internally implemented as `Array(Tuple(K, V))`.) + +You can use use syntax `m[k]` to obtain the value for key `k` in map `m`. +Also, `m[k]` scans the map, i.e. the runtime of the operation is linear in the size of the map. **Parameters** -- `key` — The key part of the pair. Arbitrary type, except [Nullable](../../sql-reference/data-types/nullable.md) and [LowCardinality](../../sql-reference/data-types/lowcardinality.md) nested with [Nullable](../../sql-reference/data-types/nullable.md) types. -- `value` — The value part of the pair. Arbitrary type, including [Map](../../sql-reference/data-types/map.md) and [Array](../../sql-reference/data-types/array.md). - -To get the value from an `a Map('key', 'value')` column, use `a['key']` syntax. This lookup works now with a linear complexity. +- `K` — The type of the Map keys. Arbitrary type except [Nullable](../../sql-reference/data-types/nullable.md) and [LowCardinality](../../sql-reference/data-types/lowcardinality.md) nested with [Nullable](../../sql-reference/data-types/nullable.md) types. +- `V` — The type of the Map values. Arbitrary type. **Examples** -Consider the table: +Create a table with a column of type map: ``` sql -CREATE TABLE table_map (a Map(String, UInt64)) ENGINE=Memory; -INSERT INTO table_map VALUES ({'key1':1, 'key2':10}), ({'key1':2,'key2':20}), ({'key1':3,'key2':30}); +CREATE TABLE tab (m Map(String, UInt64)) ENGINE=Memory; +INSERT INTO tab VALUES ({'key1':1, 'key2':10}), ({'key1':2,'key2':20}), ({'key1':3,'key2':30}); ``` -Select all `key2` values: +To select `key2` values: ```sql -SELECT a['key2'] FROM table_map; +SELECT m['key2'] FROM tab; ``` + Result: ```text -┌─arrayElement(a, 'key2')─┐ +┌─arrayElement(m, 'key2')─┐ │ 10 │ │ 20 │ │ 30 │ └─────────────────────────┘ ``` -If there's no such `key` in the `Map()` column, the query returns zeros for numerical values, empty strings or empty arrays. +If the requested key `k` is not contained in the map, `m[k]` returns the value type's default value, e.g. `0` for integer types and `''` for string types. +To check whether a key exists in a map, you can use function [mapContains](../../sql-reference/functions/tuple-map-functions#mapcontains). ```sql -INSERT INTO table_map VALUES ({'key3':100}), ({}); -SELECT a['key3'] FROM table_map; +CREATE TABLE tab (m Map(String, UInt64)) ENGINE=Memory; +INSERT INTO tab VALUES ({'key1':100}), ({}); +SELECT m['key1'] FROM tab; ``` Result: ```text -┌─arrayElement(a, 'key3')─┐ +┌─arrayElement(m, 'key1')─┐ │ 100 │ │ 0 │ └─────────────────────────┘ -┌─arrayElement(a, 'key3')─┐ -│ 0 │ -│ 0 │ -│ 0 │ -└─────────────────────────┘ ``` -## Convert Tuple to Map Type +## Converting Tuple to Map -You can cast `Tuple()` as `Map()` using [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function: +Values of type `Tuple()` can be casted to values of type `Map()` using function [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast): + +**Example** + +Query: ``` sql SELECT CAST(([1, 2, 3], ['Ready', 'Steady', 'Go']), 'Map(UInt8, String)') AS map; ``` +Result: + ``` text ┌─map───────────────────────────┐ │ {1:'Ready',2:'Steady',3:'Go'} │ └───────────────────────────────┘ ``` -## Map.keys and Map.values Subcolumns +## Reading subcolumns of Map -To optimize `Map` column processing, in some cases you can use the `keys` and `values` subcolumns instead of reading the whole column. +To avoid reading the entire map, you can use subcolumns `keys` and `values` in some cases. **Example** Query: ``` sql -CREATE TABLE t_map (`a` Map(String, UInt64)) ENGINE = Memory; +CREATE TABLE tab (m Map(String, UInt64)) ENGINE = Memory; +INSERT INTO tab VALUES (map('key1', 1, 'key2', 2, 'key3', 3)); -INSERT INTO t_map VALUES (map('key1', 1, 'key2', 2, 'key3', 3)); - -SELECT a.keys FROM t_map; - -SELECT a.values FROM t_map; +SELECT m.keys FROM tab; -- same as mapKeys(m) +SELECT m.values FROM tab; -- same as mapValues(m) ``` Result: ``` text -┌─a.keys─────────────────┐ +┌─m.keys─────────────────┐ │ ['key1','key2','key3'] │ └────────────────────────┘ -┌─a.values─┐ +┌─m.values─┐ │ [1,2,3] │ └──────────┘ ``` diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 080de94f8b7..4c7421d57c0 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -16,7 +16,7 @@ ClickHouse supports special functions for working with dictionaries that can be ClickHouse supports: - Dictionaries with a [set of functions](../../sql-reference/functions/ext-dict-functions.md). -- [Embedded dictionaries](#embedded_dictionaries) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). +- [Embedded dictionaries](#embedded-dictionaries) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). :::tip Tutorial @@ -82,7 +82,7 @@ You can [configure](#configuring-a-dictionary) any number of dictionaries in the You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../sql-reference/functions/other-functions.md) function). This functionality is not related to dictionaries. ::: -## Configuring a Dictionary {#configuring-a-dictionary} +## Configuring a Dictionary @@ -123,7 +123,7 @@ LAYOUT(...) -- Memory layout configuration LIFETIME(...) -- Lifetime of dictionary in memory ``` -## Storing Dictionaries in Memory {#storing-dictionaries-in-memory} +## Storing Dictionaries in Memory There are a variety of ways to store dictionaries in memory. @@ -415,7 +415,7 @@ or LAYOUT(COMPLEX_KEY_HASHED_ARRAY([SHARDS 1])) ``` -### range_hashed {#range_hashed} +### range_hashed The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. @@ -679,7 +679,7 @@ When searching for a dictionary, the cache is searched first. For each block of If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. -For cache dictionaries, the expiration [lifetime](#dictionary-updates) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. +For cache dictionaries, the expiration [lifetime](#refreshing-dictionary-data-using-lifetime) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../operations/system-tables/dictionaries.md) table. @@ -899,7 +899,7 @@ Other types are not supported yet. The function returns the attribute for the pr Data must completely fit into RAM. -## Refreshing dictionary data using LIFETIME {#lifetime} +## Refreshing dictionary data using LIFETIME ClickHouse periodically updates dictionaries based on the `LIFETIME` tag (defined in seconds). `LIFETIME` is the update interval for fully downloaded dictionaries and the invalidation interval for cached dictionaries. @@ -1031,7 +1031,7 @@ SOURCE(CLICKHOUSE(... update_field 'added_time' update_lag 15)) ... ``` -## Dictionary Sources {#dictionary-sources} +## Dictionary Sources @@ -1065,7 +1065,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration The source is configured in the `source` section. -For source types [Local file](#local_file), [Executable file](#executable), [HTTP(s)](#https), [ClickHouse](#clickhouse) +For source types [Local file](#local-file), [Executable file](#executable-file), [HTTP(s)](#https), [ClickHouse](#clickhouse) optional settings are available: ``` xml @@ -1089,10 +1089,10 @@ SETTINGS(format_csv_allow_single_quotes = 0) Types of sources (`source_type`): -- [Local file](#local_file) -- [Executable File](#executable) -- [Executable Pool](#executable_pool) -- [HTTP(S)](#http) +- [Local file](#local-file) +- [Executable File](#executable-file) +- [Executable Pool](#executable-pool) +- [HTTP(S)](#https) - DBMS - [ODBC](#odbc) - [MySQL](#mysql) @@ -1102,7 +1102,7 @@ Types of sources (`source_type`): - [Cassandra](#cassandra) - [PostgreSQL](#postgresql) -### Local File {#local_file} +### Local File Example of settings: @@ -1132,9 +1132,9 @@ When a dictionary with source `FILE` is created via DDL command (`CREATE DICTION - [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) -### Executable File {#executable} +### Executable File -Working with executable files depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. +Working with executable files depends on [how the dictionary is stored in memory](#storing-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. Example of settings: @@ -1161,7 +1161,7 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. -### Executable Pool {#executable_pool} +### Executable Pool Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. @@ -1196,9 +1196,9 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. -### HTTP(S) {#https} +### HTTP(S) -Working with an HTTP(S) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. +Working with an HTTP(S) server depends on [how the dictionary is stored in memory](#storing-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. Example of settings: @@ -1285,7 +1285,7 @@ Setting fields: - `db` – Name of the database. Omit it if the database name is set in the `` parameters. - `table` – Name of the table and schema if exists. - `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `query` – The custom query. Optional parameter. :::note @@ -1575,7 +1575,7 @@ Setting fields: - `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in MySQL, for example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `fail_on_connection_loss` – The configuration parameter that controls behavior of the server on connection loss. If `true`, an exception is thrown immediately if the connection between client and server was lost. If `false`, the ClickHouse server retries to execute the query three times before throwing an exception. Note that retrying leads to increased response times. Default value: `false`. @@ -1672,7 +1672,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `secure` - Use ssl for connection. - `query` – The custom query. Optional parameter. @@ -1849,7 +1849,7 @@ Setting fields: - `db` – Name of the database. - `table` – Name of the table. - `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause in PostgreSQL. For example, `id > 10 AND id < 20`. Optional parameter. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Refreshing dictionary data using LIFETIME](#refreshing-dictionary-data-using-lifetime). - `query` – The custom query. Optional parameter. :::note @@ -1873,7 +1873,7 @@ LAYOUT(FLAT()) LIFETIME(0); ``` -## Dictionary Key and Fields {#dictionary-key-and-fields} +## Dictionary Key and Fields @@ -1963,7 +1963,7 @@ PRIMARY KEY Id ### Composite Key -The key can be a `tuple` from any types of fields. The [layout](#storig-dictionaries-in-memory) in this case must be `complex_key_hashed` or `complex_key_cache`. +The key can be a `tuple` from any types of fields. The [layout](#storing-dictionaries-in-memory) in this case must be `complex_key_hashed` or `complex_key_cache`. :::tip A composite key can consist of a single element. This makes it possible to use a string as the key, for instance. @@ -2030,17 +2030,17 @@ CREATE DICTIONARY somename ( Configuration fields: -| Tag | Description | Required | -|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| -| `name` | Column name. | Yes | -| `type` | ClickHouse data type: [UInt8](../../sql-reference/data-types/int-uint.md), [UInt16](../../sql-reference/data-types/int-uint.md), [UInt32](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md), [Int8](../../sql-reference/data-types/int-uint.md), [Int16](../../sql-reference/data-types/int-uint.md), [Int32](../../sql-reference/data-types/int-uint.md), [Int64](../../sql-reference/data-types/int-uint.md), [Float32](../../sql-reference/data-types/float.md), [Float64](../../sql-reference/data-types/float.md), [UUID](../../sql-reference/data-types/uuid.md), [Decimal32](../../sql-reference/data-types/decimal.md), [Decimal64](../../sql-reference/data-types/decimal.md), [Decimal128](../../sql-reference/data-types/decimal.md), [Decimal256](../../sql-reference/data-types/decimal.md),[Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), [String](../../sql-reference/data-types/string.md), [Array](../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../sql-reference/data-types/nullable.md) is currently supported for [Flat](#flat), [Hashed](#hashed), [ComplexKeyHashed](#complex_key_hashed), [Direct](#direct), [ComplexKeyDirect](#complex_key_direct), [RangeHashed](#range_hashed), Polygon, [Cache](#cache), [ComplexKeyCache](#complex_key_cache), [SSDCache](#ssd_cache), [SSDComplexKeyCache](#complex_key_ssd_cache) dictionaries. In [IPTrie](#ip_trie) dictionaries `Nullable` types are not supported. | Yes | -| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../syntax.md#null) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | -| `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | -| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical-dictionaries).

Default value: `false`. | No | -| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | -| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. +| Tag | Description | Required | +|------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------| +| `name` | Column name. | Yes | +| `type` | ClickHouse data type: [UInt8](../../sql-reference/data-types/int-uint.md), [UInt16](../../sql-reference/data-types/int-uint.md), [UInt32](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md), [Int8](../../sql-reference/data-types/int-uint.md), [Int16](../../sql-reference/data-types/int-uint.md), [Int32](../../sql-reference/data-types/int-uint.md), [Int64](../../sql-reference/data-types/int-uint.md), [Float32](../../sql-reference/data-types/float.md), [Float64](../../sql-reference/data-types/float.md), [UUID](../../sql-reference/data-types/uuid.md), [Decimal32](../../sql-reference/data-types/decimal.md), [Decimal64](../../sql-reference/data-types/decimal.md), [Decimal128](../../sql-reference/data-types/decimal.md), [Decimal256](../../sql-reference/data-types/decimal.md),[Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), [String](../../sql-reference/data-types/string.md), [Array](../../sql-reference/data-types/array.md).
ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.
[Nullable](../../sql-reference/data-types/nullable.md) is currently supported for [Flat](#flat), [Hashed](#hashed), [ComplexKeyHashed](#complex_key_hashed), [Direct](#direct), [ComplexKeyDirect](#complex_key_direct), [RangeHashed](#range_hashed), Polygon, [Cache](#cache), [ComplexKeyCache](#complex_key_cache), [SSDCache](#ssd_cache), [SSDComplexKeyCache](#complex_key_ssd_cache) dictionaries. In [IPTrie](#ip_trie) dictionaries `Nullable` types are not supported. | Yes | +| `null_value` | Default value for a non-existing element.
In the example, it is an empty string. [NULL](../syntax.md#null) value can be used only for the `Nullable` types (see the previous line with types description). | Yes | +| `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.
The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.

Default value: no expression. | No | +| `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical-dictionaries).

Default value: `false`. | No | +| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.

Default value: `false`. | No | +| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. -## Hierarchical Dictionaries {#hierarchical-dictionaries} +## Hierarchical Dictionaries ClickHouse supports hierarchical dictionaries with a [numeric key](#numeric-key). @@ -2165,7 +2165,7 @@ Points can be specified as an array or a tuple of their coordinates. In the curr The user can upload their own data in all formats supported by ClickHouse. -There are 3 types of [in-memory storage](#storig-dictionaries-in-memory) available: +There are 3 types of [in-memory storage](#storing-dictionaries-in-memory) available: - `POLYGON_SIMPLE`. This is a naive implementation, where a linear pass through all polygons is made for each query, and membership is checked for each one without using additional indexes. @@ -2435,7 +2435,7 @@ LIFETIME(0) LAYOUT(regexp_tree); ``` -## Embedded Dictionaries {#embedded-dictionaries} +## Embedded Dictionaries diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 7b52fbff714..d87ca4a0fe7 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1261,7 +1261,7 @@ SELECT arraySort((x) -> -x, [1, 2, 3]) as res; └─────────┘ ``` -For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` function sorts the keys in ascending order, the result is \[3, 2, 1\]. Thus, the `(x) –> -x` lambda function sets the [descending order](#reverse-sort) in a sorting. +For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` function sorts the keys in ascending order, the result is \[3, 2, 1\]. Thus, the `(x) –> -x` lambda function sets the [descending order](#arrayreversesort) in a sorting. The lambda function can accept multiple arguments. In this case, you need to pass the `arraySort` function several arrays of identical length that the arguments of lambda function will correspond to. The resulting array will consist of elements from the first input array; elements from the next input array(s) specify the sorting keys. For example: @@ -1307,10 +1307,15 @@ To improve sorting efficiency, the [Schwartzian transform](https://en.wikipedia. Same as `arraySort` with additional `limit` argument allowing partial sorting. Returns an array of the same size as the original array where elements in range `[1..limit]` are sorted in ascending order. Remaining elements `(limit..N]` shall contain elements in unspecified order. -## arrayReverseSort(\[func,\] arr, ...) {#reverse-sort} +## arrayReverseSort Sorts the elements of the `arr` array in descending order. If the `func` function is specified, `arr` is sorted according to the result of the `func` function applied to the elements of the array, and then the sorted array is reversed. If `func` accepts multiple arguments, the `arrayReverseSort` function is passed several arrays that the arguments of `func` will correspond to. Detailed examples are shown at the end of `arrayReverseSort` description. +**Syntax** + +```sql +arrayReverseSort([func,] arr, ...) +``` Example of integer values sorting: ``` sql @@ -1907,10 +1912,16 @@ FROM numbers(1,10); - [arrayReduce](#arrayreduce) -## arrayReverse(arr) +## arrayReverse Returns an array of the same size as the original array containing the elements in reverse order. +**Syntax** + +```sql +arrayReverse(arr) +``` + Example: ``` sql diff --git a/docs/en/sql-reference/functions/bitmap-functions.md b/docs/en/sql-reference/functions/bitmap-functions.md index a5c8a663b71..d30c0f4dde4 100644 --- a/docs/en/sql-reference/functions/bitmap-functions.md +++ b/docs/en/sql-reference/functions/bitmap-functions.md @@ -74,7 +74,7 @@ bitmapSubsetInRange(bitmap, range_start, range_end) **Arguments** -- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – [Bitmap object](#bitmapbuild). - `range_start` – Start of the range (inclusive). [UInt32](../data-types/int-uint.md). - `range_end` – End of the range (exclusive). [UInt32](../data-types/int-uint.md). @@ -104,7 +104,7 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit) **Arguments** -- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – [Bitmap object](#bitmapbuild). - `range_start` – Start of the range (inclusive). [UInt32](../data-types/int-uint.md). - `cardinality_limit` – Maximum cardinality of the subset. [UInt32](../data-types/int-uint.md). @@ -134,7 +134,7 @@ subBitmap(bitmap, offset, cardinality_limit) **Arguments** -- `bitmap` – The bitmap. [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – The bitmap. [Bitmap object](#bitmapbuild). - `offset` – The position of the first element of the subset. [UInt32](../data-types/int-uint.md). - `cardinality_limit` – The maximum number of elements in the subset. [UInt32](../data-types/int-uint.md). @@ -162,7 +162,7 @@ bitmapContains(bitmap, needle) **Arguments** -- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `bitmap` – [Bitmap object](#bitmapbuild). - `needle` – Searched bit value. [UInt32](../data-types/int-uint.md). **Returned values** @@ -188,7 +188,7 @@ Result: Checks whether two bitmaps intersect. -If `bitmap2` contains exactly one element, consider using [bitmapContains](#bitmap_functions-bitmapcontains) instead as it works more efficiently. +If `bitmap2` contains exactly one element, consider using [bitmapContains](#bitmapcontains) instead as it works more efficiently. **Syntax** diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4092c83954a..b532e0de8f0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -83,7 +83,7 @@ Result: ``` ## makeDate32 -Like [makeDate](#makeDate) but produces a [Date32](../data-types/date32.md). +Like [makeDate](#makedate) but produces a [Date32](../data-types/date32.md). ## makeDateTime @@ -214,7 +214,7 @@ Result: **See also** -- [serverTimeZone](#serverTimeZone) +- [serverTimeZone](#servertimezone) ## serverTimeZone @@ -249,7 +249,7 @@ Result: **See also** -- [timeZone](#timeZone) +- [timeZone](#timezone) ## toTimeZone @@ -305,7 +305,7 @@ int32samoa: 1546300800 **See Also** -- [formatDateTime](#formatDateTime) - supports non-constant timezone. +- [formatDateTime](#formatdatetime) - supports non-constant timezone. - [toString](type-conversion-functions.md#tostring) - supports non-constant timezone. ## timeZoneOf @@ -1006,7 +1006,7 @@ toStartOfWeek(t[, mode[, timezone]]) **Arguments** - `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) -- `mode` - determines the first day of the week as described in the [toWeek()](date-time-functions#toweek) function +- `mode` - determines the first day of the week as described in the [toWeek()](#toweek) function - `timezone` - Optional parameter, it behaves like any other conversion function **Returned value** @@ -1049,7 +1049,7 @@ toLastDayOfWeek(t[, mode[, timezone]]) **Arguments** - `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) -- `mode` - determines the last day of the week as described in the [toWeek()](date-time-functions#toweek) function +- `mode` - determines the last day of the week as described in the [toWeek](#toweek) function - `timezone` - Optional parameter, it behaves like any other conversion function **Returned value** @@ -1719,7 +1719,7 @@ Result: **See Also** -- [fromDaysSinceYearZero](#fromDaysSinceYearZero) +- [fromDaysSinceYearZero](#fromdayssinceyearzero) ## fromDaysSinceYearZero @@ -1759,11 +1759,11 @@ Result: **See Also** -- [toDaysSinceYearZero](#toDaysSinceYearZero) +- [toDaysSinceYearZero](#todayssinceyearzero) ## fromDaysSinceYearZero32 -Like [fromDaysSinceYearZero](#fromDaysSinceYearZero) but returns a [Date32](../data-types/date32.md). +Like [fromDaysSinceYearZero](#fromdayssinceyearzero) but returns a [Date32](../data-types/date32.md). ## age @@ -1982,7 +1982,7 @@ Result: **See Also** -- [toStartOfInterval](#tostartofintervaldate_or_date_with_time-interval-x-unit--time_zone) +- [toStartOfInterval](#tostartofinterval) ## date\_add @@ -2055,7 +2055,7 @@ Result: **See Also** -- [addDate](#addDate) +- [addDate](#adddate) ## date\_sub @@ -2129,7 +2129,7 @@ Result: **See Also** -- [subDate](#subDate) +- [subDate](#subdate) ## timestamp\_add @@ -2310,7 +2310,7 @@ Alias: `SUBDATE` - [date_sub](#date_sub) -## now {#now} +## now Returns the current date and time at the moment of query analysis. The function is a constant expression. @@ -3609,7 +3609,7 @@ SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64 └───────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## formatDateTime {#formatDateTime} +## formatDateTime Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. @@ -3734,10 +3734,9 @@ LIMIT 10 **See Also** -- [formatDateTimeInJodaSyntax](##formatDateTimeInJodaSyntax) +- [formatDateTimeInJodaSyntax](#formatdatetimeinjodasyntax) - -## formatDateTimeInJodaSyntax {#formatDateTimeInJodaSyntax} +## formatDateTimeInJodaSyntax Similar to formatDateTime, except that it formats datetime in Joda style instead of MySQL style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. @@ -3902,11 +3901,11 @@ Result: **See Also** -- [fromUnixTimestampInJodaSyntax](##fromUnixTimestampInJodaSyntax) +- [fromUnixTimestampInJodaSyntax](#fromunixtimestampinjodasyntax) ## fromUnixTimestampInJodaSyntax -Same as [fromUnixTimestamp](#fromUnixTimestamp) but when called in the second way (two or three arguments), the formatting is performed using [Joda style](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL style. +Same as [fromUnixTimestamp](#fromunixtimestamp) but when called in the second way (two or three arguments), the formatting is performed using [Joda style](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL style. **Example:** @@ -4121,7 +4120,7 @@ Result: Returns the current date and time at the moment of query analysis. The function is a constant expression. :::note -This function gives the same result that `now('UTC')` would. It was added only for MySQL support and [`now`](#now-now) is the preferred usage. +This function gives the same result that `now('UTC')` would. It was added only for MySQL support and [`now`](#now) is the preferred usage. ::: **Syntax** diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 82c21ce40c8..093ee690d47 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -12,7 +12,7 @@ For dictionaries created with [DDL queries](../../sql-reference/statements/creat For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/index.md). -## dictGet, dictGetOrDefault, dictGetOrNull {#dictGet} +## dictGet, dictGetOrDefault, dictGetOrNull Retrieves values from a dictionary. diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index 8abc8006e5d..b6ac7a74092 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -4,6 +4,8 @@ sidebar_label: Geohash title: "Functions for Working with Geohash" --- +## Geohash + [Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer is the geohash string, the more precise is the geographic location. If you need to manually convert geographic coordinates to geohash strings, you can use [geohash.org](http://geohash.org/). diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index bcdd457964a..5fbc2adf2fa 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -4,6 +4,8 @@ sidebar_label: H3 Indexes title: "Functions for Working with H3 Indexes" --- +## H3 Index + [H3](https://eng.uber.com/h3/) is a geographical indexing system where Earth’s surface divided into a grid of even hexagonal cells. This system is hierarchical, i. e. each hexagon on the top level ("parent") can be split into seven even but smaller ones ("children"), and so on. The level of the hierarchy is called `resolution` and can receive a value from `0` till `15`, where `0` is the `base` level with the largest and coarsest cells. @@ -16,7 +18,7 @@ The full description of the H3 system is available at [the Uber Engineering site ## h3IsValid -Verifies whether the number is a valid [H3](#h3index) index. +Verifies whether the number is a valid [H3](#h3-index) index. **Syntax** @@ -51,7 +53,7 @@ Result: ## h3GetResolution -Defines the resolution of the given [H3](#h3index) index. +Defines the resolution of the given [H3](#h3-index) index. **Syntax** @@ -86,7 +88,7 @@ Result: ## h3EdgeAngle -Calculates the average length of the [H3](#h3index) hexagon edge in grades. +Calculates the average length of the [H3](#h3-index) hexagon edge in grades. **Syntax** @@ -100,7 +102,7 @@ h3EdgeAngle(resolution) **Returned values** -- The average length of the [H3](#h3index) hexagon edge in grades. [Float64](../../data-types/float.md). +- The average length of the [H3](#h3-index) hexagon edge in grades. [Float64](../../data-types/float.md). **Example** @@ -120,7 +122,7 @@ Result: ## h3EdgeLengthM -Calculates the average length of the [H3](#h3index) hexagon edge in meters. +Calculates the average length of the [H3](#h3-index) hexagon edge in meters. **Syntax** @@ -134,7 +136,7 @@ h3EdgeLengthM(resolution) **Returned values** -- The average length of the [H3](#h3index) hexagon edge in meters. [Float64](../../data-types/float.md). +- The average length of the [H3](#h3-index) hexagon edge in meters. [Float64](../../data-types/float.md). **Example** @@ -154,7 +156,7 @@ Result: ## h3EdgeLengthKm -Calculates the average length of the [H3](#h3index) hexagon edge in kilometers. +Calculates the average length of the [H3](#h3-index) hexagon edge in kilometers. **Syntax** @@ -168,7 +170,7 @@ h3EdgeLengthKm(resolution) **Returned values** -- The average length of the [H3](#h3index) hexagon edge in kilometers. [Float64](../../data-types/float.md). +- The average length of the [H3](#h3-index) hexagon edge in kilometers. [Float64](../../data-types/float.md). **Example** @@ -188,7 +190,7 @@ Result: ## geoToH3 -Returns [H3](#h3index) point index `(lon, lat)` with specified resolution. +Returns [H3](#h3-index) point index `(lon, lat)` with specified resolution. **Syntax** @@ -225,7 +227,7 @@ Result: ## h3ToGeo -Returns the centroid longitude and latitude corresponding to the provided [H3](#h3index) index. +Returns the centroid longitude and latitude corresponding to the provided [H3](#h3-index) index. **Syntax** @@ -294,7 +296,7 @@ Result: ## h3kRing - Lists all the [H3](#h3index) hexagons in the raduis of `k` from the given hexagon in random order. + Lists all the [H3](#h3-index) hexagons in the raduis of `k` from the given hexagon in random order. **Syntax** @@ -335,7 +337,7 @@ Result: ## h3GetBaseCell -Returns the base cell number of the [H3](#h3index) index. +Returns the base cell number of the [H3](#h3-index) index. **Syntax** @@ -437,7 +439,7 @@ Result: ## h3IndexesAreNeighbors -Returns whether or not the provided [H3](#h3index) indexes are neighbors. +Returns whether or not the provided [H3](#h3-index) indexes are neighbors. **Syntax** @@ -473,7 +475,7 @@ Result: ## h3ToChildren -Returns an array of child indexes for the given [H3](#h3index) index. +Returns an array of child indexes for the given [H3](#h3-index) index. **Syntax** @@ -508,7 +510,7 @@ Result: ## h3ToParent -Returns the parent (coarser) index containing the given [H3](#h3index) index. +Returns the parent (coarser) index containing the given [H3](#h3-index) index. **Syntax** @@ -609,7 +611,7 @@ Result: ## h3GetResolution -Returns the resolution of the [H3](#h3index) index. +Returns the resolution of the [H3](#h3-index) index. **Syntax** @@ -643,7 +645,7 @@ Result: ## h3IsResClassIII -Returns whether [H3](#h3index) index has a resolution with Class III orientation. +Returns whether [H3](#h3-index) index has a resolution with Class III orientation. **Syntax** @@ -678,7 +680,7 @@ Result: ## h3IsPentagon -Returns whether this [H3](#h3index) index represents a pentagonal cell. +Returns whether this [H3](#h3-index) index represents a pentagonal cell. **Syntax** @@ -713,7 +715,7 @@ Result: ## h3GetFaces -Returns icosahedron faces intersected by a given [H3](#h3index) index. +Returns icosahedron faces intersected by a given [H3](#h3-index) index. **Syntax** @@ -815,7 +817,7 @@ Result: ## h3ToCenterChild -Returns the center child (finer) [H3](#h3index) index contained by given [H3](#h3index) at the given resolution. +Returns the center child (finer) [H3](#h3-index) index contained by given [H3](#h3-index) at the given resolution. **Syntax** @@ -830,7 +832,7 @@ h3ToCenterChild(index, resolution) **Returned values** -- [H3](#h3index) index of the center child contained by given [H3](#h3index) at the given resolution. [UInt64](../../data-types/int-uint.md). +- [H3](#h3-index) index of the center child contained by given [H3](#h3-index) at the given resolution. [UInt64](../../data-types/int-uint.md). **Example** diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index c2572779ada..25a7a1fac8e 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -142,6 +142,34 @@ SELECT readWKTPoint('POINT (1.2 3.4)'); (1.2,3.4) ``` +## readWKTLineString + +Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format. + +### Syntax + +```sql +readWKTLineString(wkt_string) +``` + +### Arguments + +- `wkt_string`: The input WKT string representing a LineString geometry. + +### Returned value + +The function returns a ClickHouse internal representation of the linestring geometry. + +### Example + +```sql +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +``` + +```response +[(1,1),(2,2),(3,3),(1,1)] +``` + ## readWKTRing Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format. @@ -163,7 +191,7 @@ The function returns a ClickHouse internal representation of the ring (closed li ### Example ```sql -SELECT readWKTRing('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +SELECT readWKTRing('POLYGON ((1 1, 2 2, 3 3, 1 1))'); ``` ```response diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 3165b21318b..e022ce870b0 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -5,6 +5,8 @@ sidebar_label: S2 Geometry # Functions for Working with S2 Index +## S2Index + [S2](https://s2geometry.io/) is a geographical indexing system where all geographical data is represented on a three-dimensional sphere (similar to a globe). In the S2 library points are represented as the S2 Index - a specific number which encodes internally a point on the surface of a unit sphere, unlike traditional (latitude, longitude) pairs. To get the S2 point index for a given point specified in the format (latitude, longitude) use the [geoToS2](#geotos2) function. Also, you can use the [s2ToGeo](#s2togeo) function for getting geographical coordinates corresponding to the specified S2 point index. diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 506114038f7..e431ed75465 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -45,13 +45,13 @@ SELECT halfMD5(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00') Calculates the MD4 from a string and returns the resulting set of bytes as FixedString(16). -## MD5 {#md5} +## MD5 Calculates the MD5 from a string and returns the resulting set of bytes as FixedString(16). If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#siphash64} +## sipHash64 Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index 5b6a3aef2c8..11a7749b33d 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -295,7 +295,7 @@ Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns null ## toIPv6 Converts a string form of IPv6 address to [IPv6](../data-types/ipv6.md) type. If the IPv6 address has an invalid format, returns an empty value. -Similar to [IPv6StringToNum](#ipv6stringtonums) function, which converts IPv6 address to binary format. +Similar to [IPv6StringToNum](#ipv6stringtonum) function, which converts IPv6 address to binary format. If the input string contains a valid IPv4 address, then the IPv6 equivalent of the IPv4 address is returned. diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 5d73c9a83b3..7bff6a6cba5 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -5,10 +5,10 @@ sidebar_label: JSON --- There are two sets of functions to parse JSON: - - [`simpleJSON*` (`visitParam*`)](#simplejson--visitparam-functions) which is made for parsing a limited subset of JSON extremely fast. + - [`simpleJSON*` (`visitParam*`)](#simplejson-visitparam-functions) which is made for parsing a limited subset of JSON extremely fast. - [`JSONExtract*`](#jsonextract-functions) which is made for parsing ordinary JSON. -## simpleJSON / visitParam functions +## simpleJSON (visitParam) functions ClickHouse has special functions for working with simplified JSON. All these JSON functions are based on strong assumptions about what the JSON can be. They try to do as little as possible to get the job done as quickly as possible. diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 7f50fa933b6..12098efc635 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -415,8 +415,8 @@ Alias: `power(x, y)` **Arguments** -- `x` - [(U)Int8/16/32/64](../data-types/int-uint.md) or [Float*](../data-types/float.md) -- `y` - [(U)Int8/16/32/64](../data-types/int-uint.md) or [Float*](../data-types/float.md) +- `x` - [(U)Int8/16/32/64](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md) +- `y` - [(U)Int8/16/32/64](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md) **Returned value** @@ -635,8 +635,8 @@ atan2(y, x) **Arguments** -- `y` — y-coordinate of the point through which the ray passes. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). -- `x` — x-coordinate of the point through which the ray passes. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). +- `y` — y-coordinate of the point through which the ray passes. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md). +- `x` — x-coordinate of the point through which the ray passes. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md). **Returned value** @@ -670,8 +670,8 @@ hypot(x, y) **Arguments** -- `x` — The first cathetus of a right-angle triangle. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). -- `y` — The second cathetus of a right-angle triangle. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md). +- `x` — The first cathetus of a right-angle triangle. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md). +- `y` — The second cathetus of a right-angle triangle. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md). **Returned value** @@ -838,6 +838,7 @@ degrees(x) **Arguments** +- `x` — Input in radians. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md). - `x` — Input in radians. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md) or [Decimal*](../data-types/decimal.md). **Returned value** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index dfe1224f7b8..e22dd5d827c 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -212,7 +212,7 @@ toTypeName(x) ## blockSize {#blockSize} -In ClickHouse, queries are processed in blocks (chunks). +In ClickHouse, queries are processed in [blocks](../../development/architecture.md/#block-block) (chunks). This function returns the size (row count) of the block the function is called on. **Syntax** @@ -221,6 +221,33 @@ This function returns the size (row count) of the block the function is called o blockSize() ``` +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (n UInt8) ENGINE = Memory; + +INSERT INTO test +SELECT * FROM system.numbers LIMIT 5; + +SELECT blockSize() +FROM test; +``` + +Result: + +```response + ┌─blockSize()─┐ +1. │ 5 │ +2. │ 5 │ +3. │ 5 │ +4. │ 5 │ +5. │ 5 │ + └─────────────┘ +``` + ## byteSize Returns an estimation of uncompressed byte size of its arguments in memory. @@ -735,6 +762,8 @@ LIMIT 10 Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. +The opposite operations of this function are [parseReadableSize](#parsereadablesize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). + **Syntax** ```sql @@ -766,6 +795,8 @@ Result: Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. +The opposite operations of this function are [parseReadableSize](#parsereadablesize), [parseReadableSizeOrZero](#parsereadablesizeorzero), and [parseReadableSizeOrNull](#parsereadablesizeornull). + **Syntax** ```sql @@ -890,6 +921,122 @@ SELECT └────────────────────┴────────────────────────────────────────────────┘ ``` +## parseReadableSize + +Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. +If the function is unable to parse the input value, it throws an exception. + +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatreadabledecimalsize). + +**Syntax** + +```sql +formatReadableSize(x) +``` + +**Arguments** + +- `x` : Readable size with ISO/IEC 80000-13 or decimal byte unit ([String](../../sql-reference/data-types/string.md)). + +**Returned value** + +- Number of bytes, rounded up to the nearest integer ([UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +```sql +SELECT + arrayJoin(['1 B', '1 KiB', '3 MB', '5.314 KiB']) AS readable_sizes, + parseReadableSize(readable_sizes) AS sizes; +``` + +```text +┌─readable_sizes─┬───sizes─┐ +│ 1 B │ 1 │ +│ 1 KiB │ 1024 │ +│ 3 MB │ 3000000 │ +│ 5.314 KiB │ 5442 │ +└────────────────┴─────────┘ +``` + +## parseReadableSizeOrNull + +Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. +If the function is unable to parse the input value, it returns `NULL`. + +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatreadabledecimalsize). + +**Syntax** + +```sql +parseReadableSizeOrNull(x) +``` + +**Arguments** + +- `x` : Readable size with ISO/IEC 80000-13 or decimal byte unit ([String](../../sql-reference/data-types/string.md)). + +**Returned value** + +- Number of bytes, rounded up to the nearest integer, or NULL if unable to parse the input (Nullable([UInt64](../../sql-reference/data-types/int-uint.md))). + +**Example** + +```sql +SELECT + arrayJoin(['1 B', '1 KiB', '3 MB', '5.314 KiB', 'invalid']) AS readable_sizes, + parseReadableSizeOrNull(readable_sizes) AS sizes; +``` + +```text +┌─readable_sizes─┬───sizes─┐ +│ 1 B │ 1 │ +│ 1 KiB │ 1024 │ +│ 3 MB │ 3000000 │ +│ 5.314 KiB │ 5442 │ +│ invalid │ ᴺᵁᴸᴸ │ +└────────────────┴─────────┘ +``` + +## parseReadableSizeOrZero + +Given a string containing a byte size and `B`, `KiB`, `KB`, `MiB`, `MB`, etc. as a unit (i.e. [ISO/IEC 80000-13](https://en.wikipedia.org/wiki/ISO/IEC_80000) or decimal byte unit), this function returns the corresponding number of bytes. If the function is unable to parse the input value, it returns `0`. + +The inverse operations of this function are [formatReadableSize](#formatreadablesize) and [formatReadableDecimalSize](#formatreadabledecimalsize). + + +**Syntax** + +```sql +parseReadableSizeOrZero(x) +``` + +**Arguments** + +- `x` : Readable size with ISO/IEC 80000-13 or decimal byte unit ([String](../../sql-reference/data-types/string.md)). + +**Returned value** + +- Number of bytes, rounded up to the nearest integer, or 0 if unable to parse the input ([UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +```sql +SELECT + arrayJoin(['1 B', '1 KiB', '3 MB', '5.314 KiB', 'invalid']) AS readable_sizes, + parseReadableSizeOrZero(readable_sizes) AS sizes; +``` + +```text +┌─readable_sizes─┬───sizes─┐ +│ 1 B │ 1 │ +│ 1 KiB │ 1024 │ +│ 3 MB │ 3000000 │ +│ 5.314 KiB │ 5442 │ +│ invalid │ 0 │ +└────────────────┴─────────┘ +``` + ## parseTimeDelta Parse a sequence of numbers followed by something resembling a time unit. @@ -2564,7 +2711,7 @@ countDigits(x) - Number of digits. [UInt8](../data-types/int-uint.md#uint-ranges). :::note -For `Decimal` values takes into account their scales: calculates result over underlying integer type which is `(value * scale)`. For example: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. I.e. you may check decimal overflow for `Decimal64` with `countDecimal(x) > 18`. It's a slow variant of [isDecimalOverflow](#is-decimal-overflow). +For `Decimal` values takes into account their scales: calculates result over underlying integer type which is `(value * scale)`. For example: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. I.e. you may check decimal overflow for `Decimal64` with `countDecimal(x) > 18`. It's a slow variant of [isDecimalOverflow](#isdecimaloverflow). ::: **Example** @@ -2656,7 +2803,7 @@ currentProfiles() ## enabledProfiles -Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#current-profiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). +Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#currentprofiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). **Syntax** @@ -2769,11 +2916,11 @@ Result: └───────────────────────────┘ ``` -## queryID {#queryID} +## queryID Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see the example). +In contrast to [initialQueryID](#initialqueryid) function, `queryID` can return different results on different shards (see the example). **Syntax** @@ -2807,7 +2954,7 @@ Result: Returns the ID of the initial current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `initial_query_id`. -In contrast to [queryID](#query-id) function, `initialQueryID` returns the same results on different shards (see example). +In contrast to [queryID](#queryid) function, `initialQueryID` returns the same results on different shards (see example). **Syntax** @@ -2894,7 +3041,7 @@ shardCount() **See Also** -- [shardNum()](#shard-num) function example also contains `shardCount()` function call. +- [shardNum()](#shardnum) function example also contains `shardCount()` function call. ## getOSKernelVersion @@ -3568,3 +3715,108 @@ Result: ```response {'version':'1','serial_number':'2D9071D64530052D48308473922C7ADAFA85D6C5','signature_algo':'sha256WithRSAEncryption','issuer':'/CN=marsnet.local CA','not_before':'May 7 17:01:21 2024 GMT','not_after':'May 7 17:01:21 2025 GMT','subject':'/CN=chnode1','pkey_algo':'rsaEncryption'} ``` + +## lowCardinalityIndices + +Returns the position of a value in the dictionary of a [LowCardinality](../data-types/lowcardinality.md) column. Positions start at 1. Since LowCardinality have per-part dictionaries, this function may return different positions for the same value in different parts. + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- The position of the value in the dictionary of the current part. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- create two parts: + +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); + +SELECT s, lowCardinalityIndices(s) FROM test; +``` + +Result: + +```response + ┌─s──┬─lowCardinalityIndices(s)─┐ +1. │ ab │ 1 │ +2. │ cd │ 2 │ +3. │ ab │ 1 │ +4. │ ab │ 1 │ +5. │ df │ 3 │ + └────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ + 6. │ ef │ 1 │ + 7. │ cd │ 2 │ + 8. │ ab │ 3 │ + 9. │ cd │ 2 │ +10. │ ef │ 1 │ + └────┴──────────────────────────┘ +``` +## lowCardinalityKeys + +Returns the dictionary values of a [LowCardinality](../data-types/lowcardinality.md) column. If the block is smaller or larger than the dictionary size, the result will be truncated or extended with default values. Since LowCardinality have per-part dictionaries, this function may return different dictionary values in different parts. + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- The dictionary keys. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- create two parts: + +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); + +SELECT s, lowCardinalityKeys(s) FROM test; +``` + +Result: + +```response + ┌─s──┬─lowCardinalityKeys(s)─┐ +1. │ ef │ │ +2. │ cd │ ef │ +3. │ ab │ cd │ +4. │ cd │ ab │ +5. │ ef │ │ + └────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ + 6. │ ab │ │ + 7. │ cd │ ab │ + 8. │ ab │ cd │ + 9. │ ab │ df │ +10. │ df │ │ + └────┴───────────────────────┘ +``` diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index d18185c5013..e2f471d47eb 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -6,49 +6,90 @@ sidebar_label: Rounding # Rounding Functions -## floor(x\[, N\]) +## floor -Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isn’t exact. -‘N’ is an integer constant, optional parameter. By default it is zero, which means to round to an integer. -‘N’ may be negative. +Returns the largest rounded number less than or equal `x`. +A rounded number is a multiple of 1 / 10 * N, or the nearest number of the appropriate data type if 1 / 10 * N isn’t exact. -Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` +Integer arguments may be rounded with negative `N` argument, with non-negative `N` the function returns `x`, i.e. does nothing. -`x` is any numeric type. The result is a number of the same type. -For integer arguments, it makes sense to round with a negative `N` value (for non-negative `N`, the function does not do anything). -If rounding causes overflow (for example, floor(-128, -1)), an implementation-specific result is returned. +If rounding causes an overflow (for example, `floor(-128, -1)`), the result is undefined. -## ceil(x\[, N\]), ceiling(x\[, N\]) +**Syntax** -Returns the smallest round number that is greater than or equal to `x`. In every other way, it is the same as the `floor` function (see above). +``` sql +floor(x[, N]) +``` -## trunc(x\[, N\]), truncate(x\[, N\]) +**Parameters** -Returns the round number with largest absolute value that has an absolute value less than or equal to `x`‘s. In every other way, it is the same as the ’floor’ function (see above). +- `x` - The value to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` . [(U)Int*](../data-types/int-uint.md). The default is zero, which means rounding to an integer. Can be negative. + +**Returned value** + +A rounded number of the same type as `x`. + +**Examples** + +Query: + +```sql +SELECT floor(123.45, 1) AS rounded +``` + +Result: + +``` +┌─rounded─┐ +│ 123.4 │ +└─────────┘ +``` + +Query: + +```sql +SELECT floor(123.45, -1) +``` + +Result: + +``` +┌─rounded─┐ +│ 120 │ +└─────────┘ +``` + +## ceiling + +Like `floor` but returns the smallest rounded number greater than or equal `x`. + +**Syntax** + +``` sql +ceiling(x[, N]) +``` + +Alias: `ceil` + +## truncate + +Like `floor` but returns the rounded number with largest absolute value that has an absolute value less than or equal to `x`‘s. **Syntax** ```sql -trunc(input, precision) +truncate(x[, N]) ``` -Alias: `truncate`. - -**Parameters** - -- `input`: A numeric type ([Float](../data-types/float.md), [Decimal](../data-types/decimal.md) or [Integer](../data-types/int-uint.md)). -- `precision`: An [Integer](../data-types/int-uint.md) type. - -**Returned value** - -- A data type of `input`. +Alias: `trunc`. **Example** Query: ```sql -SELECT trunc(123.499, 1) as res; +SELECT truncate(123.499, 1) as res; ``` ```response @@ -57,37 +98,40 @@ SELECT trunc(123.499, 1) as res; └───────┘ ``` -## round(x\[, N\]) +## round Rounds a value to a specified number of decimal places. -The function returns the nearest number of the specified order. In case when given number has equal distance to surrounding numbers, the function uses banker’s rounding for float number types and rounds away from zero for the other number types (Decimal). +The function returns the nearest number of the specified order. +If the input value has equal distance to two neighboring numbers, the function uses banker’s rounding for [Float*](../data-types/float.md) inputs and rounds away from zero for the other number types ([Decimal*](../data-types/decimal.md). + +**Syntax** ``` sql -round(expression [, decimal_places]) +round(x[, N]) ``` **Arguments** -- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — An integer value. - - If `decimal-places > 0` then the function rounds the value to the right of the decimal point. - - If `decimal-places < 0` then the function rounds the value to the left of the decimal point. - - If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted. +- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` — The number of decimal places to round to. Integer. Defaults to `0`. + - If `N > 0`, the function rounds to the right of the decimal point. + - If `N < 0`, the function rounds to the left of the decimal point. + - If `N = 0`, the function rounds to the next integer. **Returned value:** -The rounded number of the same type as the input number. +A rounded number of the same type as `x`. **Examples** -Example of usage with Float: +Example with `Float` inputs: -``` sql +```sql SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; ``` -``` text +``` ┌───x─┬─round(divide(number, 2))─┐ │ 0 │ 0 │ │ 0.5 │ 0 │ @@ -95,13 +139,13 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; └─────┴──────────────────────────┘ ``` -Example of usage with Decimal: +Example with `Decimal` inputs: -``` sql +```sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3; ``` -``` text +``` ┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ │ 0 │ 0 │ │ 0.5 │ 1 │ @@ -109,14 +153,14 @@ SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIM └─────┴──────────────────────────────────────────────────┘ ``` -If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros` +To retain trailing zeros, enable setting `output_format_decimal_trailing_zeros`: -``` sql +```sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1; ``` -``` text +``` ┌──────x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ │ 0.0000 │ 0.0000 │ │ 0.5000 │ 1.0000 │ @@ -151,9 +195,15 @@ round(3.65, 1) = 3.6 Rounds a number to a specified decimal position. -- If the rounding number is halfway between two numbers, the function uses banker’s rounding. Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. +If the rounding number is halfway between two numbers, the function uses banker’s rounding. +Banker's rounding is a method of rounding fractional numbers +When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. +For example: 3.5 rounds up to 4, 2.5 rounds down to 2. +It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). +The [round](#round) function performs the same rounding for floating point numbers. +The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. -- In other cases, the function rounds numbers to the nearest integer. +In other cases, the function rounds numbers to the nearest integer. Using banker’s rounding, you can reduce the effect that rounding numbers has on the results of summing or subtracting these numbers. @@ -166,16 +216,20 @@ For example, sum numbers 1.5, 2.5, 3.5, 4.5 with different rounding: **Syntax** ``` sql -roundBankers(expression [, decimal_places]) +roundBankers(x [, N]) ``` **Arguments** -- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — Decimal places. An integer number. - - `decimal-places > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. - - `decimal-places < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. - - `decimal-places = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. + - `N > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. + - `N < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. + - `N = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. + +- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` — The number of decimal places to round to. Integer. Defaults to `0`. + - If `N > 0`, the function rounds to the right of the decimal point. + - If `N < 0`, the function rounds to the left of the decimal point. + - If `N = 0`, the function rounds to the next integer. **Returned value** @@ -185,13 +239,13 @@ A value rounded by the banker’s rounding method. Query: -``` sql +```sql SELECT number / 2 AS x, roundBankers(x, 0) AS b fROM system.numbers limit 10 ``` Result: -``` text +``` ┌───x─┬─b─┐ │ 0 │ 0 │ │ 0.5 │ 0 │ @@ -208,7 +262,7 @@ Result: Examples of Banker’s rounding: -``` text +``` roundBankers(0.4) = 0 roundBankers(-3.5) = -4 roundBankers(4.5) = 4 @@ -220,7 +274,7 @@ roundBankers(10.755, 2) = 10.76 **See Also** -- [round](#rounding_functions-round) +- [round](#round) ## roundToExp2 @@ -264,7 +318,7 @@ Result: ## roundDuration -Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. +Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. **Syntax** diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 342ca2b9f03..a258456345e 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1136,16 +1136,136 @@ SELECT tryBase58Decode('3dc8KtHrwM') as res, tryBase58Decode('invalid') as res_i ## base64Encode -Encodes a String or FixedString as base64. +Encodes a String or FixedString as base64, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-4). Alias: `TO_BASE64`. +**Syntax** + +```sql +base64Encode(plaintext) +``` + +**Arguments** + +- `plaintext` — [String](../data-types/string.md) column or constant. + +**Returned value** + +- A string containing the encoded value of the argument. + +**Example** + +``` sql +SELECT base64Encode('clickhouse'); +``` + +Result: + +```result +┌─base64Encode('clickhouse')─┐ +│ Y2xpY2tob3VzZQ== │ +└────────────────────────────┘ +``` + +## base64UrlEncode + +Encodes an URL (String or FixedString) as base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). + +**Syntax** + +```sql +base64UrlEncode(url) +``` + +**Arguments** + +- `url` — [String](../data-types/string.md) column or constant. + +**Returned value** + +- A string containing the encoded value of the argument. + +**Example** + +``` sql +SELECT base64UrlEncode('https://clickhouse.com'); +``` + +Result: + +```result +┌─base64UrlEncode('https://clickhouse.com')─┐ +│ aHR0cDovL2NsaWNraG91c2UuY29t │ +└───────────────────────────────────────────┘ +``` + ## base64Decode -Decodes a base64-encoded String or FixedString. Throws an exception in case of error. +Accepts a String and decodes it from base64, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-4). Throws an exception in case of an error. Alias: `FROM_BASE64`. +**Syntax** + +```sql +base64Decode(encoded) +``` + +**Arguments** + +- `encoded` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, an exception is thrown. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Example** + +``` sql +SELECT base64Decode('Y2xpY2tob3VzZQ=='); +``` + +Result: + +```result +┌─base64Decode('Y2xpY2tob3VzZQ==')─┐ +│ clickhouse │ +└──────────────────────────────────┘ +``` + +## base64UrlDecode + +Accepts a base64-encoded URL and decodes it from base64 with URL-specific modifications, according to [RFC 4648](https://datatracker.ietf.org/doc/html/rfc4648#section-5). Throws an exception in case of an error. + +**Syntax** + +```sql +base64UrlDecode(encodedUrl) +``` + +**Arguments** + +- `encodedUrl` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, an exception is thrown. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Example** + +``` sql +SELECT base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t'); +``` + +Result: + +```result +┌─base64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t')─┐ +│ https://clickhouse.com │ +└─────────────────────────────────────────────────┘ +``` + ## tryBase64Decode Like `base64Decode` but returns an empty string in case of error. @@ -1156,9 +1276,13 @@ Like `base64Decode` but returns an empty string in case of error. tryBase64Decode(encoded) ``` -**Parameters** +**Arguments** -- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error. +- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, returns an empty string. + +**Returned value** + +- A string containing the decoded value of the argument. **Examples** @@ -1169,9 +1293,41 @@ SELECT tryBase64Decode('RW5jb2RlZA==') as res, tryBase64Decode('invalid') as res ``` ```response -┌─res─────┬─res_invalid─┐ -│ Encoded │ │ -└─────────┴─────────────┘ +┌─res────────┬─res_invalid─┐ +│ clickhouse │ │ +└────────────┴─────────────┘ +``` + +## tryBase64UrlDecode + +Like `base64UrlDecode` but returns an empty string in case of error. + +**Syntax** + +```sql +tryBase64UrlDecode(encodedUrl) +``` + +**Parameters** + +- `encodedUrl`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Examples** + +Query: + +```sql +SELECT tryBase64UrlDecode('aHR0cDovL2NsaWNraG91c2UuY29t') as res, tryBase64Decode('aHR0cHM6Ly9jbGlja') as res_invalid; +``` + +```response +┌─res────────────────────┬─res_invalid─┐ +│ https://clickhouse.com │ │ +└────────────────────────┴─────────────┘ ``` ## endsWith {#endswith} @@ -1994,7 +2150,7 @@ Result: ## stringJaccardIndexUTF8 -Like [stringJaccardIndex](#stringJaccardIndex) but for UTF8-encoded strings. +Like [stringJaccardIndex](#stringjaccardindex) but for UTF8-encoded strings. ## editDistance @@ -2022,6 +2178,32 @@ Result: Alias: levenshteinDistance +## editDistanceUTF8 + +Calculates the [edit distance](https://en.wikipedia.org/wiki/Edit_distance) between two UTF8 strings. + +**Syntax** + +```sql +editDistanceUTF8(string1, string2) +``` + +**Examples** + +``` sql +SELECT editDistanceUTF8('我是谁', '我是我'); +``` + +Result: + +``` text +┌─editDistanceUTF8('我是谁', '我是我')──┐ +│ 1 │ +└─────────────────────────────────────┘ +``` + +Alias: levenshteinDistanceUTF8 + ## damerauLevenshteinDistance Calculates the [Damerau-Levenshtein distance](https://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance) between two byte strings. diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index d261cff3580..b7ba1d4feb7 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -262,7 +262,7 @@ Result: ## multiSearchAllPositionsUTF8 -Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings. +Like [multiSearchAllPositions](#multisearchallpositions) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings. **Syntax** @@ -336,7 +336,7 @@ Result: Like [`position`](#position) but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings. -Functions [`multiSearchFirstPositionCaseInsensitive`](#multiSearchFirstPositionCaseInsensitive), [`multiSearchFirstPositionUTF8`](#multiSearchFirstPositionUTF8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multiSearchFirstPositionCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchFirstPositionCaseInsensitive`](#multisearchfirstpositioncaseinsensitive), [`multiSearchFirstPositionUTF8`](#multisearchfirstpositionutf8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multisearchfirstpositioncaseinsensitiveutf8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** @@ -370,7 +370,7 @@ Result: ## multiSearchFirstPositionCaseInsensitive -Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but ignores case. +Like [`multiSearchFirstPosition`](#multisearchfirstposition) but ignores case. **Syntax** @@ -404,7 +404,7 @@ Result: ## multiSearchFirstPositionUTF8 -Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings. +Like [`multiSearchFirstPosition`](#multisearchfirstposition) but assumes `haystack` and `needle` to be UTF-8 strings. **Syntax** @@ -440,7 +440,7 @@ Result: ## multiSearchFirstPositionCaseInsensitiveUTF8 -Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case. +Like [`multiSearchFirstPosition`](#multisearchfirstposition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case. **Syntax** @@ -478,7 +478,7 @@ Result: Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. -Functions [`multiSearchFirstIndexCaseInsensitive`](#multiSearchFirstIndexCaseInsensitive), [`multiSearchFirstIndexUTF8`](#multiSearchFirstIndexUTF8) and [`multiSearchFirstIndexCaseInsensitiveUTF8`](#multiSearchFirstIndexCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchFirstIndexCaseInsensitive`](#multisearchfirstindexcaseinsensitive), [`multiSearchFirstIndexUTF8`](#multisearchfirstindexutf8) and [`multiSearchFirstIndexCaseInsensitiveUTF8`](#multisearchfirstindexcaseinsensitiveutf8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** @@ -615,7 +615,7 @@ Result: Returns 1, if at least one string needlei matches the string `haystack` and 0 otherwise. -Functions [`multiSearchAnyCaseInsensitive`](#multiSearchAnyCaseInsensitive), [`multiSearchAnyUTF8`](#multiSearchAnyUTF8) and []`multiSearchAnyCaseInsensitiveUTF8`](#multiSearchAnyCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchAnyCaseInsensitive`](#multisearchanycaseinsensitive), [`multiSearchAnyUTF8`](#multisearchanyutf8) and [`multiSearchAnyCaseInsensitiveUTF8`](#multisearchanycaseinsensitiveutf8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** @@ -719,7 +719,7 @@ Result: ## multiSearchAnyCaseInsensitiveUTF8 -Like [multiSearchAnyUTF8](#multiSearchAnyUTF8) but ignores case. +Like [multiSearchAnyUTF8](#multisearchanyutf8) but ignores case. *Syntax** @@ -880,7 +880,7 @@ extractAll(haystack, pattern) Matches all groups of the `haystack` string using the `pattern` regular expression. Returns an array of arrays, where the first array includes all fragments matching the first group, the second array - matching the second group, etc. -This function is slower than [extractAllGroupsVertical](#extractallgroups-vertical). +This function is slower than [extractAllGroupsVertical](#extractallgroupsvertical). **Syntax** @@ -952,7 +952,7 @@ Result: └────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## like {#like} +## like Returns whether string `haystack` matches the LIKE expression `pattern`. @@ -1215,7 +1215,7 @@ Result: ## ngramSearchCaseInsensitive -Provides a case-insensitive variant of [ngramSearch](#ngramSearch). +Provides a case-insensitive variant of [ngramSearch](#ngramsearch). **Syntax** @@ -1630,7 +1630,7 @@ Result: ## hasSubsequenceCaseInsensitive -Like [hasSubsequence](#hasSubsequence) but searches case-insensitively. +Like [hasSubsequence](#hassubsequence) but searches case-insensitively. **Syntax** @@ -1665,7 +1665,7 @@ Result: ## hasSubsequenceUTF8 -Like [hasSubsequence](#hasSubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. +Like [hasSubsequence](#hassubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. **Syntax** @@ -1700,7 +1700,7 @@ Result: ## hasSubsequenceCaseInsensitiveUTF8 -Like [hasSubsequenceUTF8](#hasSubsequenceUTF8) but searches case-insensitively. +Like [hasSubsequenceUTF8](#hassubsequenceutf8) but searches case-insensitively. **Syntax** diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index d9c18e2a0a2..ad40725d680 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -6,7 +6,7 @@ sidebar_label: Maps ## map -Arranges `key:value` pairs into [Map(key, value)](../data-types/map.md) data type. +Creates a value of type [Map(key, value)](../data-types/map.md) from key-value pairs. **Syntax** @@ -16,12 +16,12 @@ map(key1, value1[, key2, value2, ...]) **Arguments** -- `key` — The key part of the pair. Arbitrary type, except [Nullable](../data-types/nullable.md) and [LowCardinality](../data-types/lowcardinality.md) nested with [Nullable](../data-types/nullable.md). -- `value` — The value part of the pair. Arbitrary type, including [Map](../data-types/map.md) and [Array](../data-types/array.md). +- `key_n` — The keys of the map entries. Any type supported as key type of [Map](../data-types/map.md). +- `value_n` — The values of the map entries. Any type supported as value type of [Map](../data-types/map.md). **Returned value** -- Data structure as `key:value` pairs. [Map(key, value)](../data-types/map.md). +- A map containing `key:value` pairs. [Map(key, value)](../data-types/map.md). **Examples** @@ -41,35 +41,16 @@ Result: └──────────────────────────────────────────────────┘ ``` -Query: - -```sql -CREATE TABLE table_map (a Map(String, UInt64)) ENGINE = MergeTree() ORDER BY a; -INSERT INTO table_map SELECT map('key1', number, 'key2', number * 2) FROM numbers(3); -SELECT a['key2'] FROM table_map; -``` - -Result: - -```text -┌─arrayElement(a, 'key2')─┐ -│ 0 │ -│ 2 │ -│ 4 │ -└─────────────────────────┘ -``` - -**See Also** - -- [Map(key, value)](../data-types/map.md) data type - ## mapFromArrays -Merges an [Array](../data-types/array.md) of keys and an [Array](../data-types/array.md) of values into a [Map(key, value)](../data-types/map.md). Notice that the second argument could also be a [Map](../data-types/map.md), thus it is casted to an Array when executing. +Creates a map from an array of keys and an array of values. +The function is a convenient alternative to syntax `CAST([...], 'Map(key_type, value_type)')`. +For example, instead of writing +- `CAST((['aa', 'bb'], [4, 5]), 'Map(String, UInt32)')`, or +- `CAST([('aa',4), ('bb',5)], 'Map(String, UInt32)')` -The function is a more convenient alternative to `CAST((key_array, value_array_or_map), 'Map(key_type, value_type)')`. For example, instead of writing `CAST((['aa', 'bb'], [4, 5]), 'Map(String, UInt32)')`, you can write `mapFromArrays(['aa', 'bb'], [4, 5])`. - +you can write `mapFromArrays(['aa', 'bb'], [4, 5])`. **Syntax** @@ -81,12 +62,12 @@ Alias: `MAP_FROM_ARRAYS(keys, values)` **Arguments** -- `keys` — Given key array to create a map from. The nested type of array must be: [String](../data-types/string.md), [Integer](../data-types/int-uint.md), [LowCardinality](../data-types/lowcardinality.md), [FixedString](../data-types/fixedstring.md), [UUID](../data-types/uuid.md), [Date](../data-types/date.md), [DateTime](../data-types/datetime.md), [Date32](../data-types/date32.md), [Enum](../data-types/enum.md) -- `values` - Given value array or map to create a map from. +- `keys` — Array of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type. +- `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md). **Returned value** -- A map whose keys and values are constructed from the key array and value array/map. +- A map with keys and values constructed from the key array and value array/map. **Example** @@ -94,14 +75,25 @@ Query: ```sql select mapFromArrays(['a', 'b', 'c'], [1, 2, 3]) +``` +Result: +``` ┌─mapFromArrays(['a', 'b', 'c'], [1, 2, 3])─┐ │ {'a':1,'b':2,'c':3} │ └───────────────────────────────────────────┘ +``` +`mapFromArrays` also accepts arguments of type [Map](../data-types/map.md). These are casted to array of tuples during execution. + +```sql SELECT mapFromArrays([1, 2, 3], map('a', 1, 'b', 2, 'c', 3)) +``` +Result: + +``` ┌─mapFromArrays([1, 2, 3], map('a', 1, 'b', 2, 'c', 3))─┐ │ {1:('a',1),2:('b',2),3:('c',3)} │ └───────────────────────────────────────────────────────┘ @@ -109,9 +101,11 @@ SELECT mapFromArrays([1, 2, 3], map('a', 1, 'b', 2, 'c', 3)) ## extractKeyValuePairs -Extracts key-value pairs, i.e. a [Map(String, String)](../data-types/map.md), from a string. Parsing is robust towards noise (e.g. log files). - -A key-value pair consists of a key, followed by a `key_value_delimiter` and a value. Key value pairs must be separated by `pair_delimiter`. Quoted keys and values are also supported. +Converts a string of key-value pairs to a [Map(String, String)](../data-types/map.md). +Parsing is tolerant towards noise (e.g. log files). +Key-value pairs in the input string consist of a key, followed by a key-value delimiter, and a value. +Key value pairs are separated by a pair delimiter. +Keys and values can be quoted. **Syntax** @@ -126,17 +120,17 @@ Alias: **Arguments** - `data` - String to extract key-value pairs from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `key_value_delimiter` - Character to be used as delimiter between the key and the value. Defaults to `:`. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `pair_delimiters` - Set of character to be used as delimiters between pairs. Defaults to ` `, `,` and `;`. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `quoting_character` - Character to be used as quoting character. Defaults to `"`. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `key_value_delimiter` - Single character delimiting keys and values. Defaults to `:`. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `pair_delimiters` - Set of character delimiting pairs. Defaults to ` `, `,` and `;`. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `quoting_character` - Single character used as quoting character. Defaults to `"`. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). **Returned values** -- A [Map(String, String)](../data-types/map.md) of key-value pairs. +- A of key-value pairs. Type: [Map(String, String)](../data-types/map.md) **Examples** -Simple case: +Query ``` sql SELECT extractKeyValuePairs('name:neymar, age:31 team:psg,nationality:brazil') as kv @@ -150,7 +144,7 @@ Result: └─────────────────────────────────────────────────────────────────────────┘ ``` -Single quote as quoting character: +With a single quote `'` as quoting character: ``` sql SELECT extractKeyValuePairs('name:\'neymar\';\'age\':31;team:psg;nationality:brazil,last_key:last_value', ':', ';,', '\'') as kv @@ -178,9 +172,29 @@ Result: └────────────────────────┘ ``` +To restore a map string key-value pairs serialized with `toString`: + +```sql +SELECT + map('John', '33', 'Paula', '31') AS m, + toString(m) as map_serialized, + extractKeyValuePairs(map_serialized, ':', ',', '\'') AS map_restored +FORMAT Vertical; +``` + +Result: + +``` +Row 1: +────── +m: {'John':'33','Paula':'31'} +map_serialized: {'John':'33','Paula':'31'} +map_restored: {'John':'33','Paula':'31'} +``` + ## extractKeyValuePairsWithEscaping -Same as `extractKeyValuePairs` but with escaping support. +Same as `extractKeyValuePairs` but supports escaping. Supported escape sequences: `\x`, `\N`, `\a`, `\b`, `\e`, `\f`, `\n`, `\r`, `\t`, `\v` and `\0`. Non standard escape sequences are returned as it is (including the backslash) unless they are one of the following: @@ -229,20 +243,6 @@ Arguments are [maps](../data-types/map.md) or [tuples](../data-types/tuple.md#tu **Example** -Query with a tuple: - -```sql -SELECT mapAdd(([toUInt8(1), 2], [1, 1]), ([toUInt8(1), 2], [1, 1])) as res, toTypeName(res) as type; -``` - -Result: - -```text -┌─res───────────┬─type───────────────────────────────┐ -│ ([1,2],[2,2]) │ Tuple(Array(UInt8), Array(UInt64)) │ -└───────────────┴────────────────────────────────────┘ -``` - Query with `Map` type: ```sql @@ -257,6 +257,20 @@ Result: └──────────────────────────────┘ ``` +Query with a tuple: + +```sql +SELECT mapAdd(([toUInt8(1), 2], [1, 1]), ([toUInt8(1), 2], [1, 1])) as res, toTypeName(res) as type; +``` + +Result: + +```text +┌─res───────────┬─type───────────────────────────────┐ +│ ([1,2],[2,2]) │ Tuple(Array(UInt8), Array(UInt64)) │ +└───────────────┴────────────────────────────────────┘ +``` + ## mapSubtract Collect all the keys and subtract corresponding values. @@ -277,20 +291,6 @@ Arguments are [maps](../data-types/map.md) or [tuples](../data-types/tuple.md#tu **Example** -Query with a tuple map: - -```sql -SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt32(2), 1])) as res, toTypeName(res) as type; -``` - -Result: - -```text -┌─res────────────┬─type──────────────────────────────┐ -│ ([1,2],[-1,0]) │ Tuple(Array(UInt8), Array(Int64)) │ -└────────────────┴───────────────────────────────────┘ -``` - Query with `Map` type: ```sql @@ -305,55 +305,57 @@ Result: └───────────────────────────────────┘ ``` -## mapPopulateSeries - -Fills missing keys in the maps (key and value array pair), where keys are integers. Also, it supports specifying the max key, which is used to extend the keys array. - -**Syntax** +Query with a tuple map: ```sql -mapPopulateSeries(keys, values[, max]) -mapPopulateSeries(map[, max]) -``` - -Generates a map (a tuple with two arrays or a value of `Map` type, depending on the arguments), where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from the map with a step size of one, and corresponding values. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. - -For array arguments the number of elements in `keys` and `values` must be the same for each row. - -**Arguments** - -Arguments are [maps](../data-types/map.md) or two [arrays](../data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. - -Mapped arrays: - -- `keys` — Array of keys. [Array](../data-types/array.md#data-type-array)([Int](../data-types/int-uint.md#uint-ranges)). -- `values` — Array of values. [Array](../data-types/array.md#data-type-array)([Int](../data-types/int-uint.md#uint-ranges)). -- `max` — Maximum key value. Optional. [Int8, Int16, Int32, Int64, Int128, Int256](../data-types/int-uint.md#int-ranges). - -or - -- `map` — Map with integer keys. [Map](../data-types/map.md). - -**Returned value** - -- Depending on the arguments returns a [map](../data-types/map.md) or a [tuple](../data-types/tuple.md#tuplet1-t2) of two [arrays](../data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. - -**Example** - -Query with mapped arrays: - -```sql -SELECT mapPopulateSeries([1,2,4], [11,22,44], 5) AS res, toTypeName(res) AS type; +SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt32(2), 1])) as res, toTypeName(res) as type; ``` Result: ```text -┌─res──────────────────────────┬─type──────────────────────────────┐ -│ ([1,2,3,4,5],[11,22,0,44,0]) │ Tuple(Array(UInt8), Array(UInt8)) │ -└──────────────────────────────┴───────────────────────────────────┘ +┌─res────────────┬─type──────────────────────────────┐ +│ ([1,2],[-1,0]) │ Tuple(Array(UInt8), Array(Int64)) │ +└────────────────┴───────────────────────────────────┘ ``` +## mapPopulateSeries + +Fills missing key-value pairs in a map with integer keys. +To support extending the keys beyond the largest value, a maximum key can be specified. +More specifically, the function returns a map in which the the keys form a series from the smallest to the largest key (or `max` argument if it specified) with step size of 1, and corresponding values. +If no value is specified for a key, a default value is used as value. +In case keys repeat, only the first value (in order of appearance) is associated with the key. + +**Syntax** + +```sql +mapPopulateSeries(map[, max]) +mapPopulateSeries(keys, values[, max]) +``` + +For array arguments the number of elements in `keys` and `values` must be the same for each row. + +**Arguments** + +Arguments are [Maps](../data-types/map.md) or two [Arrays](../data-types/array.md#data-type-array), where the first and second array contains keys and values for the each key. + +Mapped arrays: + +- `map` — Map with integer keys. [Map](../data-types/map.md). + +or + +- `keys` — Array of keys. [Array](../data-types/array.md#data-type-array)([Int](../data-types/int-uint.md#uint-ranges)). +- `values` — Array of values. [Array](../data-types/array.md#data-type-array)([Int](../data-types/int-uint.md#uint-ranges)). +- `max` — Maximum key value. Optional. [Int8, Int16, Int32, Int64, Int128, Int256](../data-types/int-uint.md#int-ranges). + +**Returned value** + +- Depending on the arguments a [Map](../data-types/map.md) or a [Tuple](../data-types/tuple.md#tuplet1-t2) of two [Arrays](../data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. + +**Example** + Query with `Map` type: ```sql @@ -368,9 +370,23 @@ Result: └─────────────────────────────────────────┘ ``` +Query with mapped arrays: + +```sql +SELECT mapPopulateSeries([1,2,4], [11,22,44], 5) AS res, toTypeName(res) AS type; +``` + +Result: + +```text +┌─res──────────────────────────┬─type──────────────────────────────┐ +│ ([1,2,3,4,5],[11,22,0,44,0]) │ Tuple(Array(UInt8), Array(UInt8)) │ +└──────────────────────────────┴───────────────────────────────────┘ +``` + ## mapContains -Determines whether the `map` contains the `key` parameter. +Returns if a given key is contained in a given map. **Syntax** @@ -381,7 +397,7 @@ mapContains(map, key) **Arguments** - `map` — Map. [Map](../data-types/map.md). -- `key` — Key. Type matches the type of keys of `map` parameter. +- `key` — Key. Type must match the key type of `map`. **Returned value** @@ -392,11 +408,11 @@ mapContains(map, key) Query: ```sql -CREATE TABLE test (a Map(String,String)) ENGINE = Memory; +CREATE TABLE tab (a Map(String, String)) ENGINE = Memory; -INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); +INSERT INTO tab VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); -SELECT mapContains(a, 'name') FROM test; +SELECT mapContains(a, 'name') FROM tab; ``` @@ -411,9 +427,11 @@ Result: ## mapKeys -Returns all keys from the `map` parameter. +Returns the keys of a given map. -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [keys](../data-types/map.md#map-subcolumns) subcolumn instead of reading and processing the whole column data. The query `SELECT mapKeys(m) FROM table` transforms to `SELECT m.keys FROM table`. +This function can be optimized by enabling setting [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). +With enabled setting, the function only reads the [keys](../data-types/map.md#map-subcolumns) subcolumn instead the whole map. +The query `SELECT mapKeys(m) FROM table` is transformed to `SELECT m.keys FROM table`. **Syntax** @@ -434,11 +452,11 @@ mapKeys(map) Query: ```sql -CREATE TABLE test (a Map(String,String)) ENGINE = Memory; +CREATE TABLE tab (a Map(String, String)) ENGINE = Memory; -INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); +INSERT INTO tab VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); -SELECT mapKeys(a) FROM test; +SELECT mapKeys(a) FROM tab; ``` Result: @@ -452,9 +470,11 @@ Result: ## mapValues -Returns all values from the `map` parameter. +Returns the values of a given map. -Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [values](../data-types/map.md#map-subcolumns) subcolumn instead of reading and processing the whole column data. The query `SELECT mapValues(m) FROM table` transforms to `SELECT m.values FROM table`. +This function can be optimized by enabling setting [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns). +With enabled setting, the function only reads the [values](../data-types/map.md#map-subcolumns) subcolumn instead the whole map. +The query `SELECT mapValues(m) FROM table` is transformed to `SELECT m.values FROM table`. **Syntax** @@ -475,11 +495,11 @@ mapValues(map) Query: ```sql -CREATE TABLE test (a Map(String,String)) ENGINE = Memory; +CREATE TABLE tab (a Map(String, String)) ENGINE = Memory; -INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); +INSERT INTO tab VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); -SELECT mapValues(a) FROM test; +SELECT mapValues(a) FROM tab; ``` Result: @@ -512,11 +532,11 @@ mapContainsKeyLike(map, pattern) Query: ```sql -CREATE TABLE test (a Map(String,String)) ENGINE = Memory; +CREATE TABLE tab (a Map(String, String)) ENGINE = Memory; -INSERT INTO test VALUES ({'abc':'abc','def':'def'}), ({'hij':'hij','klm':'klm'}); +INSERT INTO tab VALUES ({'abc':'abc','def':'def'}), ({'hij':'hij','klm':'klm'}); -SELECT mapContainsKeyLike(a, 'a%') FROM test; +SELECT mapContainsKeyLike(a, 'a%') FROM tab; ``` Result: @@ -530,6 +550,8 @@ Result: ## mapExtractKeyLike +Give a map with string keys and a LIKE pattern, this function returns a map with elements where the key matches the pattern. + **Syntax** ```sql @@ -543,18 +565,18 @@ mapExtractKeyLike(map, pattern) **Returned value** -- A map contained elements the key of which matches the specified pattern. If there are no elements matched the pattern, it will return an empty map. +- A map containing elements the key matching the specified pattern. If no elements match the pattern, an empty map is returned. **Example** Query: ```sql -CREATE TABLE test (a Map(String,String)) ENGINE = Memory; +CREATE TABLE tab (a Map(String, String)) ENGINE = Memory; -INSERT INTO test VALUES ({'abc':'abc','def':'def'}), ({'hij':'hij','klm':'klm'}); +INSERT INTO tab VALUES ({'abc':'abc','def':'def'}), ({'hij':'hij','klm':'klm'}); -SELECT mapExtractKeyLike(a, 'a%') FROM test; +SELECT mapExtractKeyLike(a, 'a%') FROM tab; ``` Result: @@ -568,6 +590,8 @@ Result: ## mapApply +Applies a function to each element of a map. + **Syntax** ```sql @@ -608,6 +632,8 @@ Result: ## mapFilter +Filters a map by applying a function to each map element. + **Syntax** ```sql @@ -623,7 +649,6 @@ mapFilter(func, map) - Returns a map containing only the elements in `map` for which `func(map1[i], ..., mapN[i])` returns something other than 0. - **Example** Query: @@ -647,7 +672,6 @@ Result: └─────────────────────┘ ``` - ## mapUpdate **Syntax** @@ -683,6 +707,9 @@ Result: ## mapConcat +Concatenates multiple maps based on the equality of their keys. +If elements with the same key exist in more than one input map, all elements are added to the result map, but only the first one is accessible via operator `[]` + **Syntax** ```sql @@ -691,11 +718,11 @@ mapConcat(maps) **Arguments** -- `maps` – Arbitrary number of arguments of [Map](../data-types/map.md) type. +- `maps` – Arbitrarily many [Maps](../data-types/map.md). **Returned value** -- Returns a map with concatenated maps passed as arguments. If there are same keys in two or more maps, all of them are added to the result map, but only the first one is accessible via operator `[]` +- Returns a map with concatenated maps passed as arguments. **Examples** @@ -729,9 +756,12 @@ Result: ## mapExists(\[func,\], map) -Returns 1 if there is at least one key-value pair in `map` for which `func(key, value)` returns something other than 0. Otherwise, it returns 0. +Returns 1 if at least one key-value pair in `map` exists for which `func(key, value)` returns something other than 0. Otherwise, it returns 0. -Note that the `mapExists` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. +:::note +`mapExists` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). +You can pass a lambda function to it as the first argument. +::: **Example** @@ -743,7 +773,7 @@ SELECT mapExists((k, v) -> (v = 1), map('k1', 1, 'k2', 2)) AS res Result: -```text +``` ┌─res─┐ │ 1 │ └─────┘ @@ -753,7 +783,10 @@ Result: Returns 1 if `func(key, value)` returns something other than 0 for all key-value pairs in `map`. Otherwise, it returns 0. -Note that the `mapAll` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. +:::note +Note that the `mapAll` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). +You can pass a lambda function to it as the first argument. +::: **Example** @@ -765,7 +798,7 @@ SELECT mapAll((k, v) -> (v = 1), map('k1', 1, 'k2', 2)) AS res Result: -```text +``` ┌─res─┐ │ 0 │ └─────┘ @@ -773,7 +806,8 @@ Result: ## mapSort(\[func,\], map) -Sorts the elements of the `map` in ascending order. If the `func` function is specified, sorting order is determined by the result of the `func` function applied to the keys and values of the map. +Sorts the elements of a map in ascending order. +If the `func` function is specified, the sorting order is determined by the result of the `func` function applied to the keys and values of the map. **Examples** @@ -801,8 +835,8 @@ For more details see the [reference](../../sql-reference/functions/array-functio ## mapReverseSort(\[func,\], map) -Sorts the elements of the `map` in descending order. If the `func` function is specified, sorting order is determined by the result of the `func` function applied to the keys and values of the map. - +Sorts the elements of a map in descending order. +If the `func` function is specified, the sorting order is determined by the result of the `func` function applied to the keys and values of the map. **Examples** @@ -826,4 +860,4 @@ SELECT mapReverseSort((k, v) -> v, map('key2', 2, 'key3', 1, 'key1', 3)) AS map; └──────────────────────────────┘ ``` -For more details see the [reference](../../sql-reference/functions/array-functions.md#array_functions-reverse-sort) for `arrayReverseSort` function. +For more details see function [arrayReverseSort](../../sql-reference/functions/array-functions.md#array_functions-reverse-sort). diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 5dd1d5ceebe..61e84ca72d1 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -10,7 +10,7 @@ sidebar_label: Type Conversion ClickHouse generally uses the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion). -`to` functions and [cast](#castx-t) behave differently in some cases, for example in case of [LowCardinality](../data-types/lowcardinality.md): [cast](#castx-t) removes [LowCardinality](../data-types/lowcardinality.md) trait `to` functions don't. The same with [Nullable](../data-types/nullable.md), this behaviour is not compatible with SQL standard, and it can be changed using [cast_keep_nullable](../../operations/settings/settings.md/#cast_keep_nullable) setting. +`to` functions and [cast](#cast) behave differently in some cases, for example in case of [LowCardinality](../data-types/lowcardinality.md): [cast](#cast) removes [LowCardinality](../data-types/lowcardinality.md) trait `to` functions don't. The same with [Nullable](../data-types/nullable.md), this behaviour is not compatible with SQL standard, and it can be changed using [cast_keep_nullable](../../operations/settings/settings.md/#cast_keep_nullable) setting. :::note Be aware of potential data loss if values of a datatype are converted to a smaller datatype (for example from `Int64` to `Int32`) or between @@ -70,7 +70,7 @@ Integer value in the `Int8`, `Int16`, `Int32`, `Int64`, `Int128` or `Int256` dat Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. -The behavior of functions for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions. +The behavior of functions for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. **Example** @@ -169,7 +169,7 @@ Converts an input value to the [UInt](../data-types/int-uint.md) data type. This Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. -The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#numeric-conversion-issues), when using the functions. +The behavior of functions for negative arguments and for the [NaN and Inf](../data-types/float.md/#data_type-float-nan-inf) arguments is undefined. If you pass a string with a negative number, for example `'-32'`, ClickHouse raises an exception. Remember about [numeric conversions issues](#common-issues-with-data-conversion), when using the functions. **Example** @@ -996,7 +996,7 @@ Result: ## reinterpretAsUInt8 -Performs byte reinterpretation by treating the input value as a value of type UInt8. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt8. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1034,7 +1034,7 @@ Result: ## reinterpretAsUInt16 -Performs byte reinterpretation by treating the input value as a value of type UInt16. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt16. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1072,7 +1072,7 @@ Result: ## reinterpretAsUInt32 -Performs byte reinterpretation by treating the input value as a value of type UInt32. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt32. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1110,7 +1110,7 @@ Result: ## reinterpretAsUInt64 -Performs byte reinterpretation by treating the input value as a value of type UInt64. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt64. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1148,7 +1148,7 @@ Result: ## reinterpretAsUInt128 -Performs byte reinterpretation by treating the input value as a value of type UInt128. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt128. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1186,7 +1186,7 @@ Result: ## reinterpretAsUInt256 -Performs byte reinterpretation by treating the input value as a value of type UInt256. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type UInt256. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1224,7 +1224,7 @@ Result: ## reinterpretAsInt8 -Performs byte reinterpretation by treating the input value as a value of type Int8. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int8. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1262,7 +1262,7 @@ Result: ## reinterpretAsInt16 -Performs byte reinterpretation by treating the input value as a value of type Int16. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int16. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1300,7 +1300,7 @@ Result: ## reinterpretAsInt32 -Performs byte reinterpretation by treating the input value as a value of type Int32. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int32. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1338,7 +1338,7 @@ Result: ## reinterpretAsInt64 -Performs byte reinterpretation by treating the input value as a value of type Int64. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int64. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1376,7 +1376,7 @@ Result: ## reinterpretAsInt128 -Performs byte reinterpretation by treating the input value as a value of type Int128. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int128. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1414,7 +1414,7 @@ Result: ## reinterpretAsInt256 -Performs byte reinterpretation by treating the input value as a value of type Int256. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Int256. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1452,7 +1452,7 @@ Result: ## reinterpretAsFloat32 -Performs byte reinterpretation by treating the input value as a value of type Float32. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Float32. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1486,7 +1486,7 @@ Result: ## reinterpretAsFloat64 -Performs byte reinterpretation by treating the input value as a value of type Float64. Unlike [`CAST`](#castx-t), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. +Performs byte reinterpretation by treating the input value as a value of type Float64. Unlike [`CAST`](#cast), the function does not attempt to preserve the original value - if the target type is not able to represent the input type, the output is meaningless. **Syntax** @@ -1730,7 +1730,7 @@ Result: └─────────────────────┘ ``` -## reinterpret(x, T) +## reinterpret Uses the same source in-memory bytes sequence for `x` value and reinterprets it to destination type. @@ -1766,9 +1766,9 @@ Result: └─────────────┴──────────────┴───────────────┘ ``` -## CAST(x, T) +## CAST -Converts an input value to the specified data type. Unlike the [reinterpret](#type_conversion_function-reinterpret) function, `CAST` tries to present the same value using the new data type. If the conversion can not be done then an exception is raised. +Converts an input value to the specified data type. Unlike the [reinterpret](#reinterpret) function, `CAST` tries to present the same value using the new data type. If the conversion can not be done then an exception is raised. Several syntax variants are supported. **Syntax** @@ -1875,7 +1875,7 @@ Result: Converts `x` to the `T` data type. -The difference from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` does not allow overflow of numeric types during cast if type value `x` does not fit the bounds of type `T`. For example, `accurateCast(-1, 'UInt8')` throws an exception. +The difference from [cast](#cast) is that `accurateCast` does not allow overflow of numeric types during cast if type value `x` does not fit the bounds of type `T`. For example, `accurateCast(-1, 'UInt8')` throws an exception. **Example** @@ -2061,7 +2061,7 @@ Result: └───────────────────────────┴──────────────────────────────┘ ``` -## parseDateTime {#type_conversion_functions-parseDateTime} +## parseDateTime Converts a [String](../data-types/string.md) to [DateTime](../data-types/datetime.md) according to a [MySQL format string](https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-format). @@ -2102,15 +2102,15 @@ Alias: `TO_TIMESTAMP`. ## parseDateTimeOrZero -Same as for [parseDateTime](#type_conversion_functions-parseDateTime) except that it returns zero date when it encounters a date format that cannot be processed. +Same as for [parseDateTime](#parsedatetime) except that it returns zero date when it encounters a date format that cannot be processed. ## parseDateTimeOrNull -Same as for [parseDateTime](#type_conversion_functions-parseDateTime) except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as for [parseDateTime](#parsedatetime) except that it returns `NULL` when it encounters a date format that cannot be processed. Alias: `str_to_date`. -## parseDateTimeInJodaSyntax {#type_conversion_functions-parseDateTimeInJodaSyntax} +## parseDateTimeInJodaSyntax Similar to [parseDateTime](#parsedatetime), except that the format string is in [Joda](https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) instead of MySQL syntax. @@ -2151,11 +2151,11 @@ SELECT parseDateTimeInJodaSyntax('2023-02-24 14:53:31', 'yyyy-MM-dd HH:mm:ss', ' ## parseDateTimeInJodaSyntaxOrZero -Same as for [parseDateTimeInJodaSyntax](#type_conversion_functions-parseDateTimeInJodaSyntax) except that it returns zero date when it encounters a date format that cannot be processed. +Same as for [parseDateTimeInJodaSyntax](#parsedatetimeinjodasyntax) except that it returns zero date when it encounters a date format that cannot be processed. ## parseDateTimeInJodaSyntaxOrNull -Same as for [parseDateTimeInJodaSyntax](#type_conversion_functions-parseDateTimeInJodaSyntax) except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as for [parseDateTimeInJodaSyntax](#parsedatetimeinjodasyntax) except that it returns `NULL` when it encounters a date format that cannot be processed. ## parseDateTimeBestEffort ## parseDateTime32BestEffort @@ -2313,11 +2313,11 @@ Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it r ## parseDateTimeBestEffortUSOrNull -Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortus) function except that it returns `NULL` when it encounters a date format that cannot be processed. ## parseDateTimeBestEffortUSOrZero -Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns zero date (`1970-01-01`) or zero date with time (`1970-01-01 00:00:00`) when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortus) function except that it returns zero date (`1970-01-01`) or zero date with time (`1970-01-01 00:00:00`) when it encounters a date format that cannot be processed. ## parseDateTime64BestEffort @@ -2389,7 +2389,7 @@ Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort), except that Converts input parameter to the [LowCardinality](../data-types/lowcardinality.md) version of same data type. -To convert data from the `LowCardinality` data type use the [CAST](#type_conversion_function-cast) function. For example, `CAST(x as String)`. +To convert data from the `LowCardinality` data type use the [CAST](#cast) function. For example, `CAST(x as String)`. **Syntax** @@ -2423,11 +2423,7 @@ Result: ## toUnixTimestamp64Milli -## toUnixTimestamp64Micro - -## toUnixTimestamp64Nano - -Converts a `DateTime64` to a `Int64` value with fixed sub-second precision. Input value is scaled up or down appropriately depending on it precision. +Converts a `DateTime64` to a `Int64` value with fixed millisecond precision. The input value is scaled up or down appropriately depending on its precision. :::note The output value is a timestamp in UTC, not in the timezone of `DateTime64`. @@ -2437,24 +2433,22 @@ The output value is a timestamp in UTC, not in the timezone of `DateTime64`. ```sql toUnixTimestamp64Milli(value) -toUnixTimestamp64Micro(value) -toUnixTimestamp64Nano(value) ``` **Arguments** -- `value` — DateTime64 value with any precision. +- `value` — DateTime64 value with any precision. [DateTime64](../data-types/datetime64.md). **Returned value** -- `value` converted to the `Int64` data type. +- `value` converted to the `Int64` data type. [Int64](../data-types/int-uint.md). -**Examples** +**Example** Query: ```sql -WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +WITH toDateTime64('2009-02-13 23:31:31.011', 3, 'UTC') AS dt64 SELECT toUnixTimestamp64Milli(dt64); ``` @@ -2462,14 +2456,77 @@ Result: ```response ┌─toUnixTimestamp64Milli(dt64)─┐ -│ 1568650812345 │ +│ 1234567891011 │ └──────────────────────────────┘ ``` +## toUnixTimestamp64Micro + +Converts a `DateTime64` to a `Int64` value with fixed microsecond precision. The input value is scaled up or down appropriately depending on its precision. + +:::note +The output value is a timestamp in UTC, not in the timezone of `DateTime64`. +::: + +**Syntax** + +```sql +toUnixTimestamp64Micro(value) +``` + +**Arguments** + +- `value` — DateTime64 value with any precision. [DateTime64](../data-types/datetime64.md). + +**Returned value** + +- `value` converted to the `Int64` data type. [Int64](../data-types/int-uint.md). + +**Example** + Query: -``` sql -WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +```sql +WITH toDateTime64('1970-01-15 06:56:07.891011', 6, 'UTC') AS dt64 +SELECT toUnixTimestamp64Micro(dt64); +``` + +Result: + +```response +┌─toUnixTimestamp64Micro(dt64)─┐ +│ 1234567891011 │ +└──────────────────────────────┘ +``` + +## toUnixTimestamp64Nano + +Converts a `DateTime64` to a `Int64` value with fixed nanosecond precision. The input value is scaled up or down appropriately depending on its precision. + +:::note +The output value is a timestamp in UTC, not in the timezone of `DateTime64`. +::: + +**Syntax** + +```sql +toUnixTimestamp64Nano(value) +``` + +**Arguments** + +- `value` — DateTime64 value with any precision. [DateTime64](../data-types/datetime64.md). + +**Returned value** + +- `value` converted to the `Int64` data type. [Int64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +WITH toDateTime64('1970-01-01 00:20:34.567891011', 9, 'UTC') AS dt64 SELECT toUnixTimestamp64Nano(dt64); ``` @@ -2477,34 +2534,32 @@ Result: ```response ┌─toUnixTimestamp64Nano(dt64)─┐ -│ 1568650812345678000 │ +│ 1234567891011 │ └─────────────────────────────┘ ``` ## fromUnixTimestamp64Milli -## fromUnixTimestamp64Micro +Converts an `Int64` to a `DateTime64` value with fixed millisecond precision and optional timezone. The input value is scaled up or down appropriately depending on its precision. -## fromUnixTimestamp64Nano - -Converts an `Int64` to a `DateTime64` value with fixed sub-second precision and optional timezone. Input value is scaled up or down appropriately depending on it’s precision. Please note that input value is treated as UTC timestamp, not timestamp at given (or implicit) timezone. +:::note +Please note that input value is treated as a UTC timestamp, not timestamp at the given (or implicit) timezone. +::: **Syntax** ``` sql fromUnixTimestamp64Milli(value[, timezone]) -fromUnixTimestamp64Micro(value[, timezone]) -fromUnixTimestamp64Nano(value[, timezone]) ``` **Arguments** -- `value` — `Int64` value with any precision. -- `timezone` — `String` (optional) timezone name of the result. +- `value` — value with any precision. [Int64](../data-types/int-uint.md). +- `timezone` — (optional) timezone name of the result. [String](../data-types/string.md). **Returned value** -- `value` converted to the `DateTime64` data type. +- `value` converted to DateTime64 with precision `3`. [DateTime64](../data-types/datetime64.md). **Example** @@ -2512,15 +2567,101 @@ Query: ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC'); +SELECT + fromUnixTimestamp64Milli(i64, 'UTC') AS x, + toTypeName(x); ``` Result: ```response -┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ -│ 2009-02-13 23:31:31.011 │ -└──────────────────────────────────────┘ +┌───────────────────────x─┬─toTypeName(x)────────┐ +│ 2009-02-13 23:31:31.011 │ DateTime64(3, 'UTC') │ +└─────────────────────────┴──────────────────────┘ +``` + +## fromUnixTimestamp64Micro + +Converts an `Int64` to a `DateTime64` value with fixed microsecond precision and optional timezone. The input value is scaled up or down appropriately depending on its precision. + +:::note +Please note that input value is treated as a UTC timestamp, not timestamp at the given (or implicit) timezone. +::: + +**Syntax** + +``` sql +fromUnixTimestamp64Micro(value[, timezone]) +``` + +**Arguments** + +- `value` — value with any precision. [Int64](../data-types/int-uint.md). +- `timezone` — (optional) timezone name of the result. [String](../data-types/string.md). + +**Returned value** + +- `value` converted to DateTime64 with precision `6`. [DateTime64](../data-types/datetime64.md). + +**Example** + +Query: + +``` sql +WITH CAST(1234567891011, 'Int64') AS i64 +SELECT + fromUnixTimestamp64Micro(i64, 'UTC') AS x, + toTypeName(x); +``` + +Result: + +```response +┌──────────────────────────x─┬─toTypeName(x)────────┐ +│ 1970-01-15 06:56:07.891011 │ DateTime64(6, 'UTC') │ +└────────────────────────────┴──────────────────────┘ +``` + +## fromUnixTimestamp64Nano + +Converts an `Int64` to a `DateTime64` value with fixed nanosecond precision and optional timezone. The input value is scaled up or down appropriately depending on its precision. + +:::note +Please note that input value is treated as a UTC timestamp, not timestamp at the given (or implicit) timezone. +::: + +**Syntax** + +``` sql +fromUnixTimestamp64Nano(value[, timezone]) +``` + +**Arguments** + +- `value` — value with any precision. [Int64](../data-types/int-uint.md). +- `timezone` — (optional) timezone name of the result. [String](../data-types/string.md). + +**Returned value** + +- `value` converted to DateTime64 with precision `9`. [DateTime64](../data-types/datetime64.md). + +**Example** + +Query: + +``` sql +WITH CAST(1234567891011, 'Int64') AS i64 +SELECT + fromUnixTimestamp64Nano(i64, 'UTC') AS x, + toTypeName(x); +``` + +Result: + +```response +┌─────────────────────────────x─┬─toTypeName(x)────────┐ +│ 1970-01-01 00:20:34.567891011 │ DateTime64(9, 'UTC') │ +└───────────────────────────────┴──────────────────────┘ ``` ## formatRow diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 0323ae728a9..b7d095c796e 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -150,7 +150,7 @@ The function also works for [Arrays](array-functions.md#function-empty) and [Str **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#generateuuidv4) function. Query: @@ -190,7 +190,7 @@ The function also works for [Arrays](array-functions.md#function-notempty) or [S **Example** -To generate the UUID value, ClickHouse provides the [generateUUIDv4](#uuid-function-generate) function. +To generate the UUID value, ClickHouse provides the [generateUUIDv4](#generateuuidv4) function. Query: @@ -543,12 +543,17 @@ serverUUID() Generates a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID). -The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. +The generated Snowflake ID contains the current Unix timestamp in milliseconds (41 + 1 top zero bits), followed by a machine id (10 bits), and a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function `generateSnowflakeID` guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries. +:::note +The generated Snowflake IDs are based on the UNIX epoch 1970-01-01. +While no standard or recommendation exists for the epoch of Snowflake IDs, implementations in other systems may use a different epoch, e.g. Twitter/X (2010-11-04) or Mastodon (2015-01-01). +::: + ``` 0 1 2 3 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 @@ -605,6 +610,11 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2); ## snowflakeToDateTime +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](../data-types/datetime.md) format. **Syntax** @@ -641,6 +651,11 @@ Result: ## snowflakeToDateTime64 +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime64](../data-types/datetime64.md) format. **Syntax** @@ -677,6 +692,11 @@ Result: ## dateTimeToSnowflake +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Converts a [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. **Syntax** @@ -711,6 +731,11 @@ Result: ## dateTime64ToSnowflake +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Convert a [DateTime64](../data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. **Syntax** @@ -743,6 +768,148 @@ Result: └─────────────────────────────┘ ``` +## snowflakeIDToDateTime + +Returns the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as a value of type [DateTime](../data-types/datetime.md). + +**Syntax** + +``` sql +snowflakeIDToDateTime(value[, epoch[, time_zone]]) +``` + +**Arguments** + +- `value` — Snowflake ID. [UInt64](../data-types/int-uint.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). +- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../data-types/string.md). + +**Returned value** + +- The timestamp component of `value` as a [DateTime](../data-types/datetime.md) value. + +**Example** + +Query: + +```sql +SELECT snowflakeIDToDateTime(7204436857747984384) AS res +``` + +Result: + +``` +┌─────────────────res─┐ +│ 2024-06-06 10:59:58 │ +└─────────────────────┘ +``` + +## snowflakeIDToDateTime64 + +Returns the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as a value of type [DateTime64](../data-types/datetime64.md). + +**Syntax** + +``` sql +snowflakeIDToDateTime64(value[, epoch[, time_zone]]) +``` + +**Arguments** + +- `value` — Snowflake ID. [UInt64](../data-types/int-uint.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). +- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../data-types/string.md). + +**Returned value** + +- The timestamp component of `value` as a [DateTime64](../data-types/datetime64.md) with scale = 3, i.e. millisecond precision. + +**Example** + +Query: + +```sql +SELECT snowflakeIDToDateTime64(7204436857747984384) AS res +``` + +Result: + +``` +┌─────────────────res─┐ +│ 2024-06-06 10:59:58 │ +└─────────────────────┘ +``` + +## dateTimeToSnowflakeID + +Converts a [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. + +**Syntax** + +``` sql +dateTimeToSnowflakeID(value[, epoch]) +``` + +**Arguments** + +- `value` — Date with time. [DateTime](../data-types/datetime.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). + +**Returned value** + +- Input value converted to [UInt64](../data-types/int-uint.md) as the first Snowflake ID at that time. + +**Example** + +Query: + +```sql +SELECT toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt, dateTimeToSnowflakeID(dt) AS res; +``` + +Result: + +``` +┌──────────────────dt─┬─────────────────res─┐ +│ 2021-08-15 18:57:56 │ 6832626392367104000 │ +└─────────────────────┴─────────────────────┘ +``` + +## dateTime64ToSnowflakeID + +Convert a [DateTime64](../data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. + +**Syntax** + +``` sql +dateTime64ToSnowflakeID(value[, epoch]) +``` + +**Arguments** + +- `value` — Date with time. [DateTime64](../data-types/datetime64.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). + +**Returned value** + +- Input value converted to [UInt64](../data-types/int-uint.md) as the first Snowflake ID at that time. + +**Example** + +Query: + +```sql +SELECT toDateTime('2021-08-15 18:57:56.493', 3, 'Asia/Shanghai') AS dt, dateTime64ToSnowflakeID(dt) AS res; +``` + +Result: + +``` +┌──────────────────────dt─┬─────────────────res─┐ +│ 2021-08-15 18:57:56.493 │ 6832626394434895872 │ +└─────────────────────────┴─────────────────────┘ +``` + ## See also - [dictGetUUID](../functions/ext-dict-functions.md#ext_dict_functions-other) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 0257d21b30f..ed75b1802d8 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -235,7 +235,7 @@ If `some_predicate` is not selective enough, it will return a large amount of da ### Distributed Subqueries and max_parallel_replicas -When [max_parallel_replicas](#settings-max_parallel_replicas) is greater than 1, distributed queries are further transformed. +When [max_parallel_replicas](#distributed-subqueries-and-max_parallel_replicas) is greater than 1, distributed queries are further transformed. For example, the following: @@ -255,7 +255,7 @@ where `M` is between `1` and `3` depending on which replica the local query is e These settings affect every MergeTree-family table in the query and have the same effect as applying `SAMPLE 1/3 OFFSET (M-1)/3` on each table. -Therefore adding the [max_parallel_replicas](#settings-max_parallel_replicas) setting will only produce correct results if both tables have the same replication scheme and are sampled by UserID or a subkey of it. In particular, if `local_table_2` does not have a sampling key, incorrect results will be produced. The same rule applies to `JOIN`. +Therefore adding the [max_parallel_replicas](#distributed-subqueries-and-max_parallel_replicas) setting will only produce correct results if both tables have the same replication scheme and are sampled by UserID or a subkey of it. In particular, if `local_table_2` does not have a sampling key, incorrect results will be produced. The same rule applies to `JOIN`. One workaround if `local_table_2` does not meet the requirements, is to use `GLOBAL IN` or `GLOBAL JOIN`. diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index a23710b12bd..aa6f132e08e 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -108,7 +108,7 @@ ALTER TABLE visits RENAME COLUMN webBrowser TO browser CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name ``` -Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md/#how-to-set-partition-expression). +Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](../alter/partition.md/#how-to-set-partition-expression). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. @@ -173,7 +173,7 @@ ALTER TABLE visits MODIFY COLUMN browser Array(String) Changing the column type is the only complex action – it changes the contents of files with data. For large tables, this may take a long time. -The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#alter_add-column) description, but column type is mandatory in this case. +The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#add-column) description, but column type is mandatory in this case. Example: diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index 3cfb99cff83..f81d4f02e0c 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -16,7 +16,7 @@ Most `ALTER TABLE` queries modify table settings or data: - [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md) - [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md) - [TTL](/docs/en/sql-reference/statements/alter/ttl.md) -- [STATISTIC](/docs/en/sql-reference/statements/alter/statistic.md) +- [STATISTICS](/docs/en/sql-reference/statements/alter/statistics.md) - [APPLY DELETED MASK](/docs/en/sql-reference/statements/alter/apply-deleted-mask.md) :::note diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index ce5cecf6fd6..778816f8934 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -31,7 +31,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget ALTER TABLE table_name [ON CLUSTER cluster] DETACH PARTITION|PART partition_expr ``` -Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#alter_attach-partition) query. +Moves all data for the specified partition to the `detached` directory. The server forgets about the detached data partition as if it does not exist. The server will not know about this data until you make the [ATTACH](#attach-partitionpart) query. Example: @@ -139,7 +139,7 @@ For the query to run successfully, the following conditions must be met: ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM table1 ``` -This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`. +This query copies the data partition from `table1` to `table2` and replaces the existing partition in `table2`. The operation is atomic. Note that: @@ -252,7 +252,7 @@ Downloads a partition from another server. This query only works for the replica The query does the following: 1. Downloads the partition|part from the specified shard. In ‘path-in-zookeeper’ you must specify a path to the shard in ZooKeeper. -2. Then the query puts the downloaded data to the `detached` directory of the `table_name` table. Use the [ATTACH PARTITION\|PART](#alter_attach-partition) query to add the data to the table. +2. Then the query puts the downloaded data to the `detached` directory of the `table_name` table. Use the [ATTACH PARTITION\|PART](#attach-partitionpart) query to add the data to the table. For example: @@ -353,7 +353,7 @@ You can specify the partition expression in `ALTER ... PARTITION` queries in dif - Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH. For example, `ALTER TABLE visits ATTACH PARTITION ALL`. - As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- In the [ALTER ATTACH PART](#attach-partitionpart) and [DROP DETACHED PART](#drop-detached-partitionpart) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed. diff --git a/docs/en/sql-reference/statements/alter/statistic.md b/docs/en/sql-reference/statements/alter/statistic.md deleted file mode 100644 index 1c2e45b23fd..00000000000 --- a/docs/en/sql-reference/statements/alter/statistic.md +++ /dev/null @@ -1,25 +0,0 @@ ---- -slug: /en/sql-reference/statements/alter/statistic -sidebar_position: 45 -sidebar_label: STATISTIC ---- - -# Manipulating Column Statistics - -The following operations are available: - -- `ALTER TABLE [db].table ADD STATISTIC (columns list) TYPE type` - Adds statistic description to tables metadata. - -- `ALTER TABLE [db].table DROP STATISTIC (columns list) TYPE type` - Removes statistic description from tables metadata and deletes statistic files from disk. - -- `ALTER TABLE [db].table CLEAR STATISTIC (columns list) TYPE type` - Deletes statistic files from disk. - -- `ALTER TABLE [db.]table MATERIALIZE STATISTIC (columns list) TYPE type` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). - -The first two commands are lightweight in a sense that they only change metadata or remove files. - -Also, they are replicated, syncing statistics metadata via ZooKeeper. - -:::note -Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). -::: diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md new file mode 100644 index 00000000000..80024781f88 --- /dev/null +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -0,0 +1,33 @@ +--- +slug: /en/sql-reference/statements/alter/statistics +sidebar_position: 45 +sidebar_label: STATISTICS +--- + +# Manipulating Column Statistics + +The following operations are available: + +- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. + +- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. + +- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. + +- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. + +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). + +The first two commands are lightweight in a sense that they only change metadata or remove files. + +Also, they are replicated, syncing statistics metadata via ZooKeeper. + +There is an example adding two statistics types to two columns: + +``` +ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; +``` + +:::note +Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +::: diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 16918102f02..0253bc647e6 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -17,8 +17,8 @@ By default, tables are created only on the current server. Distributed DDL queri ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr1] [compression_codec] [TTL expr1] [COMMENT 'comment for column'], - name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [compression_codec] [TTL expr2] [COMMENT 'comment for column'], + name1 [type1] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr1] [COMMENT 'comment for column'] [compression_codec] [TTL expr1], + name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [COMMENT 'comment for column'] [compression_codec] [TTL expr2], ... ) ENGINE = engine COMMENT 'comment for table' @@ -337,7 +337,7 @@ Then, when executing the query `SELECT name FROM users_a WHERE length(name) < 5; Defines storage time for values. Can be specified only for MergeTree-family tables. For the detailed description, see [TTL for columns and tables](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). -## Column Compression Codecs +## Column Compression Codecs {#column_compression_codec} By default, ClickHouse applies `lz4` compression in the self-managed version, and `zstd` in ClickHouse Cloud. diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 1bdf22b35b0..1fabb6d8cc7 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -6,7 +6,7 @@ sidebar_label: VIEW # CREATE VIEW -Creates a new view. Views can be [normal](#normal-view), [materialized](#materialized-view), [live](#live-view-experimental), and [window](#window-view-experimental) (live view and window view are experimental features). +Creates a new view. Views can be [normal](#normal-view), [materialized](#materialized-view), [live](#live-view-deprecated), and [window](#window-view-experimental) (live view and window view are experimental features). ## Normal View diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 2850ce71781..43fa344a16d 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -33,7 +33,7 @@ GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_US - `role` — ClickHouse user role. - `user` — ClickHouse user account. -The `WITH ADMIN OPTION` clause grants [ADMIN OPTION](#admin-option-privilege) privilege to `user` or `role`. +The `WITH ADMIN OPTION` clause grants [ADMIN OPTION](#admin-option) privilege to `user` or `role`. The `WITH REPLACE OPTION` clause replace old roles by new role for the `user` or `role`, if is not specified it appends roles. ## Grant Current Grants Syntax @@ -201,7 +201,7 @@ Hierarchy of privileges: - `HDFS` - `S3` - [dictGet](#dictget) -- [displaySecretsInShowAndSelect](#display-secrets) +- [displaySecretsInShowAndSelect](#displaysecretsinshowandselect) - [NAMED COLLECTION ADMIN](#named-collection-admin) - `CREATE NAMED COLLECTION` - `DROP NAMED COLLECTION` @@ -498,7 +498,7 @@ Privilege level: `DICTIONARY`. - `GRANT dictGet ON mydictionary TO john` -### displaySecretsInShowAndSelect {#display-secrets} +### displaySecretsInShowAndSelect Allows a user to view secrets in `SHOW` and `SELECT` queries if both [`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select) diff --git a/docs/en/sql-reference/statements/select/sample.md b/docs/en/sql-reference/statements/select/sample.md index 137f86cc8b9..78e05b19bd1 100644 --- a/docs/en/sql-reference/statements/select/sample.md +++ b/docs/en/sql-reference/statements/select/sample.md @@ -27,14 +27,14 @@ The features of data sampling are listed below: For the `SAMPLE` clause the following syntax is supported: -| SAMPLE Clause Syntax | Description | -|----------------------|------------------------------| -| `SAMPLE k` | Here `k` is the number from 0 to 1. The query is executed on `k` fraction of data. For example, `SAMPLE 0.1` runs the query on 10% of data. [Read more](#select-sample-k) | -| `SAMPLE n` | Here `n` is a sufficiently large integer. The query is executed on a sample of at least `n` rows (but not significantly more than this). For example, `SAMPLE 10000000` runs the query on a minimum of 10,000,000 rows. [Read more](#select-sample-n) | -| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1. The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#select-sample-offset) | +| SAMPLE Clause Syntax | Description | +|----------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `SAMPLE k` | Here `k` is the number from 0 to 1. The query is executed on `k` fraction of data. For example, `SAMPLE 0.1` runs the query on 10% of data. [Read more](#sample-k) | +| `SAMPLE n` | Here `n` is a sufficiently large integer. The query is executed on a sample of at least `n` rows (but not significantly more than this). For example, `SAMPLE 10000000` runs the query on a minimum of 10,000,000 rows. [Read more](#sample-n) | +| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1. The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#sample-k-offset-m) | -## SAMPLE K {#select-sample-k} +## SAMPLE K Here `k` is the number from 0 to 1 (both fractional and decimal notations are supported). For example, `SAMPLE 1/2` or `SAMPLE 0.5`. @@ -54,7 +54,7 @@ ORDER BY PageViews DESC LIMIT 1000 In this example, the query is executed on a sample from 0.1 (10%) of data. Values of aggregate functions are not corrected automatically, so to get an approximate result, the value `count()` is manually multiplied by 10. -## SAMPLE N {#select-sample-n} +## SAMPLE N Here `n` is a sufficiently large integer. For example, `SAMPLE 10000000`. @@ -90,7 +90,7 @@ FROM visits SAMPLE 10000000 ``` -## SAMPLE K OFFSET M {#select-sample-offset} +## SAMPLE K OFFSET M Here `k` and `m` are numbers from 0 to 1. Examples are shown below. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 7efbff1b42b..e6d3439d2b9 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -174,7 +174,7 @@ Aborts ClickHouse process (like `kill -9 {$ pid_clickhouse-server}`) ## Managing Distributed Tables -ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) queries. You can also synchronously insert distributed data with the [distributed_foreground_insert](../../operations/settings/settings.md#distributed_foreground_insert) setting. +ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#stop-distributed-sends), [FLUSH DISTRIBUTED](#flush-distributed), and [START DISTRIBUTED SENDS](#start-distributed-sends) queries. You can also synchronously insert distributed data with the [distributed_foreground_insert](../../operations/settings/settings.md#distributed_foreground_insert) setting. ### STOP DISTRIBUTED SENDS diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index fc0286e76ad..6a4afb63db8 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -54,11 +54,11 @@ Identifiers are: - Cluster, database, table, partition, and column names. - Functions. - Data types. -- [Expression aliases](#expression_aliases). +- [Expression aliases](#expression-aliases). Identifiers can be quoted or non-quoted. The latter is preferred. -Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#syntax-keywords). Examples: `x`, `_1`, `X_y__Z123_`. +Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#keywords). Examples: `x`, `_1`, `X_y__Z123_`. If you want to use identifiers the same as keywords or you want to use other symbols in identifiers, quote it using double quotes or backticks, for example, `"id"`, `` `id` ``. diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 1510489ce83..f59fedeb3a2 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -72,6 +72,7 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. **See Also** diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index f66178afbb2..3a3162dad9a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -18,7 +18,7 @@ file([path_to_archive ::] path [,format] [,structure] [,compression]) **Parameters** -- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports in read-only mode the following [globs](#globs_in_path): `*`, `?`, `{abc,def}` (with `'abc'` and `'def'` being strings) and `{N..M}` (with `N` and `M` being numbers). +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports in read-only mode the following [globs](#globs-in-path): `*`, `?`, `{abc,def}` (with `'abc'` and `'def'` being strings) and `{N..M}` (with `N` and `M` being numbers). - `path_to_archive` - The relative path to a zip/tar/7z archive. Supports the same globs as `path`. - `format` — The [format](/docs/en/interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. @@ -128,7 +128,7 @@ Reading data from `table.csv`, located in `archive1.zip` or/and `archive2.zip`: SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); ``` -## Globs in path {#globs_in_path} +## Globs in path Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. @@ -196,6 +196,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/fileCluster.md b/docs/en/sql-reference/table-functions/fileCluster.md index 4677d2883a7..62b00fadd62 100644 --- a/docs/en/sql-reference/table-functions/fileCluster.md +++ b/docs/en/sql-reference/table-functions/fileCluster.md @@ -22,7 +22,7 @@ fileCluster(cluster_name, path[, format, structure, compression_method]) **Arguments** - `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. -- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file also supports [globs](#globs_in_path). +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file also supports [globs](#globs-in-path). - `format` — [Format](../../interfaces/formats.md#formats) of the files. Type: [String](../../sql-reference/data-types/string.md). - `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). - `compression_method` — Compression method. Supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. @@ -74,7 +74,7 @@ SELECT * FROM fileCluster('my_cluster', 'file{1,2}.csv', 'CSV', 'i UInt32, s Str ``` -## Globs in Path {#globs_in_path} +## Globs in Path All patterns supported by [File](../../sql-reference/table-functions/file.md#globs-in-path) table function are supported by FileCluster. diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index d65615e7588..28cba5ccc6a 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -97,6 +97,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index cbef80371a3..1a7e2b8d66a 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -272,6 +272,7 @@ FROM s3( - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 4dc6e435b50..3bb7aff53a7 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -53,6 +53,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 5f11f1b430b..d9da51892f9 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -414,6 +414,8 @@ $ curl -v 'http://localhost:8123/predefined_query' - `content_type` — используется со всеми типами, возвращает [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type). + - `http_response_headers` — используется со всеми типами чтобы добавить кастомные хедеры в ответ. Может использоваться в том числе для задания хедера `Content-Type` вместо `content_type`. + - `response_content` — используется с типом`static`, содержимое ответа, отправленное клиенту, при использовании префикса ‘file://’ or ‘config://’, находит содержимое из файла или конфигурации, отправленного клиенту. Далее приведены методы настройки для различных типов. @@ -509,6 +511,33 @@ max_final_threads 2 static 402 text/html; charset=UTF-8 + + en + 43 + + Say Hi! + + + +
+``` + +`http_response_headers` так же может использоваться для определения `Content-Type` вместо `content_type`. + +``` xml + + + GET + xxx + /hi + + static + 402 + + text/html; charset=UTF-8 + en + 43 + Say Hi! @@ -589,6 +618,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file:///absolute_path_file.html @@ -599,6 +631,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' static text/html; charset=UTF-8 + + 737060cd8c284d8af7ad3082f209582d + file://./relative_path_file.html diff --git a/docs/ru/sql-reference/functions/math-functions.md b/docs/ru/sql-reference/functions/math-functions.md index 367451a5b32..caacbb216bf 100644 --- a/docs/ru/sql-reference/functions/math-functions.md +++ b/docs/ru/sql-reference/functions/math-functions.md @@ -304,8 +304,8 @@ atan2(y, x) **Аргументы** -- `y` — координата y точки, в которую проведена линия. [Float64](../../sql-reference/data-types/float.md#float32-float64). -- `x` — координата х точки, в которую проведена линия. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `y` — координата y точки, в которую проведена линия. [Float64](../../sql-reference/data-types/float.md#float32-float64) или [Decimal](../../sql-reference/data-types/decimal.md). +- `x` — координата х точки, в которую проведена линия. [Float64](../../sql-reference/data-types/float.md#float32-float64) или [Decimal](../../sql-reference/data-types/decimal.md). **Возвращаемое значение** @@ -341,8 +341,8 @@ hypot(x, y) **Аргументы** -- `x` — первый катет прямоугольного треугольника. [Float64](../../sql-reference/data-types/float.md#float32-float64). -- `y` — второй катет прямоугольного треугольника. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — первый катет прямоугольного треугольника. [Float64](../../sql-reference/data-types/float.md#float32-float64) или [Decimal](../../sql-reference/data-types/decimal.md). +- `y` — второй катет прямоугольного треугольника. [Float64](../../sql-reference/data-types/float.md#float32-float64) или [Decimal](../../sql-reference/data-types/decimal.md). **Возвращаемое значение** diff --git a/docs/ru/sql-reference/functions/rounding-functions.md b/docs/ru/sql-reference/functions/rounding-functions.md index cc939f69afc..f1dd57505ea 100644 --- a/docs/ru/sql-reference/functions/rounding-functions.md +++ b/docs/ru/sql-reference/functions/rounding-functions.md @@ -10,7 +10,7 @@ sidebar_label: "Функции округления" Возвращает наибольшее круглое число, которое меньше или равно, чем x. Круглым называется число, кратное 1 / 10N или ближайшее к нему число соответствующего типа данных, если 1 / 10N не представимо точно. -N - целочисленная константа, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа. +N - целочисленный аргумент, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа. N может быть отрицательным. Примеры: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index fc258f7b4cf..fa76e84f130 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -538,16 +538,28 @@ SELECT base58Decode('3dc8KtHrwM'); Синоним: `TO_BASE64`. +## base64UrlEncode(s) + +Производит кодирование URL (String или FixedString) в base64-представление в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). + ## base64Decode(s) {#base64decode} Декодирует base64-представление s в исходную строку. При невозможности декодирования выбрасывает исключение Синоним: `FROM_BASE64`. +## base64UrlDecode(s) + +Декодирует base64-представление URL в исходную строку в соответствии с [RFC 4648](https://tools.ietf.org/html/rfc4648). При невозможности декодирования выбрасывает исключение + ## tryBase64Decode(s) {#trybase64decode} Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. +## tryBase64UrlDecode(s) + +Функционал аналогичен base64UrlDecode, но при невозможности декодирования возвращает пустую строку. + ## endsWith(s, suffix) {#endswith} Возвращает 1, если строка завершается указанным суффиксом, и 0 в противном случае. diff --git a/docs/zh/guides/improving-query-performance/skipping-indexes.md b/docs/zh/guides/improving-query-performance/skipping-indexes.md index f9f43e46927..8eb88d859f2 100644 --- a/docs/zh/guides/improving-query-performance/skipping-indexes.md +++ b/docs/zh/guides/improving-query-performance/skipping-indexes.md @@ -123,7 +123,7 @@ Bloom filter是一种数据结构,它允许对集合成员进行高效的是 有三种基于Bloom过滤器的数据跳数索引类型: -* 基本的**bloom_filter**接受一个可选参数,该参数表示在0到1之间允许的“假阳性”率(如果未指定,则使用.025)。 +* 基本的**bloom_filter**接受一个可选参数,该参数表示在0到1之间允许的“假阳性”率(如果未指定,则使用0.025)。 * 更专业的**tokenbf_v1**。需要三个参数,用来优化布隆过滤器:(1)过滤器的大小字节(大过滤器有更少的假阳性,有更高的存储成本),(2)哈希函数的个数(更多的散列函数可以减少假阳性)。(3)布隆过滤器哈希函数的种子。有关这些参数如何影响布隆过滤器功能的更多细节,请参阅 [这里](https://hur.st/bloomfilter/) 。此索引仅适用于String、FixedString和Map类型的数据。输入表达式被分割为由非字母数字字符分隔的字符序列。例如,列值`This is a candidate for a "full text" search`将被分割为`This` `is` `a` `candidate` `for` `full` `text` `search`。它用于LIKE、EQUALS、in、hasToken()和类似的长字符串中单词和其他值的搜索。例如,一种可能的用途是在非结构的应用程序日志行列中搜索少量的类名或行号。 diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 7518227a070..639e0957f49 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -57,7 +57,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::KeeperSnapshotManager manager(1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); - std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; + std::cout << "Snapshot serialized to path:" << fs::path(file_info->disk->getPath()) / file_info->path << std::endl; } catch (...) { diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 52aa601b1a2..079951be55e 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -154,8 +154,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Storages/StorageS3Settings.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/BaseDaemon.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/SentryWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index dba5c2b7d2a..0d3c1f10894 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -361,9 +361,10 @@ try } GlobalThreadPool::initialize( - config().getUInt("max_thread_pool_size", 100), - config().getUInt("max_thread_pool_free_size", 1000), - config().getUInt("thread_pool_queue_size", 10000) + /// We need to have sufficient amount of threads for connections + nuraft workers + keeper workers, 1000 is an estimation + std::min(1000U, config().getUInt("max_thread_pool_size", 1000)), + config().getUInt("max_thread_pool_free_size", 100), + config().getUInt("thread_pool_queue_size", 1000) ); /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..7bc2be806f7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -48,6 +49,7 @@ #include #include #include +#include #include #include #include @@ -70,7 +72,6 @@ #include #include #include -#include #include #include #include @@ -721,11 +722,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - Poco::ThreadPool server_pool(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -773,7 +769,27 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif - bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + bool has_trace_collector = false; + /// Disable it if we collect test coverage information, because it will work extremely slow. +#if !WITH_COVERAGE + /// Profilers cannot work reliably with any other libunwind or without PHDR cache. + has_trace_collector = hasPHDRCache() && config().has("trace_log"); +#endif + + /// Describe multiple reasons when query profiler cannot work. + +#if WITH_COVERAGE + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); +#endif + +#if defined(SANITIZER) + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + " when two different stack unwinding methods will interfere with each other."); +#endif + + if (!hasPHDRCache()) + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" + " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -782,8 +798,39 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size, - will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, - will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + + if (has_trace_collector) + { + global_context->createTraceCollector(); + + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + + Poco::ThreadPool server_pool( + /* minCapacity */3, + /* maxCapacity */server_settings.max_connections, + /* idleTime */60, + /* stackSize */POCO_THREAD_STACK_SIZE, + server_settings.global_profiler_real_time_period_ns, + server_settings.global_profiler_cpu_time_period_ns); + + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -944,6 +991,18 @@ try } } + std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); + fs::path path = path_str; + + /// Check that the process user id matches the owner of the data. + assertProcessUserMatchesDataOwner(path_str, [&](const std::string & message){ global_context->addWarningMessage(message); }); + + global_context->setPath(path_str); + + StatusFile status{path / "status", StatusFile::write_full_info}; + + ServerUUID::load(path / "uuid", log); + zkutil::validateZooKeeperConfig(config()); bool has_zookeeper = zkutil::hasZooKeeperConfig(config()); @@ -955,7 +1014,7 @@ try ConfigProcessor config_processor(config_path); loaded_config = config_processor.loadConfigWithZooKeeperIncludes( main_config_zk_node_cache, main_config_zk_changed_event, /* fallback_to_preprocessed = */ true); - config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH)); + config_processor.savePreprocessedConfig(loaded_config, path_str); config().removeConfiguration(old_configuration.get()); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); global_context->setConfig(loaded_config.configuration); @@ -1089,19 +1148,6 @@ try global_context->setRemoteHostFilter(config()); global_context->setHTTPHeaderFilter(config()); - std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); - fs::path path = path_str; - std::string default_database = server_settings.default_database.toString(); - - /// Check that the process user id matches the owner of the data. - assertProcessUserMatchesDataOwner(path_str, [&](const std::string & message){ global_context->addWarningMessage(message); }); - - global_context->setPath(path_str); - - StatusFile status{path / "status", StatusFile::write_full_info}; - - ServerUUID::load(path / "uuid", log); - /// Try to increase limit on number of open files. { rlimit rlim; @@ -1339,7 +1385,7 @@ try CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif - NamedCollectionUtils::loadIfNot(); + NamedCollectionFactory::instance().loadIfNot(); /// Initialize main config reloader. std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); @@ -1571,6 +1617,10 @@ try 0, // We don't need any threads one all the parts will be deleted new_server_settings.max_parts_cleaning_thread_pool_size); + + global_context->setMergeWorkload(new_server_settings.merge_workload); + global_context->setMutationWorkload(new_server_settings.mutation_workload); + if (config->has("resources")) { global_context->getResourceManager()->updateConfiguration(*config); @@ -1608,7 +1658,7 @@ try #if USE_SSL CertificateReloader::instance().tryLoad(*config); #endif - NamedCollectionUtils::reloadFromConfig(*config); + NamedCollectionFactory::instance().reloadFromConfig(*config); FileCacheFactory::instance().updateSettingsFromConfig(*config); @@ -1632,6 +1682,10 @@ try if (global_context->isServerCompletelyStarted()) CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability); +#if USE_GWP_ASAN + GWPAsan::setForceSampleProbability(new_server_settings.gwp_asan_force_sample_probability); +#endif + ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. @@ -1889,6 +1943,7 @@ try /// Set current database name before loading tables and databases because /// system logs may copy global context. + std::string default_database = server_settings.default_database.toString(); global_context->setCurrentDatabaseNameInGlobalContext(default_database); LOG_INFO(log, "Loading metadata from {}", path_str); @@ -1950,52 +2005,9 @@ try LOG_DEBUG(log, "Loaded metadata."); - /// Init trace collector only after trace_log system table was created - /// Disable it if we collect test coverage information, because it will work extremely slow. -#if !WITH_COVERAGE - /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache()) - { + if (has_trace_collector) global_context->initializeTraceCollector(); - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } -#endif - - /// Describe multiple reasons when query profiler cannot work. - -#if WITH_COVERAGE - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); -#endif - -#if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" - " when two different stack unwinding methods will interfere with each other."); -#endif - - if (!hasPHDRCache()) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" - " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - #if defined(OS_LINUX) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider(); if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None) @@ -2124,6 +2136,10 @@ try CannotAllocateThreadFaultInjector::setFaultProbability(server_settings.cannot_allocate_thread_fault_injection_probability); +#if USE_GWP_ASAN + GWPAsan::setForceSampleProbability(server_settings.gwp_asan_force_sample_probability); +#endif + try { global_context->startClusterDiscovery(); diff --git a/programs/server/config.xml b/programs/server/config.xml index 4b3248d9d1c..a0cb5b14007 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -371,7 +371,7 @@ - + true @@ -1396,6 +1396,14 @@ + + + 3 equal ranges +/// pk1 pk2 c1 c2 +/// ---------------------- +/// 1 1 a b +/// 1 1 b e +/// -------- +/// 1 2 e a +/// 1 2 d c +/// 1 2 e a +/// -------- +/// 2 1 a 3 +/// ---------------------- +EqualRanges getEqualRanges(const Block & block, const SortDescription & sort_description, const IColumn::Permutation & permutation) +{ + EqualRanges ranges; + const size_t rows = block.rows(); + if (sort_description.empty()) + { + ranges.push_back({0, rows}); + } + else + { + for (size_t i = 0; i < rows;) + { + size_t j = i; + while (j < rows && haveEqualSortingKeyValues(block, sort_description, permutation[i], permutation[j])) + ++j; + ranges.push_back({i, j}); + i = j; + } + } + return ranges; +} + +std::vector getCardinalitiesInPermutedRange( + const Block & block, + const std::vector & other_column_indexes, + const IColumn::Permutation & permutation, + const EqualRange & equal_range) +{ + std::vector cardinalities(other_column_indexes.size()); + for (size_t i = 0; i < other_column_indexes.size(); ++i) + { + const size_t column_id = other_column_indexes[i]; + const ColumnPtr & column = block.getByPosition(column_id).column; + cardinalities[i] = column->estimateCardinalityInPermutedRange(permutation, equal_range); + } + return cardinalities; +} + +void updatePermutationInEqualRange( + const Block & block, + const std::vector & other_column_indexes, + IColumn::Permutation & permutation, + const EqualRange & equal_range, + const std::vector & cardinalities, + const LoggerPtr & log) +{ + LOG_TEST(log, "Starting optimization in equal range"); + + std::vector column_order(other_column_indexes.size()); + iota(column_order.begin(), column_order.end(), 0); + auto cmp = [&](size_t lhs, size_t rhs) -> bool { return cardinalities[lhs] < cardinalities[rhs]; }; + stable_sort(column_order.begin(), column_order.end(), cmp); + + std::vector ranges = {equal_range}; + LOG_TEST(log, "equal_range: .from: {}, .to: {}", equal_range.from, equal_range.to); + for (size_t i : column_order) + { + const size_t column_id = other_column_indexes[i]; + const ColumnPtr & column = block.getByPosition(column_id).column; + LOG_TEST(log, "i: {}, column_id: {}, column type: {}, cardinality: {}", i, column_id, column->getName(), cardinalities[i]); + column->updatePermutation( + IColumn::PermutationSortDirection::Ascending, IColumn::PermutationSortStability::Stable, 0, 1, permutation, ranges); + } + + LOG_TEST(log, "Finish optimization in equal range"); +} + +} + +void RowOrderOptimizer::optimize(const Block & block, const SortDescription & sort_description, IColumn::Permutation & permutation) +{ + LoggerPtr log = getLogger("RowOrderOptimizer"); + + LOG_TRACE(log, "Starting optimization"); + + if (block.columns() == 0) + { + LOG_TRACE(log, "Finished optimization (block has no columns)"); + return; /// a table without columns, this should not happen in the first place ... + } + + if (permutation.empty()) + { + const size_t rows = block.rows(); + permutation.resize(rows); + iota(permutation.data(), rows, IColumn::Permutation::value_type(0)); + } + + const EqualRanges equal_ranges = getEqualRanges(block, sort_description, permutation); + const std::vector other_columns_indexes = getOtherColumnIndexes(block, sort_description); + + LOG_TRACE(log, "columns: {}, sorting key columns: {}, rows: {}, equal ranges: {}", block.columns(), sort_description.size(), block.rows(), equal_ranges.size()); + + for (const auto & equal_range : equal_ranges) + { + if (equal_range.size() <= 1) + continue; + const std::vector cardinalities = getCardinalitiesInPermutedRange(block, other_columns_indexes, permutation, equal_range); + updatePermutationInEqualRange(block, other_columns_indexes, permutation, equal_range, cardinalities, log); + } + + LOG_TRACE(log, "Finished optimization"); +} + +} diff --git a/src/Storages/MergeTree/RowOrderOptimizer.h b/src/Storages/MergeTree/RowOrderOptimizer.h new file mode 100644 index 00000000000..f321345c3e4 --- /dev/null +++ b/src/Storages/MergeTree/RowOrderOptimizer.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class RowOrderOptimizer +{ +public: + /// Given the columns in a Block with a sub-set of them as sorting key columns (usually primary key columns --> SortDescription), and a + /// permutation of the rows, this function tries to "improve" the permutation such that the data can be compressed better by generic + /// compression algorithms such as zstd. The heuristics is based on D. Lemire, O. Kaser (2011): Reordering columns for smaller + /// indexes, https://doi.org/10.1016/j.ins.2011.02.002 + /// The algorithm works like this: + /// - Divide the sorting key columns horizontally into "equal ranges". An equal range is defined by the same sorting key values on all + /// of its rows. We can re-shuffle the non-sorting-key values within each equal range freely. + /// - Determine (estimate) for each equal range the cardinality of each non-sorting-key column. + /// - The simple heuristics applied is that non-sorting key columns will be sorted (within each equal range) in order of ascending + /// cardinality. This maximizes the length of equal-value runs within the non-sorting-key columns, leading to better compressability. + static void optimize(const Block & block, const SortDescription & sort_description, IColumn::Permutation & permutation); +}; + +} diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index aaf5c1b5d87..f736c863eee 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -83,15 +83,15 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.index_name = command->index->as().name(); return res; } - else if (command->type == ASTAlterCommand::MATERIALIZE_STATISTIC) + else if (command->type == ASTAlterCommand::MATERIALIZE_STATISTICS) { MutationCommand res; res.ast = command->ptr(); - res.type = MATERIALIZE_STATISTIC; + res.type = MATERIALIZE_STATISTICS; if (command->partition) res.partition = command->partition->clone(); res.predicate = nullptr; - res.statistic_columns = command->statistic_decl->as().getColumnNames(); + res.statistics_columns = command->statistics_decl->as().getColumnNames(); return res; } else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION) @@ -150,16 +150,16 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.clear = true; return res; } - else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_STATISTIC) + else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_STATISTICS) { MutationCommand res; res.ast = command->ptr(); - res.type = MutationCommand::Type::DROP_STATISTIC; + res.type = MutationCommand::Type::DROP_STATISTICS; if (command->partition) res.partition = command->partition->clone(); if (command->clear_index) res.clear = true; - res.statistic_columns = command->statistic_decl->as().getColumnNames(); + res.statistics_columns = command->statistics_decl->as().getColumnNames(); return res; } else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_PROJECTION) diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 6e10f7d9b2d..f999aab1f4d 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -30,12 +30,12 @@ struct MutationCommand UPDATE, MATERIALIZE_INDEX, MATERIALIZE_PROJECTION, - MATERIALIZE_STATISTIC, + MATERIALIZE_STATISTICS, READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query). DROP_COLUMN, DROP_INDEX, DROP_PROJECTION, - DROP_STATISTIC, + DROP_STATISTICS, MATERIALIZE_TTL, RENAME_COLUMN, MATERIALIZE_COLUMN, @@ -51,10 +51,11 @@ struct MutationCommand /// Columns with corresponding actions std::unordered_map column_to_update_expression = {}; - /// For MATERIALIZE INDEX and PROJECTION and STATISTIC + /// For MATERIALIZE INDEX and PROJECTION and STATISTICS String index_name = {}; String projection_name = {}; - std::vector statistic_columns = {}; + std::vector statistics_columns = {}; + std::vector statistics_types = {}; /// For MATERIALIZE INDEX, UPDATE and DELETE. ASTPtr partition = {}; diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index 47b69d79ad8..ba90f21c907 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -95,7 +95,7 @@ MutableNamedCollectionPtr tryGetNamedCollectionWithOverrides( if (asts.empty()) return nullptr; - NamedCollectionUtils::loadIfNot(); + NamedCollectionFactory::instance().loadIfNot(); auto collection_name = getCollectionName(asts); if (!collection_name.has_value()) diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index a1909f514ea..b4aea096c59 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -158,7 +158,7 @@ struct fmt::formatter> } template - auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) + auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) const { return fmt::format_to(context.out(), "{}", elem.value); } diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index ada3e2e9323..163f08be420 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -249,7 +249,7 @@ AzureClientPtr StorageAzureConfiguration::createClient(bool is_read_only, bool a return result; } -void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & collection) + void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 35b19079ca9..bbaa82c51ba 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -51,7 +51,7 @@ public: ContextPtr context) override; protected: - void fromNamedCollection(const NamedCollection & collection) override; + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index a8a9ab5b557..155f51adf61 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -119,7 +119,7 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit setURL(url_str); } -void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & collection) +void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) { std::string url_str; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 01a8b9c5e3b..04884542908 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -46,7 +46,7 @@ public: ContextPtr context) override; private: - void fromNamedCollection(const NamedCollection &) override; + void fromNamedCollection(const NamedCollection &, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; void setURL(const std::string & url_); diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 2c14b38ce01..8277a769a11 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -132,11 +132,11 @@ void WriteBufferFromHDFS::sync() } -void WriteBufferFromHDFS::finalizeImpl() +WriteBufferFromHDFS::~WriteBufferFromHDFS() { try { - next(); + finalize(); } catch (...) { @@ -144,11 +144,5 @@ void WriteBufferFromHDFS::finalizeImpl() } } - -WriteBufferFromHDFS::~WriteBufferFromHDFS() -{ - finalize(); -} - } #endif diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h index 71e6e55addc..e3f0ae96a8f 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h @@ -38,8 +38,6 @@ public: std::string getFileName() const override { return filename; } private: - void finalizeImpl() override; - struct WriteBufferFromHDFSImpl; std::unique_ptr impl; const std::string filename; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 4b217b94730..b33d55105e9 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -106,15 +106,18 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, const auto & config = context->getConfigRef(); const auto & settings = context->getSettingsRef(); - const std::string config_prefix = "s3."; - auto s3_settings = getSettings(config, config_prefix, context, settings.s3_validate_request_settings); + auto s3_settings = getSettings( + config, "s3"/* config_prefix */, context, url.uri_str, settings.s3_validate_request_settings); - request_settings.updateFromSettingsIfChanged(settings); - auth_settings.updateFrom(s3_settings->auth_settings); + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) + { + s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); + s3_settings->request_settings.updateIfChanged(endpoint_settings->request_settings); + } - s3_settings->auth_settings = auth_settings; - s3_settings->request_settings = request_settings; + s3_settings->auth_settings.updateIfChanged(auth_settings); + s3_settings->request_settings.updateIfChanged(request_settings); if (!headers_from_ast.empty()) { @@ -123,10 +126,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, headers_from_ast.begin(), headers_from_ast.end()); } - if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) - s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); - - auto client = getClient(config, config_prefix, context, *s3_settings, false, &url); + auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false); auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); auto s3_capabilities = S3Capabilities { @@ -139,8 +139,9 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, key_generator, "StorageS3", false); } -void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection) +void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection, ContextPtr context) { + const auto settings = context->getSettingsRef(); validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); auto filename = collection.getOrDefault("filename", ""); @@ -159,9 +160,9 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); - request_settings = S3Settings::RequestSettings(collection); + request_settings = S3::RequestSettings(collection, settings, /* validate_settings */true); - static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + static_configuration = !auth_settings.access_key_id.value.empty() || auth_settings.no_sign_request.changed; keys = {url.key}; } @@ -357,7 +358,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ if (no_sign_request) auth_settings.no_sign_request = no_sign_request; - static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + static_configuration = !auth_settings.access_key_id.value.empty() || auth_settings.no_sign_request.changed; auth_settings.no_sign_request = no_sign_request; keys = {url.key}; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 906d10a1a9a..39a646c7df2 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -3,7 +3,7 @@ #include "config.h" #if USE_AWS_S3 -#include +#include #include namespace DB @@ -51,14 +51,14 @@ public: ContextPtr context) override; private: - void fromNamedCollection(const NamedCollection & collection) override; + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; S3::URI url; std::vector keys; S3::AuthSettings auth_settings; - S3Settings::RequestSettings request_settings; + S3::RequestSettings request_settings; HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration. /// If s3 configuration was passed from ast, then it is static. /// If from config - it can be changed with config reload. diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 2c8e60b49d0..90a97a9ea62 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -424,7 +424,7 @@ void StorageObjectStorage::Configuration::initialize( bool with_table_structure) { if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration.fromNamedCollection(*named_collection); + configuration.fromNamedCollection(*named_collection, local_context); else configuration.fromAST(engine_args, local_context, with_table_structure); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index f45d8c1f01a..cf8ec113653 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -193,7 +193,7 @@ public: String structure = "auto"; protected: - virtual void fromNamedCollection(const NamedCollection & collection) = 0; + virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; void assertInitialized() const; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 0a3cf19a590..d13aec4a4f6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -83,7 +83,6 @@ void StorageObjectStorageSink::finalize() { writer->finalize(); writer->flush(); - write_buf->finalize(); } catch (...) { @@ -91,6 +90,8 @@ void StorageObjectStorageSink::finalize() release(); throw; } + + write_buf->finalize(); } void StorageObjectStorageSink::release() diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index b31d0f8a92e..2fc6993369d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -195,12 +195,14 @@ Chunk StorageObjectStorageSource::generate() const auto & object_info = reader.getObjectInfo(); const auto & filename = object_info.getFileName(); chassert(object_info.metadata); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, - read_from_format_info.requested_virtual_columns, - getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), - object_info.metadata->size_bytes, &filename); - + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, read_from_format_info.requested_virtual_columns, + { + .path = getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), + .size = object_info.metadata->size_bytes, + .filename = &filename, + .last_modified = object_info.metadata->last_modified + }); return chunk; } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 0bcbedee41a..ea87d97d81f 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -16,7 +16,8 @@ #include #include #include -#include +#include +#include #include #include #include @@ -64,7 +65,6 @@ ProjectionDescription ProjectionDescription::clone() const other.sample_block_for_keys = sample_block_for_keys; other.metadata = metadata; other.key_size = key_size; - other.is_minmax_count_projection = is_minmax_count_projection; other.primary_key_max_column_name = primary_key_max_column_name; other.partition_value_indices = partition_value_indices; @@ -195,7 +195,6 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( ContextPtr query_context) { ProjectionDescription result; - result.is_minmax_count_projection = true; auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); @@ -282,13 +281,11 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( return result; } - void ProjectionDescription::recalculateWithNewColumns(const ColumnsDescription & new_columns, ContextPtr query_context) { *this = getProjectionFromAST(definition_ast, new_columns, query_context); } - Block ProjectionDescription::calculate(const Block & block, ContextPtr context) const { auto mut_context = Context::createCopy(context); @@ -310,7 +307,9 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 75a97697e00..5f091b4421b 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -56,8 +56,6 @@ struct ProjectionDescription size_t key_size = 0; - bool is_minmax_count_projection = false; - /// If a primary key expression is used in the minmax_count projection, store the name of max expression. String primary_key_max_column_name; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index c3b7ae64c7e..b9d3e071b6c 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -25,6 +25,7 @@ #include #include +#include #include #include #include @@ -42,6 +43,7 @@ #include #include +#include #include @@ -185,11 +187,11 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, bool read_only_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs()))) , primary_key{primary_key_} , rocksdb_dir(std::move(rocksdb_dir_)) , ttl(ttl_) , read_only(read_only_) - , log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs()))) { setInMemoryMetadata(metadata_); setSettings(std::move(settings_)); @@ -352,6 +354,72 @@ bool StorageEmbeddedRocksDB::optimize( return true; } +static_assert(rocksdb::DEBUG_LEVEL == 0); +static_assert(rocksdb::HEADER_LEVEL == 5); +static constexpr std::array, 6> rocksdb_logger_map = { + std::make_pair(DB::LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG), + std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION), + std::make_pair(DB::LogsLevel::warning, Poco::Message::Priority::PRIO_WARNING), + std::make_pair(DB::LogsLevel::error, Poco::Message::Priority::PRIO_ERROR), + std::make_pair(DB::LogsLevel::fatal, Poco::Message::Priority::PRIO_FATAL), + /// Same as default logger does for HEADER_LEVEL + std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION), +}; +class StorageEmbeddedRocksDBLogger : public rocksdb::Logger +{ +public: + explicit StorageEmbeddedRocksDBLogger(const rocksdb::InfoLogLevel log_level, LoggerRawPtr log_) + : rocksdb::Logger(log_level) + , log(log_) + {} + + void Logv(const char * format, va_list ap) override + __attribute__((format(printf, 2, 0))) + { + Logv(rocksdb::InfoLogLevel::DEBUG_LEVEL, format, ap); + } + + void Logv(const rocksdb::InfoLogLevel log_level, const char * format, va_list ap) override + __attribute__((format(printf, 3, 0))) + { + if (log_level < GetInfoLogLevel()) + return; + + auto level = rocksdb_logger_map[log_level]; + + /// stack buffer was enough + { + va_list backup_ap; + va_copy(backup_ap, ap); + std::array stack; + if (vsnprintf(stack.data(), stack.size(), format, backup_ap) < static_cast(stack.size())) + { + va_end(backup_ap); + LOG_IMPL(log, level.first, level.second, "{}", stack.data()); + return; + } + va_end(backup_ap); + } + + /// let's try with a bigger dynamic buffer (but not too huge, since + /// some of rocksdb internal code has also such a limitation, i..e + /// HdfsLogger) + { + va_list backup_ap; + va_copy(backup_ap, ap); + static constexpr int buffer_size = 30000; + std::unique_ptr buffer(new char[buffer_size]); + if (vsnprintf(buffer.get(), buffer_size, format, backup_ap) >= buffer_size) + buffer[buffer_size - 1] = 0; + va_end(backup_ap); + LOG_IMPL(log, level.first, level.second, "{}", buffer.get()); + } + } + +private: + LoggerRawPtr log; +}; + void StorageEmbeddedRocksDB::initDB() { rocksdb::Status status; @@ -448,6 +516,7 @@ void StorageEmbeddedRocksDB::initDB() } } + merged.info_log = std::make_shared(merged.info_log_level, log.get()); merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); if (ttl > 0) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 61592398954..a6aa1ba36a4 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -114,17 +114,19 @@ public: private: SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot); + LoggerPtr log; + MultiVersion storage_settings; const String primary_key; + using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; + mutable SharedMutex rocksdb_ptr_mx; String rocksdb_dir; Int32 ttl; bool read_only; void initDB(); - - LoggerPtr log; }; } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp deleted file mode 100644 index e1b998b2f7f..00000000000 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ /dev/null @@ -1,1174 +0,0 @@ -#include "config.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - -namespace ProfileEvents -{ - extern const Event S3QueueSetFileProcessingMicroseconds; - extern const Event S3QueueSetFileProcessedMicroseconds; - extern const Event S3QueueSetFileFailedMicroseconds; - extern const Event S3QueueFailedFiles; - extern const Event S3QueueProcessedFiles; - extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; - extern const Event S3QueueLockLocalFileStatusesMicroseconds; - extern const Event CannotRemoveEphemeralNode; -}; - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; -} - -namespace -{ - UInt64 getCurrentTime() - { - return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - } - - size_t generateRescheduleInterval(size_t min, size_t max) - { - /// Use more or less random interval for unordered mode cleanup task. - /// So that distributed processing cleanup tasks would not schedule cleanup at the same time. - pcg64 rng(randomSeed()); - return min + rng() % (max - min + 1); - } -} - -std::unique_lock S3QueueFilesMetadata::LocalFileStatuses::lock() const -{ - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueLockLocalFileStatusesMicroseconds); - return std::unique_lock(mutex); -} - -S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getAll() const -{ - auto lk = lock(); - return file_statuses; -} - -S3QueueFilesMetadata::FileStatusPtr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) -{ - auto lk = lock(); - auto it = file_statuses.find(filename); - if (it == file_statuses.end()) - { - if (create) - it = file_statuses.emplace(filename, std::make_shared()).first; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); - } - return it->second; -} - -bool S3QueueFilesMetadata::LocalFileStatuses::remove(const std::string & filename, bool if_exists) -{ - auto lk = lock(); - auto it = file_statuses.find(filename); - if (it == file_statuses.end()) - { - if (if_exists) - return false; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); - } - file_statuses.erase(it); - return true; -} - -std::string S3QueueFilesMetadata::NodeMetadata::toString() const -{ - Poco::JSON::Object json; - json.set("file_path", file_path); - json.set("last_processed_timestamp", getCurrentTime()); - json.set("last_exception", last_exception); - json.set("retries", retries); - json.set("processing_id", processing_id); - - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - oss.exceptions(std::ios::failbit); - Poco::JSON::Stringifier::stringify(json, oss); - return oss.str(); -} - -S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromString(const std::string & metadata_str) -{ - Poco::JSON::Parser parser; - auto json = parser.parse(metadata_str).extract(); - - NodeMetadata metadata; - metadata.file_path = json->getValue("file_path"); - metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); - metadata.last_exception = json->getValue("last_exception"); - metadata.retries = json->getValue("retries"); - metadata.processing_id = json->getValue("processing_id"); - return metadata; -} - -S3QueueFilesMetadata::S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_) - : mode(settings_.mode) - , max_set_size(settings_.s3queue_tracked_files_limit.value) - , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) - , max_loading_retries(settings_.s3queue_loading_retries.value) - , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) - , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) - , shards_num(settings_.s3queue_total_shards_num) - , threads_per_shard(settings_.s3queue_processing_threads_num) - , zookeeper_processing_path(zookeeper_path_ / "processing") - , zookeeper_processed_path(zookeeper_path_ / "processed") - , zookeeper_failed_path(zookeeper_path_ / "failed") - , zookeeper_shards_path(zookeeper_path_ / "shards") - , zookeeper_cleanup_lock_path(zookeeper_path_ / "cleanup_lock") - , log(getLogger("StorageS3Queue(" + zookeeper_path_.string() + ")")) -{ - if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) - { - task = Context::getGlobalContextInstance()->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); - task->activate(); - task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); - } -} - -S3QueueFilesMetadata::~S3QueueFilesMetadata() -{ - deactivateCleanupTask(); -} - -void S3QueueFilesMetadata::deactivateCleanupTask() -{ - shutdown = true; - if (task) - task->deactivate(); -} - -zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const -{ - return Context::getGlobalContextInstance()->getZooKeeper(); -} - -S3QueueFilesMetadata::FileStatusPtr S3QueueFilesMetadata::getFileStatus(const std::string & path) -{ - /// Return a locally cached file status. - return local_file_statuses.get(path, /* create */false); -} - -std::string S3QueueFilesMetadata::getNodeName(const std::string & path) -{ - /// Since with are dealing with paths in s3 which can have "/", - /// we cannot create a zookeeper node with the name equal to path. - /// Therefore we use a hash of the path as a node name. - - SipHash path_hash; - path_hash.update(path); - return toString(path_hash.get64()); -} - -S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( - const std::string & path, - const std::string & exception, - size_t retries) -{ - /// Create a metadata which will be stored in a node named as getNodeName(path). - - /// Since node name is just a hash we want to know to which file it corresponds, - /// so we keep "file_path" in nodes data. - /// "last_processed_timestamp" is needed for TTL metadata nodes enabled by s3queue_tracked_file_ttl_sec. - /// "last_exception" is kept for introspection, should also be visible in system.s3queue_log if it is enabled. - /// "retries" is kept for retrying the processing enabled by s3queue_loading_retries. - NodeMetadata metadata; - metadata.file_path = path; - metadata.last_processed_timestamp = getCurrentTime(); - metadata.last_exception = exception; - metadata.retries = retries; - return metadata; -} - -bool S3QueueFilesMetadata::isShardedProcessing() const -{ - return getProcessingIdsNum() > 1 && mode == S3QueueMode::ORDERED; -} - -size_t S3QueueFilesMetadata::registerNewShard() -{ - if (!isShardedProcessing()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot register a new shard, because processing is not sharded"); - } - - const auto zk_client = getZooKeeper(); - zk_client->createIfNotExists(zookeeper_shards_path, ""); - - std::string shard_node_path; - size_t shard_id = 0; - for (size_t i = 0; i < shards_num; ++i) - { - const auto node_path = getZooKeeperPathForShard(i); - auto err = zk_client->tryCreate(node_path, "", zkutil::CreateMode::Persistent); - if (err == Coordination::Error::ZOK) - { - shard_node_path = node_path; - shard_id = i; - break; - } - else if (err == Coordination::Error::ZNODEEXISTS) - continue; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected error: {}", magic_enum::enum_name(err)); - } - - if (shard_node_path.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to register a new shard"); - - LOG_TRACE(log, "Using shard {} (zk node: {})", shard_id, shard_node_path); - return shard_id; -} - -std::string S3QueueFilesMetadata::getZooKeeperPathForShard(size_t shard_id) const -{ - return zookeeper_shards_path / ("shard" + toString(shard_id)); -} - -void S3QueueFilesMetadata::registerNewShard(size_t shard_id) -{ - if (!isShardedProcessing()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot register a new shard, because processing is not sharded"); - } - - const auto zk_client = getZooKeeper(); - const auto node_path = getZooKeeperPathForShard(shard_id); - zk_client->createAncestors(node_path); - - auto err = zk_client->tryCreate(node_path, "", zkutil::CreateMode::Persistent); - if (err != Coordination::Error::ZOK) - { - if (err == Coordination::Error::ZNODEEXISTS) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot register shard {}: already exists", shard_id); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected error: {}", magic_enum::enum_name(err)); - } -} - -bool S3QueueFilesMetadata::isShardRegistered(size_t shard_id) -{ - const auto zk_client = getZooKeeper(); - const auto node_path = getZooKeeperPathForShard(shard_id); - return zk_client->exists(node_path); -} - -void S3QueueFilesMetadata::unregisterShard(size_t shard_id) -{ - if (!isShardedProcessing()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot unregister a shard, because processing is not sharded"); - } - - const auto zk_client = getZooKeeper(); - const auto node_path = getZooKeeperPathForShard(shard_id); - auto error_code = zk_client->tryRemove(node_path); - if (error_code != Coordination::Error::ZOK - && error_code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException::fromPath(error_code, node_path); -} - -size_t S3QueueFilesMetadata::getProcessingIdsNum() const -{ - return shards_num * threads_per_shard; -} - -std::vector S3QueueFilesMetadata::getProcessingIdsForShard(size_t shard_id) const -{ - std::vector res(threads_per_shard); - std::iota(res.begin(), res.end(), shard_id * threads_per_shard); - return res; -} - -bool S3QueueFilesMetadata::isProcessingIdBelongsToShard(size_t id, size_t shard_id) const -{ - return shard_id * threads_per_shard <= id && id < (shard_id + 1) * threads_per_shard; -} - -size_t S3QueueFilesMetadata::getIdForProcessingThread(size_t thread_id, size_t shard_id) const -{ - return shard_id * threads_per_shard + thread_id; -} - -size_t S3QueueFilesMetadata::getProcessingIdForPath(const std::string & path) const -{ - return sipHash64(path) % getProcessingIdsNum(); -} - -S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) -{ - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); - auto file_status = local_file_statuses.get(path, /* create */true); - - /// Check locally cached file status. - /// Processed or Failed state is always cached. - /// Processing state is cached only if processing is being done by current clickhouse server - /// (because If another server is doing the processing, - /// we cannot know if state changes without checking with zookeeper so there is no point in cache here). - - { - std::lock_guard lock(file_status->metadata_lock); - switch (file_status->state) - { - case FileStatus::State::Processing: - { - LOG_TEST(log, "File {} is already processing", path); - return {}; - } - case FileStatus::State::Processed: - { - LOG_TEST(log, "File {} is already processed", path); - return {}; - } - case FileStatus::State::Failed: - { - /// If max_loading_retries == 0, file is not retriable. - if (max_loading_retries == 0) - { - LOG_TEST(log, "File {} is failed and processing retries are disabled", path); - return {}; - } - - /// Otherwise file_status->retries is also cached. - /// In case file_status->retries >= max_loading_retries we can fully rely that it is true - /// and will not attempt processing it. - /// But in case file_status->retries < max_loading_retries we cannot be sure - /// (another server could have done a try after we cached retries value), - /// so check with zookeeper here. - if (file_status->retries >= max_loading_retries) - { - LOG_TEST(log, "File {} is failed and processing retries are exceeeded", path); - return {}; - } - - break; - } - case FileStatus::State::None: - { - /// The file was not processed by current server and file status was not cached, - /// check metadata in zookeeper. - break; - } - } - } - - /// Another thread could already be trying to set file as processing. - /// So there is no need to attempt the same, better to continue with the next file. - std::unique_lock processing_lock(file_status->processing_lock, std::defer_lock); - if (!processing_lock.try_lock()) - { - return {}; - } - - /// Let's go and check metadata in zookeeper and try to create a /processing ephemeral node. - /// If successful, return result with processing node holder. - SetFileProcessingResult result; - ProcessingNodeHolderPtr processing_node_holder; - - switch (mode) - { - case S3QueueMode::ORDERED: - { - std::tie(result, processing_node_holder) = trySetFileAsProcessingForOrderedMode(path, file_status); - break; - } - case S3QueueMode::UNORDERED: - { - std::tie(result, processing_node_holder) = trySetFileAsProcessingForUnorderedMode(path, file_status); - break; - } - } - - /// Cache file status, save some statistics. - switch (result) - { - case SetFileProcessingResult::Success: - { - std::lock_guard lock(file_status->metadata_lock); - file_status->state = FileStatus::State::Processing; - - file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); - timer.cancel(); - - if (!file_status->processing_start_time) - file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - - return processing_node_holder; - } - case SetFileProcessingResult::AlreadyProcessed: - { - std::lock_guard lock(file_status->metadata_lock); - file_status->state = FileStatus::State::Processed; - return {}; - } - case SetFileProcessingResult::AlreadyFailed: - { - std::lock_guard lock(file_status->metadata_lock); - file_status->state = FileStatus::State::Failed; - return {}; - } - case SetFileProcessingResult::ProcessingByOtherNode: - { - /// We cannot save any local state here, see comment above. - return {}; - } - } -} - -std::pair -S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path, const FileStatusPtr & file_status) -{ - /// In one zookeeper transaction do the following: - /// 1. check that corresponding persistent nodes do not exist in processed/ and failed/; - /// 2. create an ephemenral node in /processing if it does not exist; - /// Return corresponding status if any of the step failed. - - const auto node_name = getNodeName(path); - const auto zk_client = getZooKeeper(); - auto node_metadata = createNodeMetadata(path); - node_metadata.processing_id = getRandomASCIIString(10); - - Coordination::Requests requests; - - requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processed_path / node_name, -1)); - - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name, -1)); - - requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - - if (code == Coordination::Error::ZOK) - { - auto holder = std::make_unique( - node_metadata.processing_id, path, zookeeper_processing_path / node_name, file_status, zk_client, log); - return std::pair{SetFileProcessingResult::Success, std::move(holder)}; - } - - if (responses[0]->error != Coordination::Error::ZOK) - { - return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; - } - else if (responses[2]->error != Coordination::Error::ZOK) - { - return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; - } - else if (responses[4]->error != Coordination::Error::ZOK) - { - return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", magic_enum::enum_name(code)); - } -} - -std::pair -S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path, const FileStatusPtr & file_status) -{ - /// Same as for Unordered mode. - /// The only difference is the check if the file is already processed. - /// For Ordered mode we do not keep a separate /processed/hash_node for each file - /// but instead we only keep a maximum processed file - /// (since all files are ordered and new files have a lexically bigger name, it makes sense). - - const auto node_name = getNodeName(path); - const auto zk_client = getZooKeeper(); - auto node_metadata = createNodeMetadata(path); - node_metadata.processing_id = getRandomASCIIString(10); - - while (true) - { - /// Get a /processed node content - max_processed path. - /// Compare our path to it. - /// If file is not yet processed, check corresponding /failed node and try create /processing node - /// and in the same zookeeper transaction also check that /processed node did not change - /// in between, e.g. that stat.version remained the same. - /// If the version did change - retry (since we cannot do Get and Create requests - /// in the same zookeeper transaction, so we use a while loop with tries). - - auto processed_node = isShardedProcessing() - ? zookeeper_processed_path / toString(getProcessingIdForPath(path)) - : zookeeper_processed_path; - - NodeMetadata processed_node_metadata; - Coordination::Stat processed_node_stat; - std::string data; - auto processed_node_exists = zk_client->tryGet(processed_node, data, &processed_node_stat); - if (processed_node_exists && !data.empty()) - processed_node_metadata = NodeMetadata::fromString(data); - - auto max_processed_file_path = processed_node_metadata.file_path; - if (!max_processed_file_path.empty() && path <= max_processed_file_path) - { - LOG_TEST(log, "File {} is already processed, max processed file: {}", path, max_processed_file_path); - return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; - } - - Coordination::Requests requests; - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name, -1)); - - requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); - - if (processed_node_exists) - { - requests.push_back(zkutil::makeCheckRequest(processed_node, processed_node_stat.version)); - } - else - { - requests.push_back(zkutil::makeCreateRequest(processed_node, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(processed_node, -1)); - } - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - { - auto holder = std::make_unique( - node_metadata.processing_id, path, zookeeper_processing_path / node_name, file_status, zk_client, log); - - LOG_TEST(log, "File {} is ready to be processed", path); - return std::pair{SetFileProcessingResult::Success, std::move(holder)}; - } - - if (responses[0]->error != Coordination::Error::ZOK) - { - LOG_TEST(log, "Skipping file `{}`: failed", path); - return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; - } - else if (responses[2]->error != Coordination::Error::ZOK) - { - LOG_TEST(log, "Skipping file `{}`: already processing", path); - return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; - } - else - { - LOG_TEST(log, "Version of max processed file changed. Retrying the check for file `{}`", path); - } - } -} - -void S3QueueFilesMetadata::setFileProcessed(ProcessingNodeHolderPtr holder) -{ - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); - auto file_status = holder->getFileStatus(); - { - std::lock_guard lock(file_status->metadata_lock); - file_status->state = FileStatus::State::Processed; - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - } - - SCOPE_EXIT({ - file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); - timer.cancel(); - }); - - switch (mode) - { - case S3QueueMode::ORDERED: - { - setFileProcessedForOrderedMode(holder); - break; - } - case S3QueueMode::UNORDERED: - { - setFileProcessedForUnorderedMode(holder); - break; - } - } - - ProfileEvents::increment(ProfileEvents::S3QueueProcessedFiles); -} - -void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolderPtr holder) -{ - /// Create a persistent node in /processed and remove ephemeral node from /processing. - - const auto & path = holder->path; - const auto node_name = getNodeName(path); - const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = getZooKeeper(); - - Coordination::Requests requests; - requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, node_metadata, zkutil::CreateMode::Persistent)); - - Coordination::Responses responses; - if (holder->remove(&requests, &responses)) - { - LOG_TRACE(log, "Moved file `{}` to processed", path); - if (max_loading_retries) - zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); - return; - } - - if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot create a persistent node in /processed since it already exists"); - } - - LOG_WARNING(log, - "Cannot set file ({}) as processed since ephemeral node in /processing" - "does not exist with expected id, " - "this could be a result of expired zookeeper session", path); -} - - -void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) -{ - auto processed_node_path = isShardedProcessing() - ? zookeeper_processed_path / toString(getProcessingIdForPath(holder->path)) - : zookeeper_processed_path; - - setFileProcessedForOrderedModeImpl(holder->path, holder, processed_node_path); -} - -void S3QueueFilesMetadata::setFileProcessedForOrderedModeImpl( - const std::string & path, ProcessingNodeHolderPtr holder, const std::string & processed_node_path) -{ - /// Update a persistent node in /processed and remove ephemeral node from /processing. - - const auto node_name = getNodeName(path); - const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = getZooKeeper(); - - LOG_TRACE(log, "Setting file `{}` as processed (at {})", path, processed_node_path); - while (true) - { - std::string res; - Coordination::Stat stat; - bool exists = zk_client->tryGet(processed_node_path, res, &stat); - Coordination::Requests requests; - if (exists) - { - if (!res.empty()) - { - auto metadata = NodeMetadata::fromString(res); - if (metadata.file_path >= path) - { - LOG_TRACE(log, "File {} is already processed, current max processed file: {}", path, metadata.file_path); - return; - } - } - requests.push_back(zkutil::makeSetRequest(processed_node_path, node_metadata, stat.version)); - } - else - { - requests.push_back(zkutil::makeCreateRequest(processed_node_path, node_metadata, zkutil::CreateMode::Persistent)); - } - - Coordination::Responses responses; - if (holder) - { - if (holder->remove(&requests, &responses)) - { - LOG_TRACE(log, "Moved file `{}` to processed", path); - if (max_loading_retries) - zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); - return; - } - } - else - { - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - { - LOG_TRACE(log, "Moved file `{}` to processed", path); - return; - } - } - - /// Failed to update max processed node, retry. - if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) - { - LOG_TRACE(log, "Failed to update processed node for path {} ({}). Will retry.", - path, magic_enum::enum_name(responses[0]->error)); - continue; - } - - LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " - "does not exist with expected processing id does not exist, " - "this could be a result of expired zookeeper session", path); - return; - } -} - -void S3QueueFilesMetadata::setFileProcessed(const std::string & path, size_t shard_id) -{ - if (mode != S3QueueMode::ORDERED) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can set file as preprocessed only for Ordered mode"); - - if (isShardedProcessing()) - { - for (const auto & processor : getProcessingIdsForShard(shard_id)) - setFileProcessedForOrderedModeImpl(path, nullptr, zookeeper_processed_path / toString(processor)); - } - else - { - setFileProcessedForOrderedModeImpl(path, nullptr, zookeeper_processed_path); - } -} - -void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const String & exception_message) -{ - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); - const auto & path = holder->path; - - auto file_status = holder->getFileStatus(); - { - std::lock_guard lock(file_status->metadata_lock); - file_status->state = FileStatus::State::Failed; - file_status->last_exception = exception_message; - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - } - - ProfileEvents::increment(ProfileEvents::S3QueueFailedFiles); - - SCOPE_EXIT({ - file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); - timer.cancel(); - }); - - const auto node_name = getNodeName(path); - auto node_metadata = createNodeMetadata(path, exception_message); - const auto zk_client = getZooKeeper(); - - /// Is file retriable? - if (max_loading_retries == 0) - { - /// File is not retriable, - /// just create a node in /failed and remove a node from /processing. - - Coordination::Requests requests; - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, - node_metadata.toString(), - zkutil::CreateMode::Persistent)); - Coordination::Responses responses; - if (holder->remove(&requests, &responses)) - { - LOG_TRACE(log, "File `{}` failed to process and will not be retried. " - "Error: {}", path, exception_message); - return; - } - - if (responses[0]->error != Coordination::Error::ZOK) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot create a persistent node in /failed since it already exists"); - } - - LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " - "does not exist with expected processing id does not exist, " - "this could be a result of expired zookeeper session", path); - - return; - } - - /// So file is retriable. - /// Let's do an optimization here. - /// Instead of creating a persistent /failed/node_hash node - /// we create a persistent /failed/node_hash.retriable node. - /// This allows us to make less zookeeper requests as we avoid checking - /// the number of already done retries in trySetFileAsProcessing. - - const auto node_name_with_retriable_suffix = node_name + ".retriable"; - Coordination::Stat stat; - std::string res; - - /// Extract the number of already done retries from node_hash.retriable node if it exists. - if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) - { - auto failed_node_metadata = NodeMetadata::fromString(res); - node_metadata.retries = failed_node_metadata.retries + 1; - - std::lock_guard lock(file_status->metadata_lock); - file_status->retries = node_metadata.retries; - } - - LOG_TRACE(log, "File `{}` failed to process, try {}/{} (Error: {})", - path, node_metadata.retries, max_loading_retries, exception_message); - - /// Check if file can be retried further or not. - if (node_metadata.retries >= max_loading_retries) - { - /// File is no longer retriable. - /// Make a persistent node /failed/node_hash, remove /failed/node_hash.retriable node and node in /processing. - - Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name_with_retriable_suffix, - stat.version)); - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, - node_metadata.toString(), - zkutil::CreateMode::Persistent)); - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - return; - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); - } - else - { - /// File is still retriable, update retries count and remove node from /processing. - - Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); - if (node_metadata.retries == 0) - { - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name_with_retriable_suffix, - node_metadata.toString(), - zkutil::CreateMode::Persistent)); - } - else - { - requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name_with_retriable_suffix, - node_metadata.toString(), - stat.version)); - } - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - return; - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); - } -} - -S3QueueFilesMetadata::ProcessingNodeHolder::ProcessingNodeHolder( - const std::string & processing_id_, - const std::string & path_, - const std::string & zk_node_path_, - FileStatusPtr file_status_, - zkutil::ZooKeeperPtr zk_client_, - LoggerPtr logger_) - : zk_client(zk_client_) - , file_status(file_status_) - , path(path_) - , zk_node_path(zk_node_path_) - , processing_id(processing_id_) - , log(logger_) -{ -} - -S3QueueFilesMetadata::ProcessingNodeHolder::~ProcessingNodeHolder() -{ - if (!removed) - remove(); -} - -bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * requests, Coordination::Responses * responses) -{ - if (removed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing node is already removed"); - - LOG_TEST(log, "Removing processing node {} ({})", zk_node_path, path); - - try - { - if (!zk_client->expired()) - { - /// Is is possible that we created an ephemeral processing node - /// but session expired and someone other created an ephemeral processing node. - /// To avoid deleting this new node, check processing_id. - std::string res; - Coordination::Stat stat; - if (zk_client->tryGet(zk_node_path, res, &stat)) - { - auto node_metadata = NodeMetadata::fromString(res); - if (node_metadata.processing_id == processing_id) - { - if (requests) - { - requests->push_back(zkutil::makeRemoveRequest(zk_node_path, stat.version)); - auto code = zk_client->tryMulti(*requests, *responses); - removed = code == Coordination::Error::ZOK; - } - else - { - zk_client->remove(zk_node_path); - removed = true; - } - return removed; - } - else - LOG_WARNING(log, "Cannot remove {} since processing id changed: {} -> {}", - zk_node_path, processing_id, node_metadata.processing_id); - } - else - LOG_DEBUG(log, "Cannot remove {}, node doesn't exist, " - "probably because of session expiration", zk_node_path); - - /// TODO: this actually would mean that we already processed (or partially processed) - /// the data but another thread will try processing it again and data can be duplicated. - /// This can be solved via persistenly saving last processed offset in the file. - } - else - { - ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); - LOG_DEBUG(log, "Cannot remove {} since session has been expired", zk_node_path); - } - } - catch (...) - { - ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); - LOG_ERROR(log, "Failed to remove processing node for file {}: {}", path, getCurrentExceptionMessage(true)); - } - return false; -} - -void S3QueueFilesMetadata::cleanupThreadFunc() -{ - /// A background task is responsible for maintaining - /// max_set_size and max_set_age settings for `unordered` processing mode. - - if (shutdown) - return; - - try - { - cleanupThreadFuncImpl(); - } - catch (...) - { - LOG_ERROR(log, "Failed to cleanup nodes in zookeeper: {}", getCurrentExceptionMessage(true)); - } - - if (shutdown) - return; - - task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); -} - -void S3QueueFilesMetadata::cleanupThreadFuncImpl() -{ - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueCleanupMaxSetSizeOrTTLMicroseconds); - const auto zk_client = getZooKeeper(); - - Strings processed_nodes; - auto code = zk_client->tryGetChildren(zookeeper_processed_path, processed_nodes); - if (code != Coordination::Error::ZOK) - { - if (code == Coordination::Error::ZNONODE) - { - LOG_TEST(log, "Path {} does not exist", zookeeper_processed_path.string()); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error: {}", magic_enum::enum_name(code)); - } - - Strings failed_nodes; - code = zk_client->tryGetChildren(zookeeper_failed_path, failed_nodes); - if (code != Coordination::Error::ZOK) - { - if (code == Coordination::Error::ZNONODE) - { - LOG_TEST(log, "Path {} does not exist", zookeeper_failed_path.string()); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error: {}", magic_enum::enum_name(code)); - } - - const size_t nodes_num = processed_nodes.size() + failed_nodes.size(); - if (!nodes_num) - { - LOG_TEST(log, "There are neither processed nor failed nodes"); - return; - } - - chassert(max_set_size || max_set_age_sec); - const bool check_nodes_limit = max_set_size > 0; - const bool check_nodes_ttl = max_set_age_sec > 0; - - const bool nodes_limit_exceeded = nodes_num > max_set_size; - if ((!nodes_limit_exceeded || !check_nodes_limit) && !check_nodes_ttl) - { - LOG_TEST(log, "No limit exceeded"); - return; - } - - LOG_TRACE(log, "Will check limits for {} nodes", nodes_num); - - /// Create a lock so that with distributed processing - /// multiple nodes do not execute cleanup in parallel. - auto ephemeral_node = zkutil::EphemeralNodeHolder::tryCreate(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); - if (!ephemeral_node) - { - LOG_TEST(log, "Cleanup is already being executed by another node"); - return; - } - /// TODO because of this lock we might not update local file statuses on time on one of the nodes. - - struct Node - { - std::string zk_path; - NodeMetadata metadata; - }; - auto node_cmp = [](const Node & a, const Node & b) - { - return std::tie(a.metadata.last_processed_timestamp, a.metadata.file_path) - < std::tie(b.metadata.last_processed_timestamp, b.metadata.file_path); - }; - - /// Ordered in ascending order of timestamps. - std::set sorted_nodes(node_cmp); - - for (const auto & node : processed_nodes) - { - const std::string path = zookeeper_processed_path / node; - try - { - std::string metadata_str; - if (zk_client->tryGet(path, metadata_str)) - { - sorted_nodes.emplace(path, NodeMetadata::fromString(metadata_str)); - LOG_TEST(log, "Fetched metadata for node {}", path); - } - else - LOG_ERROR(log, "Failed to fetch node metadata {}", path); - } - catch (const zkutil::KeeperException & e) - { - if (e.code != Coordination::Error::ZCONNECTIONLOSS) - { - LOG_WARNING(log, "Unexpected exception: {}", getCurrentExceptionMessage(true)); - chassert(false); - } - - /// Will retry with a new zk connection. - throw; - } - } - - for (const auto & node : failed_nodes) - { - const std::string path = zookeeper_failed_path / node; - try - { - std::string metadata_str; - if (zk_client->tryGet(path, metadata_str)) - { - sorted_nodes.emplace(path, NodeMetadata::fromString(metadata_str)); - LOG_TEST(log, "Fetched metadata for node {}", path); - } - else - LOG_ERROR(log, "Failed to fetch node metadata {}", path); - } - catch (const zkutil::KeeperException & e) - { - if (e.code != Coordination::Error::ZCONNECTIONLOSS) - { - LOG_WARNING(log, "Unexpected exception: {}", getCurrentExceptionMessage(true)); - chassert(false); - } - - /// Will retry with a new zk connection. - throw; - } - } - - auto get_nodes_str = [&]() - { - WriteBufferFromOwnString wb; - for (const auto & [node, metadata] : sorted_nodes) - wb << fmt::format("Node: {}, path: {}, timestamp: {};\n", node, metadata.file_path, metadata.last_processed_timestamp); - return wb.str(); - }; - LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", max_set_size, max_set_age_sec, get_nodes_str()); - - size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes_num - max_set_size : 0; - for (const auto & node : sorted_nodes) - { - if (nodes_to_remove) - { - LOG_TRACE(log, "Removing node at path {} ({}) because max files limit is reached", - node.metadata.file_path, node.zk_path); - - local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); - - code = zk_client->tryRemove(node.zk_path); - if (code == Coordination::Error::ZOK) - --nodes_to_remove; - else - LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", node.zk_path, code); - } - else if (check_nodes_ttl) - { - UInt64 node_age = getCurrentTime() - node.metadata.last_processed_timestamp; - if (node_age >= max_set_age_sec) - { - LOG_TRACE(log, "Removing node at path {} ({}) because file is reached", - node.metadata.file_path, node.zk_path); - - local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); - - code = zk_client->tryRemove(node.zk_path); - if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", node.zk_path, code); - } - else if (!nodes_to_remove) - { - /// Nodes limit satisfied. - /// Nodes ttl satisfied as well as if current node is under tll, then all remaining as well - /// (because we are iterating in timestamp ascending order). - break; - } - } - else - { - /// Nodes limit and ttl are satisfied. - break; - } - } - - LOG_TRACE(log, "Node limits check finished"); -} - -bool S3QueueFilesMetadata::checkSettings(const S3QueueSettings & settings) const -{ - return mode == settings.mode - && max_set_size == settings.s3queue_tracked_files_limit.value - && max_set_age_sec == settings.s3queue_tracked_file_ttl_sec.value - && max_loading_retries == settings.s3queue_loading_retries.value - && min_cleanup_interval_ms == settings.s3queue_cleanup_interval_min_ms.value - && max_cleanup_interval_ms == settings.s3queue_cleanup_interval_max_ms.value; -} - -} diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h deleted file mode 100644 index e26af1d25c5..00000000000 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ /dev/null @@ -1,215 +0,0 @@ -#pragma once -#include "config.h" - -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; -namespace Poco { class Logger; } - -namespace DB -{ -struct S3QueueSettings; -class StorageS3Queue; - -/** - * A class for managing S3Queue metadata in zookeeper, e.g. - * the following folders: - * - /processing - * - /processed - * - /failed - * - * Depending on S3Queue processing mode (ordered or unordered) - * we can differently store metadata in /processed node. - * - * Implements caching of zookeeper metadata for faster responses. - * Cached part is located in LocalFileStatuses. - * - * In case of Unordered mode - if files TTL is enabled or maximum tracked files limit is set - * starts a background cleanup thread which is responsible for maintaining them. - */ -class S3QueueFilesMetadata -{ -public: - class ProcessingNodeHolder; - using ProcessingNodeHolderPtr = std::shared_ptr; - - S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_); - - ~S3QueueFilesMetadata(); - - void setFileProcessed(ProcessingNodeHolderPtr holder); - void setFileProcessed(const std::string & path, size_t shard_id); - - void setFileFailed(ProcessingNodeHolderPtr holder, const std::string & exception_message); - - struct FileStatus - { - enum class State : uint8_t - { - Processing, - Processed, - Failed, - None - }; - State state = State::None; - - std::atomic processed_rows = 0; - time_t processing_start_time = 0; - time_t processing_end_time = 0; - size_t retries = 0; - std::string last_exception; - ProfileEvents::Counters profile_counters; - - std::mutex processing_lock; - std::mutex metadata_lock; - }; - using FileStatusPtr = std::shared_ptr; - using FileStatuses = std::unordered_map; - - /// Set file as processing, if it is not alreaty processed, failed or processing. - ProcessingNodeHolderPtr trySetFileAsProcessing(const std::string & path); - - FileStatusPtr getFileStatus(const std::string & path); - - FileStatuses getFileStateses() const { return local_file_statuses.getAll(); } - - bool checkSettings(const S3QueueSettings & settings) const; - - void deactivateCleanupTask(); - - /// Should the table use sharded processing? - /// We use sharded processing for Ordered mode of S3Queue table. - /// It allows to parallelize processing within a single server - /// and to allow distributed processing. - bool isShardedProcessing() const; - - /// Register a new shard for processing. - /// Return a shard id of registered shard. - size_t registerNewShard(); - /// Register a new shard for processing by given id. - /// Throws exception if shard by this id is already registered. - void registerNewShard(size_t shard_id); - /// Unregister shard from keeper. - void unregisterShard(size_t shard_id); - bool isShardRegistered(size_t shard_id); - - /// Total number of processing ids. - /// A processing id identifies a single processing thread. - /// There might be several processing ids per shard. - size_t getProcessingIdsNum() const; - /// Get processing ids identified with requested shard. - std::vector getProcessingIdsForShard(size_t shard_id) const; - /// Check if given processing id belongs to a given shard. - bool isProcessingIdBelongsToShard(size_t id, size_t shard_id) const; - /// Get a processing id for processing thread by given thread id. - /// thread id is a value in range [0, threads_per_shard]. - size_t getIdForProcessingThread(size_t thread_id, size_t shard_id) const; - - /// Calculate which processing id corresponds to a given file path. - /// The file will be processed by a thread related to this processing id. - size_t getProcessingIdForPath(const std::string & path) const; - -private: - const S3QueueMode mode; - const UInt64 max_set_size; - const UInt64 max_set_age_sec; - const UInt64 max_loading_retries; - const size_t min_cleanup_interval_ms; - const size_t max_cleanup_interval_ms; - const size_t shards_num; - const size_t threads_per_shard; - - const fs::path zookeeper_processing_path; - const fs::path zookeeper_processed_path; - const fs::path zookeeper_failed_path; - const fs::path zookeeper_shards_path; - const fs::path zookeeper_cleanup_lock_path; - - LoggerPtr log; - - std::atomic_bool shutdown = false; - BackgroundSchedulePool::TaskHolder task; - - std::string getNodeName(const std::string & path); - - zkutil::ZooKeeperPtr getZooKeeper() const; - - void setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder); - void setFileProcessedForUnorderedMode(ProcessingNodeHolderPtr holder); - std::string getZooKeeperPathForShard(size_t shard_id) const; - - void setFileProcessedForOrderedModeImpl( - const std::string & path, ProcessingNodeHolderPtr holder, const std::string & processed_node_path); - - enum class SetFileProcessingResult : uint8_t - { - Success, - ProcessingByOtherNode, - AlreadyProcessed, - AlreadyFailed, - }; - std::pair trySetFileAsProcessingForOrderedMode(const std::string & path, const FileStatusPtr & file_status); - std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path, const FileStatusPtr & file_status); - - struct NodeMetadata - { - std::string file_path; UInt64 last_processed_timestamp = 0; - std::string last_exception; - UInt64 retries = 0; - std::string processing_id; /// For ephemeral processing node. - - std::string toString() const; - static NodeMetadata fromString(const std::string & metadata_str); - }; - - NodeMetadata createNodeMetadata(const std::string & path, const std::string & exception = "", size_t retries = 0); - - void cleanupThreadFunc(); - void cleanupThreadFuncImpl(); - - struct LocalFileStatuses - { - FileStatuses file_statuses; - mutable std::mutex mutex; - - FileStatuses getAll() const; - FileStatusPtr get(const std::string & filename, bool create); - bool remove(const std::string & filename, bool if_exists); - std::unique_lock lock() const; - }; - LocalFileStatuses local_file_statuses; -}; - -class S3QueueFilesMetadata::ProcessingNodeHolder -{ - friend class S3QueueFilesMetadata; -public: - ProcessingNodeHolder( - const std::string & processing_id_, - const std::string & path_, - const std::string & zk_node_path_, - FileStatusPtr file_status_, - zkutil::ZooKeeperPtr zk_client_, - LoggerPtr logger_); - - ~ProcessingNodeHolder(); - - FileStatusPtr getFileStatus() { return file_status; } - -private: - bool remove(Coordination::Requests * requests = nullptr, Coordination::Responses * responses = nullptr); - - zkutil::ZooKeeperPtr zk_client; - FileStatusPtr file_status; - std::string path; - std::string zk_node_path; - std::string processing_id; - bool removed = false; - LoggerPtr log; -}; - -} diff --git a/src/Storages/S3Queue/S3QueueIFileMetadata.cpp b/src/Storages/S3Queue/S3QueueIFileMetadata.cpp new file mode 100644 index 00000000000..6c4089115d4 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueIFileMetadata.cpp @@ -0,0 +1,354 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace ProfileEvents +{ + extern const Event S3QueueProcessedFiles; + extern const Event S3QueueFailedFiles; +}; + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + zkutil::ZooKeeperPtr getZooKeeper() + { + return Context::getGlobalContextInstance()->getZooKeeper(); + } + + time_t now() + { + return std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } +} + +void S3QueueIFileMetadata::FileStatus::onProcessing() +{ + state = FileStatus::State::Processing; + processing_start_time = now(); +} + +void S3QueueIFileMetadata::FileStatus::onProcessed() +{ + state = FileStatus::State::Processed; + processing_end_time = now(); +} + +void S3QueueIFileMetadata::FileStatus::onFailed(const std::string & exception) +{ + state = FileStatus::State::Failed; + processing_end_time = now(); + std::lock_guard lock(last_exception_mutex); + last_exception = exception; +} + +std::string S3QueueIFileMetadata::FileStatus::getException() const +{ + std::lock_guard lock(last_exception_mutex); + return last_exception; +} + +std::string S3QueueIFileMetadata::NodeMetadata::toString() const +{ + Poco::JSON::Object json; + json.set("file_path", file_path); + json.set("last_processed_timestamp", now()); + json.set("last_exception", last_exception); + json.set("retries", retries); + json.set("processing_id", processing_id); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); +} + +S3QueueIFileMetadata::NodeMetadata S3QueueIFileMetadata::NodeMetadata::fromString(const std::string & metadata_str) +{ + Poco::JSON::Parser parser; + auto json = parser.parse(metadata_str).extract(); + chassert(json); + + NodeMetadata metadata; + metadata.file_path = json->getValue("file_path"); + metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); + metadata.last_exception = json->getValue("last_exception"); + metadata.retries = json->getValue("retries"); + metadata.processing_id = json->getValue("processing_id"); + return metadata; +} + +S3QueueIFileMetadata::S3QueueIFileMetadata( + const std::string & path_, + const std::string & processing_node_path_, + const std::string & processed_node_path_, + const std::string & failed_node_path_, + FileStatusPtr file_status_, + size_t max_loading_retries_, + LoggerPtr log_) + : path(path_) + , node_name(getNodeName(path_)) + , file_status(file_status_) + , max_loading_retries(max_loading_retries_) + , processing_node_path(processing_node_path_) + , processed_node_path(processed_node_path_) + , failed_node_path(failed_node_path_) + , node_metadata(createNodeMetadata(path)) + , log(log_) + , processing_node_id_path(processing_node_path + "_processing_id") +{ + LOG_TEST(log, "Path: {}, node_name: {}, max_loading_retries: {}, " + "processed_path: {}, processing_path: {}, failed_path: {}", + path, node_name, max_loading_retries, + processed_node_path, processing_node_path, failed_node_path); +} + +S3QueueIFileMetadata::~S3QueueIFileMetadata() +{ + if (processing_id_version.has_value()) + { + file_status->onFailed("Uncaught exception"); + LOG_TEST(log, "Removing processing node in destructor for file: {}", path); + try + { + auto zk_client = getZooKeeper(); + + Coordination::Requests requests; + requests.push_back(zkutil::makeCheckRequest(processing_node_id_path, processing_id_version.value())); + requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + + Coordination::Responses responses; + const auto code = zk_client->tryMulti(requests, responses); + if (code != Coordination::Error::ZOK + && !Coordination::isHardwareError(code) + && code != Coordination::Error::ZBADVERSION + && code != Coordination::Error::ZNONODE) + { + LOG_WARNING(log, "Unexpected error while removing processing node: {}", code); + chassert(false); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + +std::string S3QueueIFileMetadata::getNodeName(const std::string & path) +{ + /// Since with are dealing with paths in s3 which can have "/", + /// we cannot create a zookeeper node with the name equal to path. + /// Therefore we use a hash of the path as a node name. + + SipHash path_hash; + path_hash.update(path); + return toString(path_hash.get64()); +} + +S3QueueIFileMetadata::NodeMetadata S3QueueIFileMetadata::createNodeMetadata( + const std::string & path, + const std::string & exception, + size_t retries) +{ + /// Create a metadata which will be stored in a node named as getNodeName(path). + + /// Since node name is just a hash we want to know to which file it corresponds, + /// so we keep "file_path" in nodes data. + /// "last_processed_timestamp" is needed for TTL metadata nodes enabled by s3queue_tracked_file_ttl_sec. + /// "last_exception" is kept for introspection, should also be visible in system.s3queue_log if it is enabled. + /// "retries" is kept for retrying the processing enabled by s3queue_loading_retries. + NodeMetadata metadata; + metadata.file_path = path; + metadata.last_processed_timestamp = now(); + metadata.last_exception = exception; + metadata.retries = retries; + return metadata; +} + +std::string S3QueueIFileMetadata::getProcessorInfo(const std::string & processor_id) +{ + /// Add information which will be useful for debugging just in case. + Poco::JSON::Object json; + json.set("hostname", DNSResolver::instance().getHostName()); + json.set("processor_id", processor_id); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); +} + +bool S3QueueIFileMetadata::setProcessing() +{ + auto state = file_status->state.load(); + if (state == FileStatus::State::Processing + || state == FileStatus::State::Processed + || (state == FileStatus::State::Failed && file_status->retries >= max_loading_retries)) + { + LOG_TEST(log, "File {} has non-processable state `{}`", path, file_status->state.load()); + return false; + } + + /// An optimization for local parallel processing. + std::unique_lock processing_lock(file_status->processing_lock, std::defer_lock); + if (!processing_lock.try_lock()) + return {}; + + auto [success, file_state] = setProcessingImpl(); + if (success) + file_status->onProcessing(); + else + file_status->updateState(file_state); + + LOG_TEST(log, "File {} has state `{}`: will {}process (processing id version: {})", + path, file_state, success ? "" : "not ", + processing_id_version.has_value() ? toString(processing_id_version.value()) : "None"); + + return success; +} + +void S3QueueIFileMetadata::setProcessed() +{ + LOG_TRACE(log, "Setting file {} as processed (path: {})", path, processed_node_path); + + ProfileEvents::increment(ProfileEvents::S3QueueProcessedFiles); + file_status->onProcessed(); + setProcessedImpl(); + + processing_id.reset(); + processing_id_version.reset(); + + LOG_TRACE(log, "Set file {} as processed (rows: {})", path, file_status->processed_rows); +} + +void S3QueueIFileMetadata::setFailed(const std::string & exception) +{ + LOG_TRACE(log, "Setting file {} as failed (exception: {}, path: {})", path, exception, failed_node_path); + + ProfileEvents::increment(ProfileEvents::S3QueueFailedFiles); + file_status->onFailed(exception); + node_metadata.last_exception = exception; + + if (max_loading_retries == 0) + setFailedNonRetriable(); + else + setFailedRetriable(); + + processing_id.reset(); + processing_id_version.reset(); + + LOG_TRACE(log, "Set file {} as failed (rows: {})", path, file_status->processed_rows); +} + +void S3QueueIFileMetadata::setFailedNonRetriable() +{ + auto zk_client = getZooKeeper(); + Coordination::Requests requests; + requests.push_back(zkutil::makeCreateRequest(failed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + + Coordination::Responses responses; + const auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + { + LOG_TRACE(log, "File `{}` failed to process and will not be retried. ", path); + return; + } + + if (Coordination::isHardwareError(responses[0]->error)) + { + LOG_WARNING(log, "Cannot set file as failed: lost connection to keeper"); + return; + } + + if (responses[0]->error == Coordination::Error::ZNODEEXISTS) + { + LOG_WARNING(log, "Cannot create a persistent node in /failed since it already exists"); + chassert(false); + return; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error while setting file as failed: {}", code); +} + +void S3QueueIFileMetadata::setFailedRetriable() +{ + /// Instead of creating a persistent /failed/node_hash node + /// we create a persistent /failed/node_hash.retriable node. + /// This allows us to make less zookeeper requests as we avoid checking + /// the number of already done retries in trySetFileAsProcessing. + + auto retrieable_failed_node_path = failed_node_path + ".retriable"; + auto zk_client = getZooKeeper(); + + /// Extract the number of already done retries from node_hash.retriable node if it exists. + Coordination::Stat stat; + std::string res; + if (zk_client->tryGet(retrieable_failed_node_path, res, &stat)) + { + auto failed_node_metadata = NodeMetadata::fromString(res); + node_metadata.retries = failed_node_metadata.retries + 1; + file_status->retries = node_metadata.retries; + } + + LOG_TRACE(log, "File `{}` failed to process, try {}/{}", + path, node_metadata.retries, max_loading_retries); + + Coordination::Requests requests; + if (node_metadata.retries >= max_loading_retries) + { + /// File is no longer retriable. + /// Make a persistent node /failed/node_hash, + /// remove /failed/node_hash.retriable node and node in /processing. + + requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + requests.push_back(zkutil::makeRemoveRequest(retrieable_failed_node_path, stat.version)); + requests.push_back( + zkutil::makeCreateRequest( + failed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); + + } + else + { + /// File is still retriable, + /// update retries count and remove node from /processing. + + requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + if (node_metadata.retries == 0) + { + requests.push_back( + zkutil::makeCreateRequest( + retrieable_failed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); + } + else + { + requests.push_back( + zkutil::makeSetRequest( + retrieable_failed_node_path, node_metadata.toString(), stat.version)); + } + } + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Failed to set file {} as failed (code: {})", path, code); +} + +} diff --git a/src/Storages/S3Queue/S3QueueIFileMetadata.h b/src/Storages/S3Queue/S3QueueIFileMetadata.h new file mode 100644 index 00000000000..e0b0d16cbcc --- /dev/null +++ b/src/Storages/S3Queue/S3QueueIFileMetadata.h @@ -0,0 +1,114 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class S3QueueIFileMetadata +{ +public: + struct FileStatus + { + enum class State : uint8_t + { + Processing, + Processed, + Failed, + None + }; + + void onProcessing(); + void onProcessed(); + void onFailed(const std::string & exception); + void updateState(State state_) { state = state_; } + + std::string getException() const; + + std::mutex processing_lock; + + std::atomic state = State::None; + std::atomic processed_rows = 0; + std::atomic processing_start_time = 0; + std::atomic processing_end_time = 0; + std::atomic retries = 0; + ProfileEvents::Counters profile_counters; + + private: + mutable std::mutex last_exception_mutex; + std::string last_exception; + }; + using FileStatusPtr = std::shared_ptr; + + explicit S3QueueIFileMetadata( + const std::string & path_, + const std::string & processing_node_path_, + const std::string & processed_node_path_, + const std::string & failed_node_path_, + FileStatusPtr file_status_, + size_t max_loading_retries_, + LoggerPtr log_); + + virtual ~S3QueueIFileMetadata(); + + bool setProcessing(); + void setProcessed(); + void setFailed(const std::string & exception); + + virtual void setProcessedAtStartRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr & zk_client) = 0; + + FileStatusPtr getFileStatus() { return file_status; } + + struct NodeMetadata + { + std::string file_path; UInt64 last_processed_timestamp = 0; + std::string last_exception; + UInt64 retries = 0; + std::string processing_id; /// For ephemeral processing node. + + std::string toString() const; + static NodeMetadata fromString(const std::string & metadata_str); + }; + +protected: + virtual std::pair setProcessingImpl() = 0; + virtual void setProcessedImpl() = 0; + void setFailedNonRetriable(); + void setFailedRetriable(); + + const std::string path; + const std::string node_name; + const FileStatusPtr file_status; + const size_t max_loading_retries; + + const std::string processing_node_path; + const std::string processed_node_path; + const std::string failed_node_path; + + NodeMetadata node_metadata; + LoggerPtr log; + + /// processing node is ephemeral, so we cannot verify with it if + /// this node was created by a certain processor on a previous s3 queue processing stage, + /// because we could get a session expired in between the stages + /// and someone else could just create this processing node. + /// Therefore we also create a persistent processing node + /// which is updated on each creation of ephemeral processing node. + /// We use the version of this node to verify the version of the processing ephemeral node. + const std::string processing_node_id_path; + /// Id of the processor. + std::optional processing_id; + /// Version of the processing id persistent node. + std::optional processing_id_version; + + static std::string getNodeName(const std::string & path); + + static NodeMetadata createNodeMetadata(const std::string & path, const std::string & exception = {}, size_t retries = 0); + + static std::string getProcessorInfo(const std::string & processor_id); +}; + +} diff --git a/src/Storages/S3Queue/S3QueueMetadata.cpp b/src/Storages/S3Queue/S3QueueMetadata.cpp new file mode 100644 index 00000000000..e828e9f0716 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueMetadata.cpp @@ -0,0 +1,485 @@ +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace ProfileEvents +{ + extern const Event S3QueueSetFileProcessingMicroseconds; + extern const Event S3QueueSetFileProcessedMicroseconds; + extern const Event S3QueueSetFileFailedMicroseconds; + extern const Event S3QueueFailedFiles; + extern const Event S3QueueProcessedFiles; + extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; + extern const Event S3QueueLockLocalFileStatusesMicroseconds; +}; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int REPLICA_ALREADY_EXISTS; + extern const int INCOMPATIBLE_COLUMNS; +} + +namespace +{ + UInt64 getCurrentTime() + { + return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + } + + size_t generateRescheduleInterval(size_t min, size_t max) + { + /// Use more or less random interval for unordered mode cleanup task. + /// So that distributed processing cleanup tasks would not schedule cleanup at the same time. + pcg64 rng(randomSeed()); + return min + rng() % (max - min + 1); + } + + zkutil::ZooKeeperPtr getZooKeeper() + { + return Context::getGlobalContextInstance()->getZooKeeper(); + } +} + +class S3QueueMetadata::LocalFileStatuses +{ +public: + LocalFileStatuses() = default; + + FileStatuses getAll() const + { + auto lk = lock(); + return file_statuses; + } + + FileStatusPtr get(const std::string & filename, bool create) + { + auto lk = lock(); + auto it = file_statuses.find(filename); + if (it == file_statuses.end()) + { + if (create) + it = file_statuses.emplace(filename, std::make_shared()).first; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); + } + return it->second; + } + + bool remove(const std::string & filename, bool if_exists) + { + auto lk = lock(); + auto it = file_statuses.find(filename); + if (it == file_statuses.end()) + { + if (if_exists) + return false; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); + } + file_statuses.erase(it); + return true; + } + +private: + FileStatuses file_statuses; + mutable std::mutex mutex; + + std::unique_lock lock() const + { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueLockLocalFileStatusesMicroseconds); + return std::unique_lock(mutex); + } +}; + +S3QueueMetadata::S3QueueMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_) + : settings(settings_) + , zookeeper_path(zookeeper_path_) + , buckets_num(getBucketsNum(settings_)) + , log(getLogger("StorageS3Queue(" + zookeeper_path_.string() + ")")) + , local_file_statuses(std::make_shared()) +{ + if (settings.mode == S3QueueMode::UNORDERED + && (settings.s3queue_tracked_files_limit || settings.s3queue_tracked_file_ttl_sec)) + { + task = Context::getGlobalContextInstance()->getSchedulePool().createTask( + "S3QueueCleanupFunc", + [this] { cleanupThreadFunc(); }); + + task->activate(); + task->scheduleAfter( + generateRescheduleInterval( + settings.s3queue_cleanup_interval_min_ms, settings.s3queue_cleanup_interval_max_ms)); + } +} + +S3QueueMetadata::~S3QueueMetadata() +{ + shutdown(); +} + +void S3QueueMetadata::shutdown() +{ + shutdown_called = true; + if (task) + task->deactivate(); +} + +void S3QueueMetadata::checkSettings(const S3QueueSettings & settings_) const +{ + S3QueueTableMetadata::checkEquals(settings, settings_); +} + +S3QueueMetadata::FileStatusPtr S3QueueMetadata::getFileStatus(const std::string & path) +{ + return local_file_statuses->get(path, /* create */false); +} + +S3QueueMetadata::FileStatuses S3QueueMetadata::getFileStatuses() const +{ + return local_file_statuses->getAll(); +} + +S3QueueMetadata::FileMetadataPtr S3QueueMetadata::getFileMetadata( + const std::string & path, + S3QueueOrderedFileMetadata::BucketInfoPtr bucket_info) +{ + auto file_status = local_file_statuses->get(path, /* create */true); + switch (settings.mode.value) + { + case S3QueueMode::ORDERED: + return std::make_shared( + zookeeper_path, + path, + file_status, + bucket_info, + buckets_num, + settings.s3queue_loading_retries, + log); + case S3QueueMode::UNORDERED: + return std::make_shared( + zookeeper_path, + path, + file_status, + settings.s3queue_loading_retries, + log); + } +} + +size_t S3QueueMetadata::getBucketsNum(const S3QueueSettings & settings) +{ + if (settings.s3queue_buckets) + return settings.s3queue_buckets; + if (settings.s3queue_processing_threads_num) + return settings.s3queue_processing_threads_num; + return 0; +} + +size_t S3QueueMetadata::getBucketsNum(const S3QueueTableMetadata & settings) +{ + if (settings.buckets) + return settings.buckets; + if (settings.processing_threads_num) + return settings.processing_threads_num; + return 0; +} + +bool S3QueueMetadata::useBucketsForProcessing() const +{ + return settings.mode == S3QueueMode::ORDERED && (buckets_num > 1); +} + +S3QueueMetadata::Bucket S3QueueMetadata::getBucketForPath(const std::string & path) const +{ + return S3QueueOrderedFileMetadata::getBucketForPath(path, buckets_num); +} + +S3QueueOrderedFileMetadata::BucketHolderPtr +S3QueueMetadata::tryAcquireBucket(const Bucket & bucket, const Processor & processor) +{ + return S3QueueOrderedFileMetadata::tryAcquireBucket(zookeeper_path, bucket, processor); +} + +void S3QueueMetadata::initialize( + const ConfigurationPtr & configuration, + const StorageInMemoryMetadata & storage_metadata) +{ + const auto metadata_from_table = S3QueueTableMetadata(*configuration, settings, storage_metadata); + const auto & columns_from_table = storage_metadata.getColumns(); + const auto table_metadata_path = zookeeper_path / "metadata"; + const auto metadata_paths = settings.mode == S3QueueMode::ORDERED + ? S3QueueOrderedFileMetadata::getMetadataPaths(buckets_num) + : S3QueueUnorderedFileMetadata::getMetadataPaths(); + + auto zookeeper = getZooKeeper(); + zookeeper->createAncestors(zookeeper_path); + + for (size_t i = 0; i < 1000; ++i) + { + if (zookeeper->exists(table_metadata_path)) + { + const auto metadata_from_zk = S3QueueTableMetadata::parse(zookeeper->get(fs::path(zookeeper_path) / "metadata")); + const auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); + + metadata_from_table.checkEquals(metadata_from_zk); + if (columns_from_zk != columns_from_table) + { + throw Exception( + ErrorCodes::INCOMPATIBLE_COLUMNS, + "Table columns structure in ZooKeeper is different from local table structure. " + "Local columns:\n{}\nZookeeper columns:\n{}", + columns_from_table.toString(), columns_from_zk.toString()); + } + return; + } + + Coordination::Requests requests; + requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(table_metadata_path, metadata_from_table.toString(), zkutil::CreateMode::Persistent)); + + for (const auto & path : metadata_paths) + { + const auto zk_path = zookeeper_path / path; + requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); + } + + if (!settings.s3queue_last_processed_path.value.empty()) + getFileMetadata(settings.s3queue_last_processed_path)->setProcessedAtStartRequests(requests, zookeeper); + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(requests, responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + auto exception = zkutil::KeeperMultiException(code, requests, responses); + LOG_INFO(log, "Got code `{}` for path: {}. " + "It looks like the table {} was created by another server at the same moment, " + "will retry", code, exception.getPathForFirstFailedOp(), zookeeper_path.string()); + continue; + } + else if (code != Coordination::Error::ZOK) + zkutil::KeeperMultiException::check(code, requests, responses); + + return; + } + + 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"); +} + +void S3QueueMetadata::cleanupThreadFunc() +{ + /// A background task is responsible for maintaining + /// settings.s3queue_tracked_files_limit and max_set_age settings for `unordered` processing mode. + + if (shutdown_called) + return; + + try + { + cleanupThreadFuncImpl(); + } + catch (...) + { + LOG_ERROR(log, "Failed to cleanup nodes in zookeeper: {}", getCurrentExceptionMessage(true)); + } + + if (shutdown_called) + return; + + task->scheduleAfter( + generateRescheduleInterval( + settings.s3queue_cleanup_interval_min_ms, settings.s3queue_cleanup_interval_max_ms)); +} + +void S3QueueMetadata::cleanupThreadFuncImpl() +{ + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueCleanupMaxSetSizeOrTTLMicroseconds); + const auto zk_client = getZooKeeper(); + const fs::path zookeeper_processed_path = zookeeper_path / "processed"; + const fs::path zookeeper_failed_path = zookeeper_path / "failed"; + const fs::path zookeeper_cleanup_lock_path = zookeeper_path / "cleanup_lock"; + + Strings processed_nodes; + auto code = zk_client->tryGetChildren(zookeeper_processed_path, processed_nodes); + if (code != Coordination::Error::ZOK) + { + if (code == Coordination::Error::ZNONODE) + { + LOG_TEST(log, "Path {} does not exist", zookeeper_processed_path.string()); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error: {}", magic_enum::enum_name(code)); + } + + Strings failed_nodes; + code = zk_client->tryGetChildren(zookeeper_failed_path, failed_nodes); + if (code != Coordination::Error::ZOK) + { + if (code == Coordination::Error::ZNONODE) + { + LOG_TEST(log, "Path {} does not exist", zookeeper_failed_path.string()); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error: {}", magic_enum::enum_name(code)); + } + + const size_t nodes_num = processed_nodes.size() + failed_nodes.size(); + if (!nodes_num) + { + LOG_TEST(log, "There are neither processed nor failed nodes (in {} and in {})", + zookeeper_processed_path.string(), zookeeper_failed_path.string()); + return; + } + + chassert(settings.s3queue_tracked_files_limit || settings.s3queue_tracked_file_ttl_sec); + const bool check_nodes_limit = settings.s3queue_tracked_files_limit > 0; + const bool check_nodes_ttl = settings.s3queue_tracked_file_ttl_sec > 0; + + const bool nodes_limit_exceeded = nodes_num > settings.s3queue_tracked_files_limit; + if ((!nodes_limit_exceeded || !check_nodes_limit) && !check_nodes_ttl) + { + LOG_TEST(log, "No limit exceeded"); + return; + } + + LOG_TRACE(log, "Will check limits for {} nodes", nodes_num); + + /// Create a lock so that with distributed processing + /// multiple nodes do not execute cleanup in parallel. + auto ephemeral_node = zkutil::EphemeralNodeHolder::tryCreate(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); + if (!ephemeral_node) + { + LOG_TEST(log, "Cleanup is already being executed by another node"); + return; + } + /// TODO because of this lock we might not update local file statuses on time on one of the nodes. + + struct Node + { + std::string zk_path; + S3QueueIFileMetadata::NodeMetadata metadata; + }; + auto node_cmp = [](const Node & a, const Node & b) + { + return std::tie(a.metadata.last_processed_timestamp, a.metadata.file_path) + < std::tie(b.metadata.last_processed_timestamp, b.metadata.file_path); + }; + + /// Ordered in ascending order of timestamps. + std::set sorted_nodes(node_cmp); + + auto fetch_nodes = [&](const Strings & nodes, const fs::path & base_path) + { + for (const auto & node : nodes) + { + const std::string path = base_path / node; + try + { + std::string metadata_str; + if (zk_client->tryGet(path, metadata_str)) + { + sorted_nodes.emplace(path, S3QueueIFileMetadata::NodeMetadata::fromString(metadata_str)); + LOG_TEST(log, "Fetched metadata for node {}", path); + } + else + LOG_ERROR(log, "Failed to fetch node metadata {}", path); + } + catch (const zkutil::KeeperException & e) + { + if (!Coordination::isHardwareError(e.code)) + { + LOG_WARNING(log, "Unexpected exception: {}", getCurrentExceptionMessage(true)); + chassert(false); + } + + /// Will retry with a new zk connection. + throw; + } + } + }; + + fetch_nodes(processed_nodes, zookeeper_processed_path); + fetch_nodes(failed_nodes, zookeeper_failed_path); + + auto get_nodes_str = [&]() + { + WriteBufferFromOwnString wb; + for (const auto & [node, metadata] : sorted_nodes) + wb << fmt::format("Node: {}, path: {}, timestamp: {};\n", node, metadata.file_path, metadata.last_processed_timestamp); + return wb.str(); + }; + LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", settings.s3queue_tracked_files_limit, settings.s3queue_tracked_file_ttl_sec, get_nodes_str()); + + size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes_num - settings.s3queue_tracked_files_limit : 0; + for (const auto & node : sorted_nodes) + { + if (nodes_to_remove) + { + LOG_TRACE(log, "Removing node at path {} ({}) because max files limit is reached", + node.metadata.file_path, node.zk_path); + + local_file_statuses->remove(node.metadata.file_path, /* if_exists */true); + + code = zk_client->tryRemove(node.zk_path); + if (code == Coordination::Error::ZOK) + --nodes_to_remove; + else + LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", node.zk_path, code); + } + else if (check_nodes_ttl) + { + UInt64 node_age = getCurrentTime() - node.metadata.last_processed_timestamp; + if (node_age >= settings.s3queue_tracked_file_ttl_sec) + { + LOG_TRACE(log, "Removing node at path {} ({}) because file ttl is reached", + node.metadata.file_path, node.zk_path); + + local_file_statuses->remove(node.metadata.file_path, /* if_exists */true); + + code = zk_client->tryRemove(node.zk_path); + if (code != Coordination::Error::ZOK) + LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", node.zk_path, code); + } + else if (!nodes_to_remove) + { + /// Nodes limit satisfied. + /// Nodes ttl satisfied as well as if current node is under tll, then all remaining as well + /// (because we are iterating in timestamp ascending order). + break; + } + } + else + { + /// Nodes limit and ttl are satisfied. + break; + } + } + + LOG_TRACE(log, "Node limits check finished"); +} + +} diff --git a/src/Storages/S3Queue/S3QueueMetadata.h b/src/Storages/S3Queue/S3QueueMetadata.h new file mode 100644 index 00000000000..25d01fb52b9 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueMetadata.h @@ -0,0 +1,94 @@ +#pragma once +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; +namespace Poco { class Logger; } + +namespace DB +{ +struct S3QueueSettings; +class StorageS3Queue; +struct S3QueueTableMetadata; +struct StorageInMemoryMetadata; +using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + +/** + * A class for managing S3Queue metadata in zookeeper, e.g. + * the following folders: + * - /processed + * - /processing + * - /failed + * + * In case we use buckets for processing for Ordered mode, the structure looks like: + * - /buckets//processed -- persistent node, information about last processed file. + * - /buckets//lock -- ephemeral node, used for acquiring bucket lock. + * - /processing + * - /failed + * + * Depending on S3Queue processing mode (ordered or unordered) + * we can differently store metadata in /processed node. + * + * Implements caching of zookeeper metadata for faster responses. + * Cached part is located in LocalFileStatuses. + * + * In case of Unordered mode - if files TTL is enabled or maximum tracked files limit is set + * starts a background cleanup thread which is responsible for maintaining them. + */ +class S3QueueMetadata +{ +public: + using FileStatus = S3QueueIFileMetadata::FileStatus; + using FileMetadataPtr = std::shared_ptr; + using FileStatusPtr = std::shared_ptr; + using FileStatuses = std::unordered_map; + using Bucket = size_t; + using Processor = std::string; + + S3QueueMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_); + ~S3QueueMetadata(); + + void initialize(const ConfigurationPtr & configuration, const StorageInMemoryMetadata & storage_metadata); + void checkSettings(const S3QueueSettings & settings) const; + void shutdown(); + + FileMetadataPtr getFileMetadata(const std::string & path, S3QueueOrderedFileMetadata::BucketInfoPtr bucket_info = {}); + + FileStatusPtr getFileStatus(const std::string & path); + FileStatuses getFileStatuses() const; + + /// Method of Ordered mode parallel processing. + bool useBucketsForProcessing() const; + Bucket getBucketForPath(const std::string & path) const; + S3QueueOrderedFileMetadata::BucketHolderPtr tryAcquireBucket(const Bucket & bucket, const Processor & processor); + + static size_t getBucketsNum(const S3QueueSettings & settings); + static size_t getBucketsNum(const S3QueueTableMetadata & settings); + +private: + void cleanupThreadFunc(); + void cleanupThreadFuncImpl(); + + const S3QueueSettings settings; + const fs::path zookeeper_path; + const size_t buckets_num; + + LoggerPtr log; + + std::atomic_bool shutdown_called = false; + BackgroundSchedulePool::TaskHolder task; + + class LocalFileStatuses; + std::shared_ptr local_file_statuses; +}; + +} diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp index 0c3c26adfe0..a319b21ca3e 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -21,17 +21,13 @@ S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3 auto it = metadata_by_path.find(zookeeper_path); if (it == metadata_by_path.end()) { - auto files_metadata = std::make_shared(zookeeper_path, settings); + auto files_metadata = std::make_shared(zookeeper_path, settings); it = metadata_by_path.emplace(zookeeper_path, std::move(files_metadata)).first; } - else if (it->second.metadata->checkSettings(settings)) - { - it->second.ref_count += 1; - } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with the same `s3queue_zookeeper_path` " - "was already created but with different settings"); + it->second.metadata->checkSettings(settings); + it->second.ref_count += 1; } return it->second.metadata; } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.h b/src/Storages/S3Queue/S3QueueMetadataFactory.h index c5e94d59050..80e96f8aa7e 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.h +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace DB { @@ -9,7 +9,7 @@ namespace DB class S3QueueMetadataFactory final : private boost::noncopyable { public: - using FilesMetadataPtr = std::shared_ptr; + using FilesMetadataPtr = std::shared_ptr; static S3QueueMetadataFactory & instance(); @@ -22,9 +22,9 @@ public: private: struct Metadata { - explicit Metadata(std::shared_ptr metadata_) : metadata(metadata_), ref_count(1) {} + explicit Metadata(std::shared_ptr metadata_) : metadata(metadata_), ref_count(1) {} - std::shared_ptr metadata; + std::shared_ptr metadata; /// TODO: the ref count should be kept in keeper, because of the case with distributed processing. size_t ref_count = 0; }; diff --git a/src/Storages/S3Queue/S3QueueOrderedFileMetadata.cpp b/src/Storages/S3Queue/S3QueueOrderedFileMetadata.cpp new file mode 100644 index 00000000000..bac87c95cc9 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueOrderedFileMetadata.cpp @@ -0,0 +1,413 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + S3QueueOrderedFileMetadata::Bucket getBucketForPathImpl(const std::string & path, size_t buckets_num) + { + return sipHash64(path) % buckets_num; + } + + std::string getProcessedPathForBucket(const std::filesystem::path & zk_path, size_t bucket) + { + return zk_path / "buckets" / toString(bucket) / "processed"; + } + + std::string getProcessedPath(const std::filesystem::path & zk_path, const std::string & path, size_t buckets_num) + { + if (buckets_num > 1) + return getProcessedPathForBucket(zk_path, getBucketForPathImpl(path, buckets_num)); + else + return zk_path / "processed"; + } + + zkutil::ZooKeeperPtr getZooKeeper() + { + return Context::getGlobalContextInstance()->getZooKeeper(); + } +} + +S3QueueOrderedFileMetadata::BucketHolder::BucketHolder( + const Bucket & bucket_, + int bucket_version_, + const std::string & bucket_lock_path_, + const std::string & bucket_lock_id_path_, + zkutil::ZooKeeperPtr zk_client_) + : bucket_info(std::make_shared(BucketInfo{ + .bucket = bucket_, + .bucket_version = bucket_version_, + .bucket_lock_path = bucket_lock_path_, + .bucket_lock_id_path = bucket_lock_id_path_})) + , zk_client(zk_client_) +{ +} + +void S3QueueOrderedFileMetadata::BucketHolder::release() +{ + if (released) + return; + + released = true; + LOG_TEST(getLogger("S3QueueBucketHolder"), "Releasing bucket {}", bucket_info->bucket); + + Coordination::Requests requests; + /// Check that bucket lock version has not changed + /// (which could happen if session had expired as bucket_lock_path is ephemeral node). + requests.push_back(zkutil::makeCheckRequest(bucket_info->bucket_lock_id_path, bucket_info->bucket_version)); + /// Remove bucket lock. + requests.push_back(zkutil::makeRemoveRequest(bucket_info->bucket_lock_path, -1)); + + Coordination::Responses responses; + const auto code = zk_client->tryMulti(requests, responses); + zkutil::KeeperMultiException::check(code, requests, responses); +} + +S3QueueOrderedFileMetadata::BucketHolder::~BucketHolder() +{ + try + { + release(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +S3QueueOrderedFileMetadata::S3QueueOrderedFileMetadata( + const std::filesystem::path & zk_path_, + const std::string & path_, + FileStatusPtr file_status_, + BucketInfoPtr bucket_info_, + size_t buckets_num_, + size_t max_loading_retries_, + LoggerPtr log_) + : S3QueueIFileMetadata( + path_, + /* processing_node_path */zk_path_ / "processing" / getNodeName(path_), + /* processed_node_path */getProcessedPath(zk_path_, path_, buckets_num_), + /* failed_node_path */zk_path_ / "failed" / getNodeName(path_), + file_status_, + max_loading_retries_, + log_) + , buckets_num(buckets_num_) + , zk_path(zk_path_) + , bucket_info(bucket_info_) +{ +} + +std::vector S3QueueOrderedFileMetadata::getMetadataPaths(size_t buckets_num) +{ + if (buckets_num > 1) + { + std::vector paths{"buckets", "failed", "processing"}; + for (size_t i = 0; i < buckets_num; ++i) + paths.push_back("buckets/" + toString(i)); + return paths; + } + else + return {"failed", "processing"}; +} + +bool S3QueueOrderedFileMetadata::getMaxProcessedFile( + NodeMetadata & result, + Coordination::Stat * stat, + const zkutil::ZooKeeperPtr & zk_client) +{ + return getMaxProcessedFile(result, stat, processed_node_path, zk_client); +} + +bool S3QueueOrderedFileMetadata::getMaxProcessedFile( + NodeMetadata & result, + Coordination::Stat * stat, + const std::string & processed_node_path_, + const zkutil::ZooKeeperPtr & zk_client) +{ + std::string data; + if (zk_client->tryGet(processed_node_path_, data, stat)) + { + if (!data.empty()) + result = NodeMetadata::fromString(data); + return true; + } + return false; +} + +S3QueueOrderedFileMetadata::Bucket S3QueueOrderedFileMetadata::getBucketForPath(const std::string & path_, size_t buckets_num) +{ + return getBucketForPathImpl(path_, buckets_num); +} + +S3QueueOrderedFileMetadata::BucketHolderPtr S3QueueOrderedFileMetadata::tryAcquireBucket( + const std::filesystem::path & zk_path, + const Bucket & bucket, + const Processor & processor) +{ + const auto zk_client = getZooKeeper(); + const auto bucket_lock_path = zk_path / "buckets" / toString(bucket) / "lock"; + const auto bucket_lock_id_path = zk_path / "buckets" / toString(bucket) / "lock_id"; + const auto processor_info = getProcessorInfo(processor); + + Coordination::Requests requests; + + /// Create bucket lock node as ephemeral node. + requests.push_back(zkutil::makeCreateRequest(bucket_lock_path, "", zkutil::CreateMode::Ephemeral)); + + /// Create bucket lock id node as persistent node if it does not exist yet. + requests.push_back( + zkutil::makeCreateRequest( + bucket_lock_id_path, processor_info, zkutil::CreateMode::Persistent, /* ignore_if_exists */true)); + + /// Update bucket lock id path. We use its version as a version of ephemeral bucket lock node. + /// (See comment near S3QueueIFileMetadata::processing_node_version). + requests.push_back(zkutil::makeSetRequest(bucket_lock_id_path, processor_info, -1)); + + Coordination::Responses responses; + const auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + { + const auto * set_response = dynamic_cast(responses[2].get()); + const auto bucket_lock_version = set_response->stat.version; + + LOG_TEST( + getLogger("S3QueueOrderedFileMetadata"), + "Processor {} acquired bucket {} for processing (bucket lock version: {})", + processor, bucket, bucket_lock_version); + + return std::make_shared( + bucket, + bucket_lock_version, + bucket_lock_path, + bucket_lock_id_path, + zk_client); + } + + if (code == Coordination::Error::ZNODEEXISTS) + return nullptr; + + if (Coordination::isHardwareError(code)) + return nullptr; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error: {}", code); +} + +std::pair S3QueueOrderedFileMetadata::setProcessingImpl() +{ + /// In one zookeeper transaction do the following: + enum RequestType + { + /// node_name is not within failed persistent nodes + FAILED_PATH_DOESNT_EXIST = 0, + /// node_name ephemeral processing node was successfully created + CREATED_PROCESSING_PATH = 2, + /// update processing id + SET_PROCESSING_ID = 4, + /// bucket version did not change + CHECKED_BUCKET_VERSION = 5, + /// max_processed_node version did not change + CHECKED_MAX_PROCESSED_PATH = 6, + }; + + const auto zk_client = getZooKeeper(); + processing_id = node_metadata.processing_id = getRandomASCIIString(10); + auto processor_info = getProcessorInfo(processing_id.value()); + + while (true) + { + NodeMetadata processed_node; + Coordination::Stat processed_node_stat; + bool has_processed_node = getMaxProcessedFile(processed_node, &processed_node_stat, zk_client); + if (has_processed_node) + { + LOG_TEST(log, "Current max processed file {} from path: {}", + processed_node.file_path, processed_node_path); + + if (!processed_node.file_path.empty() && path <= processed_node.file_path) + { + return {false, FileStatus::State::Processed}; + } + } + + Coordination::Requests requests; + requests.push_back(zkutil::makeCreateRequest(failed_node_path, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(failed_node_path, -1)); + requests.push_back(zkutil::makeCreateRequest(processing_node_path, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); + + requests.push_back( + zkutil::makeCreateRequest( + processing_node_id_path, processor_info, zkutil::CreateMode::Persistent, /* ignore_if_exists */true)); + requests.push_back(zkutil::makeSetRequest(processing_node_id_path, processor_info, -1)); + + if (bucket_info) + requests.push_back(zkutil::makeCheckRequest(bucket_info->bucket_lock_id_path, bucket_info->bucket_version)); + + /// TODO: for ordered processing with buckets it should be enough to check only bucket lock version, + /// so may be remove creation and check for processing_node_id if bucket_info is set? + + if (has_processed_node) + { + requests.push_back(zkutil::makeCheckRequest(processed_node_path, processed_node_stat.version)); + } + else + { + requests.push_back(zkutil::makeCreateRequest(processed_node_path, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(processed_node_path, -1)); + } + + Coordination::Responses responses; + const auto code = zk_client->tryMulti(requests, responses); + auto is_request_failed = [&](RequestType type) { return responses[type]->error != Coordination::Error::ZOK; }; + + if (code == Coordination::Error::ZOK) + { + const auto * set_response = dynamic_cast(responses[SET_PROCESSING_ID].get()); + processing_id_version = set_response->stat.version; + return {true, FileStatus::State::None}; + } + + if (is_request_failed(FAILED_PATH_DOESNT_EXIST)) + return {false, FileStatus::State::Failed}; + + if (is_request_failed(CREATED_PROCESSING_PATH)) + return {false, FileStatus::State::Processing}; + + if (bucket_info && is_request_failed(CHECKED_BUCKET_VERSION)) + { + LOG_TEST(log, "Version of bucket lock changed: {}. Will retry for file `{}`", code, path); + continue; + } + + if (is_request_failed(bucket_info ? CHECKED_MAX_PROCESSED_PATH : CHECKED_BUCKET_VERSION)) + { + LOG_TEST(log, "Version of max processed file changed: {}. Will retry for file `{}`", code, path); + continue; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected response state: {}", code); + } +} + +void S3QueueOrderedFileMetadata::setProcessedAtStartRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr & zk_client) +{ + if (buckets_num > 1) + { + for (size_t i = 0; i < buckets_num; ++i) + { + auto path = getProcessedPathForBucket(zk_path, i); + setProcessedRequests(requests, zk_client, path, /* ignore_if_exists */true); + } + } + else + { + setProcessedRequests(requests, zk_client, processed_node_path, /* ignore_if_exists */true); + } +} + +void S3QueueOrderedFileMetadata::setProcessedRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr & zk_client, + const std::string & processed_node_path_, + bool ignore_if_exists) +{ + NodeMetadata processed_node; + Coordination::Stat processed_node_stat; + if (getMaxProcessedFile(processed_node, &processed_node_stat, processed_node_path_, zk_client)) + { + LOG_TEST(log, "Current max processed file: {}, condition less: {}", + processed_node.file_path, bool(path <= processed_node.file_path)); + + if (!processed_node.file_path.empty() && path <= processed_node.file_path) + { + LOG_TRACE(log, "File {} is already processed, current max processed file: {}", path, processed_node.file_path); + + if (ignore_if_exists) + return; + + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "File ({}) is already processed, while expected it not to be (path: {})", + path, processed_node_path_); + } + requests.push_back(zkutil::makeSetRequest(processed_node_path_, node_metadata.toString(), processed_node_stat.version)); + } + else + { + LOG_TEST(log, "Max processed file does not exist, creating at: {}", processed_node_path_); + requests.push_back(zkutil::makeCreateRequest(processed_node_path_, node_metadata.toString(), zkutil::CreateMode::Persistent)); + } + + if (processing_id_version.has_value()) + { + requests.push_back(zkutil::makeCheckRequest(processing_node_id_path, processing_id_version.value())); + requests.push_back(zkutil::makeRemoveRequest(processing_node_id_path, processing_id_version.value())); + requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + } +} + +void S3QueueOrderedFileMetadata::setProcessedImpl() +{ + /// In one zookeeper transaction do the following: + enum RequestType + { + SET_MAX_PROCESSED_PATH = 0, + CHECK_PROCESSING_ID_PATH = 1, /// Optional. + REMOVE_PROCESSING_ID_PATH = 2, /// Optional. + REMOVE_PROCESSING_PATH = 3, /// Optional. + }; + + const auto zk_client = getZooKeeper(); + std::string failure_reason; + + while (true) + { + Coordination::Requests requests; + setProcessedRequests(requests, zk_client, processed_node_path, /* ignore_if_exists */false); + + Coordination::Responses responses; + auto is_request_failed = [&](RequestType type) { return responses[type]->error != Coordination::Error::ZOK; }; + + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + { + if (max_loading_retries) + zk_client->tryRemove(failed_node_path + ".retriable", -1); + return; + } + + if (Coordination::isHardwareError(code)) + failure_reason = "Lost connection to keeper"; + else if (is_request_failed(SET_MAX_PROCESSED_PATH)) + { + LOG_TRACE(log, "Cannot set file {} as processed. " + "Failed to update processed node: {}. " + "Will retry.", path, code); + continue; + } + else if (is_request_failed(CHECK_PROCESSING_ID_PATH)) + failure_reason = "Version of processing id node changed"; + else if (is_request_failed(REMOVE_PROCESSING_PATH)) + failure_reason = "Failed to remove processing path"; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", code); + + LOG_WARNING(log, "Cannot set file {} as processed: {}. Reason: {}", path, code, failure_reason); + return; + } +} + +} diff --git a/src/Storages/S3Queue/S3QueueOrderedFileMetadata.h b/src/Storages/S3Queue/S3QueueOrderedFileMetadata.h new file mode 100644 index 00000000000..698ec0f54cc --- /dev/null +++ b/src/Storages/S3Queue/S3QueueOrderedFileMetadata.h @@ -0,0 +1,97 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +class S3QueueOrderedFileMetadata : public S3QueueIFileMetadata +{ +public: + using Processor = std::string; + using Bucket = size_t; + struct BucketInfo + { + Bucket bucket; + int bucket_version; + std::string bucket_lock_path; + std::string bucket_lock_id_path; + }; + using BucketInfoPtr = std::shared_ptr; + + explicit S3QueueOrderedFileMetadata( + const std::filesystem::path & zk_path_, + const std::string & path_, + FileStatusPtr file_status_, + BucketInfoPtr bucket_info_, + size_t buckets_num_, + size_t max_loading_retries_, + LoggerPtr log_); + + struct BucketHolder; + using BucketHolderPtr = std::shared_ptr; + + static BucketHolderPtr tryAcquireBucket( + const std::filesystem::path & zk_path, + const Bucket & bucket, + const Processor & processor); + + static S3QueueOrderedFileMetadata::Bucket getBucketForPath(const std::string & path, size_t buckets_num); + + static std::vector getMetadataPaths(size_t buckets_num); + + void setProcessedAtStartRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr & zk_client) override; + +private: + const size_t buckets_num; + const std::string zk_path; + const BucketInfoPtr bucket_info; + + std::pair setProcessingImpl() override; + void setProcessedImpl() override; + + bool getMaxProcessedFile( + NodeMetadata & result, + Coordination::Stat * stat, + const zkutil::ZooKeeperPtr & zk_client); + + bool getMaxProcessedFile( + NodeMetadata & result, + Coordination::Stat * stat, + const std::string & processed_node_path_, + const zkutil::ZooKeeperPtr & zk_client); + + void setProcessedRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr & zk_client, + const std::string & processed_node_path_, + bool ignore_if_exists); +}; + +struct S3QueueOrderedFileMetadata::BucketHolder +{ + BucketHolder( + const Bucket & bucket_, + int bucket_version_, + const std::string & bucket_lock_path_, + const std::string & bucket_lock_id_path_, + zkutil::ZooKeeperPtr zk_client_); + + ~BucketHolder(); + + Bucket getBucket() const { return bucket_info->bucket; } + BucketInfoPtr getBucketInfo() const { return bucket_info; } + + void release(); + +private: + BucketInfoPtr bucket_info; + const zkutil::ZooKeeperPtr zk_client; + bool released = false; +}; + +} diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index c26e973a1c0..4a92d99c411 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -30,8 +30,7 @@ class ASTStorage; M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ M(UInt32, s3queue_cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \ M(UInt32, s3queue_cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \ - M(UInt32, s3queue_total_shards_num, 1, "Value 0 means disabled", 0) \ - M(UInt32, s3queue_current_shard_num, 0, "", 0) \ + M(UInt32, s3queue_buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index c8aaece0711..b5b1a8dd992 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,10 +1,10 @@ #include "config.h" -#if USE_AWS_S3 #include #include #include #include +#include #include #include #include @@ -32,18 +32,16 @@ namespace ErrorCodes } StorageS3QueueSource::S3QueueObjectInfo::S3QueueObjectInfo( - const std::string & key_, - const ObjectMetadata & object_metadata_, - Metadata::ProcessingNodeHolderPtr processing_holder_) - : ObjectInfo(key_, object_metadata_) + const ObjectInfo & object_info, + Metadata::FileMetadataPtr processing_holder_) + : ObjectInfo(object_info.relative_path, object_info.metadata) , processing_holder(processing_holder_) { } StorageS3QueueSource::FileIterator::FileIterator( - std::shared_ptr metadata_, + std::shared_ptr metadata_, std::unique_ptr glob_iterator_, - size_t current_shard_, std::atomic & shutdown_called_, LoggerPtr logger_) : StorageObjectStorageSource::IIterator("S3QueueIterator") @@ -51,109 +49,7 @@ StorageS3QueueSource::FileIterator::FileIterator( , glob_iterator(std::move(glob_iterator_)) , shutdown_called(shutdown_called_) , log(logger_) - , sharded_processing(metadata->isShardedProcessing()) - , current_shard(current_shard_) { - if (sharded_processing) - { - for (const auto & id : metadata->getProcessingIdsForShard(current_shard)) - sharded_keys.emplace(id, std::deque{}); - } -} - -StorageS3QueueSource::ObjectInfoPtr StorageS3QueueSource::FileIterator::nextImpl(size_t processor) -{ - while (!shutdown_called) - { - ObjectInfoPtr val{nullptr}; - - { - std::unique_lock lk(sharded_keys_mutex, std::defer_lock); - if (sharded_processing) - { - /// To make sure order on keys in each shard in sharded_keys - /// we need to check sharded_keys and to next() under lock. - lk.lock(); - - if (auto it = sharded_keys.find(processor); it != sharded_keys.end()) - { - auto & keys = it->second; - if (!keys.empty()) - { - val = keys.front(); - keys.pop_front(); - chassert(processor == metadata->getProcessingIdForPath(val->relative_path)); - } - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Processing id {} does not exist (Expected ids: {})", - processor, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); - } - } - - if (!val) - { - val = glob_iterator->next(processor); - if (val && sharded_processing) - { - const auto processing_id_for_key = metadata->getProcessingIdForPath(val->relative_path); - if (processor != processing_id_for_key) - { - if (metadata->isProcessingIdBelongsToShard(processing_id_for_key, current_shard)) - { - LOG_TEST(log, "Putting key {} into queue of processor {} (total: {})", - val->relative_path, processing_id_for_key, sharded_keys.size()); - - if (auto it = sharded_keys.find(processing_id_for_key); it != sharded_keys.end()) - { - it->second.push_back(val); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Processing id {} does not exist (Expected ids: {})", - processing_id_for_key, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); - } - } - continue; - } - } - } - } - - if (!val) - return {}; - - if (shutdown_called) - { - LOG_TEST(log, "Shutdown was called, stopping file iterator"); - return {}; - } - - auto processing_holder = metadata->trySetFileAsProcessing(val->relative_path); - if (shutdown_called) - { - LOG_TEST(log, "Shutdown was called, stopping file iterator"); - return {}; - } - - LOG_TEST(log, "Checking if can process key {} for processing_id {}", val->relative_path, processor); - - if (processing_holder) - { - return std::make_shared(val->relative_path, val->metadata.value(), processing_holder); - } - else if (sharded_processing - && metadata->getFileStatus(val->relative_path)->state == S3QueueFilesMetadata::FileStatus::State::Processing) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "File {} is processing by someone else in sharded processing. " - "It is a bug", val->relative_path); - } - } - return {}; } size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() @@ -161,12 +57,242 @@ size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method estimateKeysCount is not implemented"); } +StorageS3QueueSource::ObjectInfoPtr StorageS3QueueSource::FileIterator::nextImpl(size_t processor) +{ + ObjectInfoPtr object_info; + S3QueueOrderedFileMetadata::BucketInfoPtr bucket_info; + + while (!shutdown_called) + { + if (metadata->useBucketsForProcessing()) + std::tie(object_info, bucket_info) = getNextKeyFromAcquiredBucket(processor); + else + object_info = glob_iterator->next(processor); + + if (!object_info) + return {}; + + if (shutdown_called) + { + LOG_TEST(log, "Shutdown was called, stopping file iterator"); + return {}; + } + + auto file_metadata = metadata->getFileMetadata(object_info->relative_path, bucket_info); + if (file_metadata->setProcessing()) + return std::make_shared(*object_info, file_metadata); + } + return {}; +} + +std::pair +StorageS3QueueSource::FileIterator::getNextKeyFromAcquiredBucket(size_t processor) +{ + /// We need this lock to maintain consistency between listing s3 directory + /// and getting/putting result into listed_keys_cache. + std::lock_guard lock(buckets_mutex); + + auto bucket_holder_it = bucket_holders.emplace(processor, nullptr).first; + auto current_processor = toString(processor); + + LOG_TEST( + log, "Current processor: {}, acquired bucket: {}", + processor, bucket_holder_it->second ? toString(bucket_holder_it->second->getBucket()) : "None"); + + while (true) + { + /// Each processing thread gets next path from glob_iterator->next() + /// and checks if corresponding bucket is already acquired by someone. + /// In case it is already acquired, they put the key into listed_keys_cache, + /// so that the thread who acquired the bucket will be able to see + /// those keys without the need to list s3 directory once again. + if (bucket_holder_it->second) + { + const auto bucket = bucket_holder_it->second->getBucket(); + auto it = listed_keys_cache.find(bucket); + if (it != listed_keys_cache.end()) + { + /// `bucket_keys` -- keys we iterated so far and which were not taken for processing. + /// `bucket_processor` -- processor id of the thread which has acquired the bucket. + auto & [bucket_keys, bucket_processor] = it->second; + + /// Check correctness just in case. + if (!bucket_processor.has_value()) + { + bucket_processor = current_processor; + } + else if (bucket_processor.value() != current_processor) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected current processor {} to be equal to {} for bucket {}", + current_processor, + bucket_processor.has_value() ? toString(bucket_processor.value()) : "None", + bucket); + } + + /// Take next key to process + if (!bucket_keys.empty()) + { + /// Take the key from the front, the order is important. + auto object_info = bucket_keys.front(); + bucket_keys.pop_front(); + + LOG_TEST(log, "Current bucket: {}, will process file: {}", + bucket, object_info->getFileName()); + + return std::pair{object_info, bucket_holder_it->second->getBucketInfo()}; + } + + LOG_TEST(log, "Cache of bucket {} is empty", bucket); + + /// No more keys in bucket, remove it from cache. + listed_keys_cache.erase(it); + } + else + { + LOG_TEST(log, "Cache of bucket {} is empty", bucket); + } + + if (iterator_finished) + { + /// Bucket is fully processed - release the bucket. + bucket_holder_it->second->release(); + bucket_holder_it->second.reset(); + } + } + /// If processing thread has already acquired some bucket + /// and while listing s3 directory gets a key which is in a different bucket, + /// it puts the key into listed_keys_cache to allow others to process it, + /// because one processing thread can acquire only one bucket at a time. + /// Once a thread is finished with its acquired bucket, it checks listed_keys_cache + /// to see if there are keys from buckets not acquired by anyone. + if (!bucket_holder_it->second) + { + for (auto it = listed_keys_cache.begin(); it != listed_keys_cache.end();) + { + auto & [bucket, bucket_info] = *it; + auto & [bucket_keys, bucket_processor] = bucket_info; + + LOG_TEST(log, "Bucket: {}, cached keys: {}, processor: {}", + bucket, bucket_keys.size(), bucket_processor.has_value() ? toString(bucket_processor.value()) : "None"); + + if (bucket_processor.has_value()) + { + LOG_TEST(log, "Bucket {} is already locked for processing by {} (keys: {})", + bucket, bucket_processor.value(), bucket_keys.size()); + ++it; + continue; + } + + if (bucket_keys.empty()) + { + /// No more keys in bucket, remove it from cache. + /// We still might add new keys to this bucket if !iterator_finished. + it = listed_keys_cache.erase(it); + continue; + } + + bucket_holder_it->second = metadata->tryAcquireBucket(bucket, current_processor); + if (!bucket_holder_it->second) + { + LOG_TEST(log, "Bucket {} is already locked for processing (keys: {})", + bucket, bucket_keys.size()); + ++it; + continue; + } + + bucket_processor = current_processor; + + /// Take the key from the front, the order is important. + auto object_info = bucket_keys.front(); + bucket_keys.pop_front(); + + LOG_TEST(log, "Acquired bucket: {}, will process file: {}", + bucket, object_info->getFileName()); + + return std::pair{object_info, bucket_holder_it->second->getBucketInfo()}; + } + } + + if (iterator_finished) + { + LOG_TEST(log, "Reached the end of file iterator and nothing left in keys cache"); + return {}; + } + + auto object_info = glob_iterator->next(processor); + if (object_info) + { + const auto bucket = metadata->getBucketForPath(object_info->relative_path); + auto & bucket_cache = listed_keys_cache[bucket]; + + LOG_TEST(log, "Found next file: {}, bucket: {}, current bucket: {}, cached_keys: {}", + object_info->getFileName(), bucket, + bucket_holder_it->second ? toString(bucket_holder_it->second->getBucket()) : "None", + bucket_cache.keys.size()); + + if (bucket_holder_it->second) + { + if (bucket_holder_it->second->getBucket() != bucket) + { + /// Acquired bucket differs from object's bucket, + /// put it into bucket's cache and continue. + bucket_cache.keys.emplace_back(object_info); + continue; + } + /// Bucket is already acquired, process the file. + return std::pair{object_info, bucket_holder_it->second->getBucketInfo()}; + } + else + { + bucket_holder_it->second = metadata->tryAcquireBucket(bucket, current_processor); + if (bucket_holder_it->second) + { + bucket_cache.processor = current_processor; + if (!bucket_cache.keys.empty()) + { + /// We have to maintain ordering between keys, + /// so if some keys are already in cache - start with them. + bucket_cache.keys.emplace_back(object_info); + object_info = bucket_cache.keys.front(); + bucket_cache.keys.pop_front(); + } + return std::pair{object_info, bucket_holder_it->second->getBucketInfo()}; + } + else + { + LOG_TEST(log, "Bucket {} is already locked for processing", bucket); + bucket_cache.keys.emplace_back(object_info); + continue; + } + } + } + else + { + if (bucket_holder_it->second) + { + bucket_holder_it->second->release(); + bucket_holder_it->second.reset(); + } + + LOG_TEST(log, "Reached the end of file iterator"); + iterator_finished = true; + + if (listed_keys_cache.empty()) + return {}; + else + continue; + } + } +} + StorageS3QueueSource::StorageS3QueueSource( String name_, + size_t processor_id_, const Block & header_, std::unique_ptr internal_source_, - std::shared_ptr files_metadata_, - size_t processing_id_, + std::shared_ptr files_metadata_, const S3QueueAction & action_, RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, @@ -179,8 +305,8 @@ StorageS3QueueSource::StorageS3QueueSource( : ISource(header_) , WithContext(context_) , name(std::move(name_)) + , processor_id(processor_id_) , action(action_) - , processing_id(processing_id_) , files_metadata(files_metadata_) , internal_source(std::move(internal_source_)) , requested_virtual_columns(requested_virtual_columns_) @@ -198,12 +324,12 @@ String StorageS3QueueSource::getName() const return name; } -void StorageS3QueueSource::lazyInitialize() +void StorageS3QueueSource::lazyInitialize(size_t processor) { if (initialized) return; - internal_source->lazyInitialize(processing_id); + internal_source->lazyInitialize(processor); reader = std::move(internal_source->reader); if (reader) reader_future = std::move(internal_source->reader_future); @@ -212,15 +338,16 @@ void StorageS3QueueSource::lazyInitialize() Chunk StorageS3QueueSource::generate() { - lazyInitialize(); + lazyInitialize(processor_id); while (true) { if (!reader) break; - const auto * key_with_info = dynamic_cast(&reader.getObjectInfo()); - auto file_status = key_with_info->processing_holder->getFileStatus(); + const auto * object_info = dynamic_cast(&reader.getObjectInfo()); + auto file_metadata = object_info->processing_holder; + auto file_status = file_metadata->getFileStatus(); if (isCancelled()) { @@ -230,12 +357,12 @@ Chunk StorageS3QueueSource::generate() { try { - files_metadata->setFileFailed(key_with_info->processing_holder, "Cancelled"); + file_metadata->setFailed("Cancelled"); } catch (...) { LOG_ERROR(log, "Failed to set file {} as failed: {}", - key_with_info->relative_path, getCurrentExceptionMessage(true)); + object_info->relative_path, getCurrentExceptionMessage(true)); } appendLogElement(reader.getObjectInfo().getPath(), *file_status, processed_rows_from_file, false); @@ -259,12 +386,12 @@ Chunk StorageS3QueueSource::generate() try { - files_metadata->setFileFailed(key_with_info->processing_holder, "Table is dropped"); + file_metadata->setFailed("Table is dropped"); } catch (...) { LOG_ERROR(log, "Failed to set file {} as failed: {}", - key_with_info->relative_path, getCurrentExceptionMessage(true)); + object_info->relative_path, getCurrentExceptionMessage(true)); } appendLogElement(path, *file_status, processed_rows_from_file, false); @@ -294,8 +421,14 @@ Chunk StorageS3QueueSource::generate() file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, path, reader.getObjectInfo().metadata->size_bytes); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = path, + .size = reader.getObjectInfo().metadata->size_bytes + }); + + return chunk; } } @@ -304,14 +437,14 @@ Chunk StorageS3QueueSource::generate() const auto message = getCurrentExceptionMessage(true); LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", path, message); - files_metadata->setFileFailed(key_with_info->processing_holder, message); + file_metadata->setFailed(message); appendLogElement(path, *file_status, processed_rows_from_file, false); throw; } - files_metadata->setFileProcessed(key_with_info->processing_holder); - applyActionAfterProcessing(path); + file_metadata->setProcessed(); + applyActionAfterProcessing(reader.getObjectInfo().relative_path); appendLogElement(path, *file_status, processed_rows_from_file, true); file_status.reset(); @@ -334,7 +467,7 @@ Chunk StorageS3QueueSource::generate() /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. internal_source->create_reader_pool->wait(); - reader_future = internal_source->createReaderAsync(processing_id); + reader_future = internal_source->createReaderAsync(processor_id); } return {}; @@ -357,7 +490,7 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) void StorageS3QueueSource::appendLogElement( const std::string & filename, - S3QueueFilesMetadata::FileStatus & file_status_, + S3QueueMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) { @@ -366,7 +499,6 @@ void StorageS3QueueSource::appendLogElement( S3QueueLogElement elem{}; { - std::lock_guard lock(file_status_.metadata_lock); elem = S3QueueLogElement { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), @@ -379,12 +511,10 @@ void StorageS3QueueSource::appendLogElement( .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), .processing_start_time = file_status_.processing_start_time, .processing_end_time = file_status_.processing_end_time, - .exception = file_status_.last_exception, + .exception = file_status_.getException(), }; } s3_queue_log->add(std::move(elem)); } } - -#endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 663577e055b..6e098f8cb63 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -1,10 +1,9 @@ #pragma once #include "config.h" -#if USE_AWS_S3 #include #include -#include +#include #include #include #include @@ -21,14 +20,13 @@ class StorageS3QueueSource : public ISource, WithContext { public: using Storage = StorageObjectStorage; - using ConfigurationPtr = Storage::ConfigurationPtr; using GlobIterator = StorageObjectStorageSource::GlobIterator; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; - using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; + using FileStatusPtr = S3QueueMetadata::FileStatusPtr; using ReaderHolder = StorageObjectStorageSource::ReaderHolder; - using Metadata = S3QueueFilesMetadata; + using Metadata = S3QueueMetadata; using ObjectInfo = StorageObjectStorageSource::ObjectInfo; using ObjectInfoPtr = std::shared_ptr; using ObjectInfos = std::vector; @@ -36,20 +34,18 @@ public: struct S3QueueObjectInfo : public ObjectInfo { S3QueueObjectInfo( - const std::string & key_, - const ObjectMetadata & object_metadata_, - Metadata::ProcessingNodeHolderPtr processing_holder_); + const ObjectInfo & object_info, + Metadata::FileMetadataPtr processing_holder_); - Metadata::ProcessingNodeHolderPtr processing_holder; + Metadata::FileMetadataPtr processing_holder; }; class FileIterator : public StorageObjectStorageSource::IIterator { public: FileIterator( - std::shared_ptr metadata_, + std::shared_ptr metadata_, std::unique_ptr glob_iterator_, - size_t current_shard_, std::atomic & shutdown_called_, LoggerPtr logger_); @@ -61,24 +57,35 @@ public: size_t estimatedKeysCount() override; private: - const std::shared_ptr metadata; + using Bucket = S3QueueMetadata::Bucket; + using Processor = S3QueueMetadata::Processor; + + const std::shared_ptr metadata; const std::unique_ptr glob_iterator; + std::atomic & shutdown_called; std::mutex mutex; LoggerPtr log; - const bool sharded_processing; - const size_t current_shard; - std::unordered_map> sharded_keys; - std::mutex sharded_keys_mutex; + std::mutex buckets_mutex; + struct ListedKeys + { + std::deque keys; + std::optional processor; + }; + std::unordered_map listed_keys_cache; + bool iterator_finished = false; + std::unordered_map bucket_holders; + + std::pair getNextKeyFromAcquiredBucket(size_t processor); }; StorageS3QueueSource( String name_, + size_t processor_id_, const Block & header_, std::unique_ptr internal_source_, - std::shared_ptr files_metadata_, - size_t processing_id_, + std::shared_ptr files_metadata_, const S3QueueAction & action_, RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, @@ -97,9 +104,9 @@ public: private: const String name; + const size_t processor_id; const S3QueueAction action; - const size_t processing_id; - const std::shared_ptr files_metadata; + const std::shared_ptr files_metadata; const std::shared_ptr internal_source; const NamesAndTypesList requested_virtual_columns; const std::atomic & shutdown_called; @@ -115,10 +122,11 @@ private: std::atomic initialized{false}; size_t processed_rows_from_file = 0; + S3QueueOrderedFileMetadata::BucketHolderPtr current_bucket_holder; + void applyActionAfterProcessing(const String & path); - void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); - void lazyInitialize(); + void appendLogElement(const std::string & filename, S3QueueMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); + void lazyInitialize(size_t processor); }; } -#endif diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index f0b7568ae7f..ecaa7ad57cc 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -1,12 +1,11 @@ #include -#if USE_AWS_S3 - #include #include #include #include #include +#include #include @@ -40,10 +39,10 @@ S3QueueTableMetadata::S3QueueTableMetadata( format_name = configuration.format; after_processing = engine_settings.after_processing.toString(); mode = engine_settings.mode.toString(); - s3queue_tracked_files_limit = engine_settings.s3queue_tracked_files_limit; - s3queue_tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec; - s3queue_total_shards_num = engine_settings.s3queue_total_shards_num; - s3queue_processing_threads_num = engine_settings.s3queue_processing_threads_num; + tracked_files_limit = engine_settings.s3queue_tracked_files_limit; + tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec; + buckets = engine_settings.s3queue_buckets; + processing_threads_num = engine_settings.s3queue_processing_threads_num; columns = storage_metadata.getColumns().toString(); } @@ -52,14 +51,15 @@ String S3QueueTableMetadata::toString() const Poco::JSON::Object json; json.set("after_processing", after_processing); json.set("mode", mode); - json.set("s3queue_tracked_files_limit", s3queue_tracked_files_limit); - json.set("s3queue_tracked_file_ttl_sec", s3queue_tracked_file_ttl_sec); - json.set("s3queue_total_shards_num", s3queue_total_shards_num); - json.set("s3queue_processing_threads_num", s3queue_processing_threads_num); + json.set("tracked_files_limit", tracked_files_limit); + json.set("tracked_file_ttl_sec", tracked_file_ttl_sec); + json.set("processing_threads_num", processing_threads_num); + json.set("buckets", buckets); json.set("format_name", format_name); json.set("columns", columns); + json.set("last_processed_file", last_processed_path); - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); return oss.str(); @@ -72,20 +72,34 @@ void S3QueueTableMetadata::read(const String & metadata_str) after_processing = json->getValue("after_processing"); mode = json->getValue("mode"); - s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); - s3queue_tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); + format_name = json->getValue("format_name"); columns = json->getValue("columns"); - if (json->has("s3queue_total_shards_num")) - s3queue_total_shards_num = json->getValue("s3queue_total_shards_num"); - else - s3queue_total_shards_num = 1; + /// Check with "s3queue_" prefix for compatibility. + { + if (json->has("s3queue_tracked_files_limit")) + tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); + if (json->has("s3queue_tracked_file_ttl_sec")) + tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); + if (json->has("s3queue_processing_threads_num")) + processing_threads_num = json->getValue("s3queue_processing_threads_num"); + } - if (json->has("s3queue_processing_threads_num")) - s3queue_processing_threads_num = json->getValue("s3queue_processing_threads_num"); - else - s3queue_processing_threads_num = 1; + if (json->has("tracked_files_limit")) + tracked_files_limit = json->getValue("tracked_files_limit"); + + if (json->has("tracked_file_ttl_sec")) + tracked_file_ttl_sec = json->getValue("tracked_file_ttl_sec"); + + if (json->has("last_processed_file")) + last_processed_path = json->getValue("last_processed_file"); + + if (json->has("processing_threads_num")) + processing_threads_num = json->getValue("processing_threads_num"); + + if (json->has("buckets")) + buckets = json->getValue("buckets"); } S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) @@ -95,6 +109,11 @@ S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) return metadata; } +void S3QueueTableMetadata::checkEquals(const S3QueueTableMetadata & from_zk) const +{ + checkImmutableFieldsEquals(from_zk); +} + void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const { if (after_processing != from_zk.after_processing) @@ -113,21 +132,21 @@ void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata from_zk.mode, mode); - if (s3queue_tracked_files_limit != from_zk.s3queue_tracked_files_limit) + if (tracked_files_limit != from_zk.tracked_files_limit) throw Exception( ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in max set size. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_tracked_files_limit, - s3queue_tracked_files_limit); + from_zk.tracked_files_limit, + tracked_files_limit); - if (s3queue_tracked_file_ttl_sec != from_zk.s3queue_tracked_file_ttl_sec) + if (tracked_file_ttl_sec != from_zk.tracked_file_ttl_sec) throw Exception( ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in max set age. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_tracked_file_ttl_sec, - s3queue_tracked_file_ttl_sec); + from_zk.tracked_file_ttl_sec, + tracked_file_ttl_sec); if (format_name != from_zk.format_name) throw Exception( @@ -137,34 +156,97 @@ void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata from_zk.format_name, format_name); + if (last_processed_path != from_zk.last_processed_path) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in last processed path. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.last_processed_path, + last_processed_path); + if (modeFromString(mode) == S3QueueMode::ORDERED) { - if (s3queue_processing_threads_num != from_zk.s3queue_processing_threads_num) + if (buckets != from_zk.buckets) { throw Exception( ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in s3queue_processing_threads_num setting. " + "Existing table metadata in ZooKeeper differs in s3queue_buckets setting. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_processing_threads_num, - s3queue_processing_threads_num); + from_zk.buckets, buckets); } - if (s3queue_total_shards_num != from_zk.s3queue_total_shards_num) + + if (S3QueueMetadata::getBucketsNum(*this) != S3QueueMetadata::getBucketsNum(from_zk)) { throw Exception( ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in s3queue_total_shards_num setting. " + "Existing table metadata in ZooKeeper differs in processing buckets. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_total_shards_num, - s3queue_total_shards_num); + S3QueueMetadata::getBucketsNum(*this), S3QueueMetadata::getBucketsNum(from_zk)); } } } -void S3QueueTableMetadata::checkEquals(const S3QueueTableMetadata & from_zk) const +void S3QueueTableMetadata::checkEquals(const S3QueueSettings & current, const S3QueueSettings & expected) { - checkImmutableFieldsEquals(from_zk); -} + if (current.after_processing != expected.after_processing) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs " + "in action after processing. Stored in ZooKeeper: {}, local: {}", + expected.after_processing.toString(), + current.after_processing.toString()); -} + if (current.mode != expected.mode) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in engine mode. " + "Stored in ZooKeeper: {}, local: {}", + expected.mode.toString(), + current.mode.toString()); -#endif + if (current.s3queue_tracked_files_limit != expected.s3queue_tracked_files_limit) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in max set size. " + "Stored in ZooKeeper: {}, local: {}", + expected.s3queue_tracked_files_limit, + current.s3queue_tracked_files_limit); + + if (current.s3queue_tracked_file_ttl_sec != expected.s3queue_tracked_file_ttl_sec) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in max set age. " + "Stored in ZooKeeper: {}, local: {}", + expected.s3queue_tracked_file_ttl_sec, + current.s3queue_tracked_file_ttl_sec); + + if (current.s3queue_last_processed_path.value != expected.s3queue_last_processed_path.value) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in last_processed_path. " + "Stored in ZooKeeper: {}, local: {}", + expected.s3queue_last_processed_path.value, + current.s3queue_last_processed_path.value); + + if (current.mode == S3QueueMode::ORDERED) + { + if (current.s3queue_buckets != expected.s3queue_buckets) + { + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in s3queue_buckets setting. " + "Stored in ZooKeeper: {}, local: {}", + expected.s3queue_buckets, current.s3queue_buckets); + } + + if (S3QueueMetadata::getBucketsNum(current) != S3QueueMetadata::getBucketsNum(expected)) + { + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in processing buckets. " + "Stored in ZooKeeper: {}, local: {}", + S3QueueMetadata::getBucketsNum(current), S3QueueMetadata::getBucketsNum(expected)); + } + } +} +} diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index bb8f8ccf2c4..d53b60570ae 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -1,7 +1,5 @@ #pragma once -#if USE_AWS_S3 - #include #include #include @@ -22,10 +20,11 @@ struct S3QueueTableMetadata String columns; String after_processing; String mode; - UInt64 s3queue_tracked_files_limit = 0; - UInt64 s3queue_tracked_file_ttl_sec = 0; - UInt64 s3queue_total_shards_num = 1; - UInt64 s3queue_processing_threads_num = 1; + UInt64 tracked_files_limit = 0; + UInt64 tracked_file_ttl_sec = 0; + UInt64 buckets = 0; + UInt64 processing_threads_num = 1; + String last_processed_path; S3QueueTableMetadata() = default; S3QueueTableMetadata( @@ -39,6 +38,7 @@ struct S3QueueTableMetadata String toString() const; void checkEquals(const S3QueueTableMetadata & from_zk) const; + static void checkEquals(const S3QueueSettings & current, const S3QueueSettings & expected); private: void checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const; @@ -46,5 +46,3 @@ private: } - -#endif diff --git a/src/Storages/S3Queue/S3QueueUnorderedFileMetadata.cpp b/src/Storages/S3Queue/S3QueueUnorderedFileMetadata.cpp new file mode 100644 index 00000000000..c61e9557fc2 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueUnorderedFileMetadata.cpp @@ -0,0 +1,155 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + zkutil::ZooKeeperPtr getZooKeeper() + { + return Context::getGlobalContextInstance()->getZooKeeper(); + } +} + +S3QueueUnorderedFileMetadata::S3QueueUnorderedFileMetadata( + const std::filesystem::path & zk_path, + const std::string & path_, + FileStatusPtr file_status_, + size_t max_loading_retries_, + LoggerPtr log_) + : S3QueueIFileMetadata( + path_, + /* processing_node_path */zk_path / "processing" / getNodeName(path_), + /* processed_node_path */zk_path / "processed" / getNodeName(path_), + /* failed_node_path */zk_path / "failed" / getNodeName(path_), + file_status_, + max_loading_retries_, + log_) +{ +} + +std::pair S3QueueUnorderedFileMetadata::setProcessingImpl() +{ + /// In one zookeeper transaction do the following: + enum RequestType + { + /// node_name is not within processed persistent nodes + PROCESSED_PATH_DOESNT_EXIST = 0, + /// node_name is not within failed persistent nodes + FAILED_PATH_DOESNT_EXIST = 2, + /// node_name ephemeral processing node was successfully created + CREATED_PROCESSING_PATH = 4, + /// update processing id + SET_PROCESSING_ID = 6, + }; + + const auto zk_client = getZooKeeper(); + processing_id = node_metadata.processing_id = getRandomASCIIString(10); + auto processor_info = getProcessorInfo(processing_id.value()); + + Coordination::Requests requests; + requests.push_back(zkutil::makeCreateRequest(processed_node_path, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(processed_node_path, -1)); + requests.push_back(zkutil::makeCreateRequest(failed_node_path, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(failed_node_path, -1)); + requests.push_back(zkutil::makeCreateRequest(processing_node_path, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); + + requests.push_back( + zkutil::makeCreateRequest( + processing_node_id_path, processor_info, zkutil::CreateMode::Persistent, /* ignore_if_exists */true)); + requests.push_back(zkutil::makeSetRequest(processing_node_id_path, processor_info, -1)); + + Coordination::Responses responses; + const auto code = zk_client->tryMulti(requests, responses); + auto is_request_failed = [&](RequestType type) { return responses[type]->error != Coordination::Error::ZOK; }; + + if (code == Coordination::Error::ZOK) + { + const auto * set_response = dynamic_cast(responses[SET_PROCESSING_ID].get()); + processing_id_version = set_response->stat.version; + return std::pair{true, FileStatus::State::None}; + } + + if (is_request_failed(PROCESSED_PATH_DOESNT_EXIST)) + return {false, FileStatus::State::Processed}; + + if (is_request_failed(FAILED_PATH_DOESNT_EXIST)) + return {false, FileStatus::State::Failed}; + + if (is_request_failed(CREATED_PROCESSING_PATH)) + return {false, FileStatus::State::Processing}; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", magic_enum::enum_name(code)); +} + +void S3QueueUnorderedFileMetadata::setProcessedAtStartRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr &) +{ + requests.push_back( + zkutil::makeCreateRequest( + processed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); +} + +void S3QueueUnorderedFileMetadata::setProcessedImpl() +{ + /// In one zookeeper transaction do the following: + enum RequestType + { + SET_MAX_PROCESSED_PATH = 0, + CHECK_PROCESSING_ID_PATH = 1, /// Optional. + REMOVE_PROCESSING_ID_PATH = 2, /// Optional. + REMOVE_PROCESSING_PATH = 3, /// Optional. + }; + + const auto zk_client = getZooKeeper(); + std::string failure_reason; + + Coordination::Requests requests; + requests.push_back( + zkutil::makeCreateRequest( + processed_node_path, node_metadata.toString(), zkutil::CreateMode::Persistent)); + + if (processing_id_version.has_value()) + { + requests.push_back(zkutil::makeCheckRequest(processing_node_id_path, processing_id_version.value())); + requests.push_back(zkutil::makeRemoveRequest(processing_node_id_path, processing_id_version.value())); + requests.push_back(zkutil::makeRemoveRequest(processing_node_path, -1)); + } + + Coordination::Responses responses; + auto is_request_failed = [&](RequestType type) { return responses[type]->error != Coordination::Error::ZOK; }; + + const auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + { + if (max_loading_retries) + zk_client->tryRemove(failed_node_path + ".retriable", -1); + + LOG_TRACE(log, "Moved file `{}` to processed (node path: {})", path, processed_node_path); + return; + } + + if (Coordination::isHardwareError(code)) + failure_reason = "Lost connection to keeper"; + else if (is_request_failed(SET_MAX_PROCESSED_PATH)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot create a persistent node in /processed since it already exists"); + else if (is_request_failed(CHECK_PROCESSING_ID_PATH)) + failure_reason = "Version of processing id node changed"; + else if (is_request_failed(REMOVE_PROCESSING_PATH)) + failure_reason = "Failed to remove processing path"; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", code); + + LOG_WARNING(log, "Cannot set file {} as processed: {}. Reason: {}", path, code, failure_reason); +} + +} diff --git a/src/Storages/S3Queue/S3QueueUnorderedFileMetadata.h b/src/Storages/S3Queue/S3QueueUnorderedFileMetadata.h new file mode 100644 index 00000000000..24c2765bf3a --- /dev/null +++ b/src/Storages/S3Queue/S3QueueUnorderedFileMetadata.h @@ -0,0 +1,32 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class S3QueueUnorderedFileMetadata : public S3QueueIFileMetadata +{ +public: + using Bucket = size_t; + + explicit S3QueueUnorderedFileMetadata( + const std::filesystem::path & zk_path, + const std::string & path_, + FileStatusPtr file_status_, + size_t max_loading_retries_, + LoggerPtr log_); + + static std::vector getMetadataPaths() { return {"processed", "failed", "processing"}; } + + void setProcessedAtStartRequests( + Coordination::Requests & requests, + const zkutil::ZooKeeperPtr & zk_client) override; + +private: + std::pair setProcessingImpl() override; + void setProcessedImpl() override; +}; + +} diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index f8eb288921c..afb75a21b21 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,7 +1,6 @@ #include #include "config.h" -#if USE_AWS_S3 #include #include #include @@ -18,7 +17,7 @@ #include #include #include -#include +#include #include #include #include @@ -48,8 +47,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int S3_ERROR; extern const int QUERY_NOT_ALLOWED; - extern const int REPLICA_ALREADY_EXISTS; - extern const int INCOMPATIBLE_COLUMNS; } namespace @@ -74,8 +71,14 @@ namespace return zkutil::extractZooKeeperPath(result_zk_path, true); } - void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings) + void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings, bool is_attach) { + if (!is_attach && !s3queue_settings.mode.changed) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `mode` (Unordered/Ordered) is not specified, but is required."); + } + /// In case !is_attach, we leave Ordered mode as default for compatibility. + if (!s3queue_settings.s3queue_processing_threads_num) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `s3queue_processing_threads_num` cannot be set to zero"); @@ -104,13 +107,12 @@ StorageS3Queue::StorageS3Queue( const String & comment, ContextPtr context_, std::optional format_settings_, - ASTStorage * engine_args, + ASTStorage * /* engine_args */, LoadingStrictnessLevel mode) : IStorage(table_id_) , WithContext(context_) , s3queue_settings(std::move(s3queue_settings_)) , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) - , after_processing(s3queue_settings->after_processing) , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) @@ -129,15 +131,7 @@ StorageS3Queue::StorageS3Queue( throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } - if (mode == LoadingStrictnessLevel::CREATE - && !context_->getSettingsRef().s3queue_allow_experimental_sharded_mode - && s3queue_settings->mode == S3QueueMode::ORDERED - && (s3queue_settings->s3queue_total_shards_num > 1 || s3queue_settings->s3queue_processing_threads_num > 1)) - { - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue sharded mode is not allowed. To enable use `s3queue_allow_experimental_sharded_mode`"); - } - - checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef()); + checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef(), mode > LoadingStrictnessLevel::CREATE); object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); FormatFactory::instance().checkFormatName(configuration->format); @@ -157,28 +151,18 @@ StorageS3Queue::StorageS3Queue( LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); - createOrCheckMetadata(storage_metadata); - /// Get metadata manager from S3QueueMetadataFactory, /// it will increase the ref count for the metadata object. /// The ref count is decreased when StorageS3Queue::drop() method is called. files_metadata = S3QueueMetadataFactory::instance().getOrCreate(zk_path, *s3queue_settings); - - if (files_metadata->isShardedProcessing()) + try { - if (!s3queue_settings->s3queue_current_shard_num.changed) - { - s3queue_settings->s3queue_current_shard_num = static_cast(files_metadata->registerNewShard()); - engine_args->settings->changes.setSetting("s3queue_current_shard_num", s3queue_settings->s3queue_current_shard_num.value); - } - else if (!files_metadata->isShardRegistered(s3queue_settings->s3queue_current_shard_num)) - { - files_metadata->registerNewShard(s3queue_settings->s3queue_current_shard_num); - } + files_metadata->initialize(configuration_, storage_metadata); } - if (s3queue_settings->mode == S3QueueMode::ORDERED && !s3queue_settings->s3queue_last_processed_path.value.empty()) + catch (...) { - files_metadata->setFileProcessed(s3queue_settings->s3queue_last_processed_path.value, s3queue_settings->s3queue_current_shard_num); + S3QueueMetadataFactory::instance().remove(zk_path); + throw; } } @@ -201,14 +185,7 @@ void StorageS3Queue::shutdown(bool is_drop) if (files_metadata) { - files_metadata->deactivateCleanupTask(); - - if (is_drop && files_metadata->isShardedProcessing()) - { - files_metadata->unregisterShard(s3queue_settings->s3queue_current_shard_num); - LOG_TRACE(log, "Unregistered shard {} from zookeeper", s3queue_settings->s3queue_current_shard_num); - } - + files_metadata->shutdown(); files_metadata.reset(); } LOG_TRACE(log, "Shut down storage"); @@ -328,9 +305,9 @@ void ReadFromS3Queue::initializePipeline(QueryPipelineBuilder & pipeline, const createIterator(nullptr); for (size_t i = 0; i < adjusted_num_streams; ++i) pipes.emplace_back(storage->createSource( + i, info, iterator, - storage->files_metadata->getIdForProcessingThread(i, storage->s3queue_settings->s3queue_current_shard_num), max_block_size, context)); auto pipe = Pipe::unitePipes(std::move(pipes)); @@ -344,9 +321,9 @@ void ReadFromS3Queue::initializePipeline(QueryPipelineBuilder & pipeline, const } std::shared_ptr StorageS3Queue::createSource( + size_t processor_id, const ReadFromFormatInfo & info, std::shared_ptr file_iterator, - size_t processing_id, size_t max_block_size, ContextPtr local_context) { @@ -368,9 +345,20 @@ std::shared_ptr StorageS3Queue::createSource( }; auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr; return std::make_shared( - getName(), info.source_header, std::move(internal_source), - files_metadata, processing_id, after_processing, file_deleter, info.requested_virtual_columns, - local_context, shutdown_called, table_is_being_dropped, s3_queue_log, getStorageID(), log); + getName(), + processor_id, + info.source_header, + std::move(internal_source), + files_metadata, + s3queue_settings->after_processing, + file_deleter, + info.requested_virtual_columns, + local_context, + shutdown_called, + table_is_being_dropped, + s3_queue_log, + getStorageID(), + log); } bool StorageS3Queue::hasDependencies(const StorageID & table_id) @@ -471,10 +459,7 @@ bool StorageS3Queue::streamToViews() pipes.reserve(s3queue_settings->s3queue_processing_threads_num); for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource( - read_from_format_info, file_iterator, files_metadata->getIdForProcessingThread(i, s3queue_settings->s3queue_current_shard_num), - DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); - + auto source = createSource(i, read_from_format_info, file_iterator, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); @@ -497,88 +482,16 @@ zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const return getContext()->getZooKeeper(); } -void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata) -{ - auto zookeeper = getZooKeeper(); - zookeeper->createAncestors(zk_path); - - for (size_t i = 0; i < 1000; ++i) - { - Coordination::Requests requests; - if (zookeeper->exists(zk_path / "metadata")) - { - checkTableStructure(zk_path, storage_metadata); - } - else - { - std::string metadata = S3QueueTableMetadata(*configuration, *s3queue_settings, storage_metadata).toString(); - requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processed", "", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(zk_path / "failed", "", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processing", "", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(zk_path / "metadata", metadata, zkutil::CreateMode::Persistent)); - } - - if (!requests.empty()) - { - Coordination::Responses responses; - auto code = zookeeper->tryMulti(requests, responses); - if (code == Coordination::Error::ZNODEEXISTS) - { - LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path.string()); - continue; - } - else if (code != Coordination::Error::ZOK) - { - zkutil::KeeperMultiException::check(code, requests, responses); - } - } - - return; - } - - throw Exception( - ErrorCodes::REPLICA_ALREADY_EXISTS, - "Cannot create table, because it is created concurrently every time or because " - "of wrong zk_path or because of logical error"); -} - - -void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata) -{ - // Verify that list of columns and table settings match those specified in ZK (/metadata). - // If not, throw an exception. - - auto zookeeper = getZooKeeper(); - String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata"); - auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); - - S3QueueTableMetadata old_metadata(*configuration, *s3queue_settings, storage_metadata); - old_metadata.checkEquals(metadata_from_zk); - - auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); - const ColumnsDescription & old_columns = storage_metadata.getColumns(); - if (columns_from_zk != old_columns) - { - throw Exception( - ErrorCodes::INCOMPATIBLE_COLUMNS, - "Table columns structure in ZooKeeper is different from local table structure. Local columns:\n" - "{}\nZookeeper columns:\n{}", - old_columns.toString(), - columns_from_zk.toString()); - } -} - std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) { auto settings = configuration->getQuerySettings(local_context); auto glob_iterator = std::make_unique( object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match); - return std::make_shared( - files_metadata, std::move(glob_iterator), s3queue_settings->s3queue_current_shard_num, shutdown_called, log); + return std::make_shared(files_metadata, std::move(glob_iterator), shutdown_called, log); } +#if USE_AWS_S3 void registerStorageS3Queue(StorageFactory & factory) { factory.registerStorage( @@ -645,8 +558,6 @@ void registerStorageS3Queue(StorageFactory & factory) .source_access_type = AccessType::S3, }); } +#endif } - - -#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 83b7bc6667b..ef83a1ccc25 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -1,7 +1,6 @@ #pragma once #include "config.h" -#if USE_AWS_S3 #include #include #include @@ -16,7 +15,7 @@ namespace DB { -class S3QueueFilesMetadata; +class S3QueueMetadata; class StorageS3Queue : public IStorage, WithContext { @@ -59,9 +58,8 @@ private: const std::unique_ptr s3queue_settings; const fs::path zk_path; - const S3QueueAction after_processing; - std::shared_ptr files_metadata; + std::shared_ptr files_metadata; ConfigurationPtr configuration; ObjectStoragePtr object_storage; @@ -86,20 +84,15 @@ private: std::shared_ptr createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate); std::shared_ptr createSource( + size_t processor_id, const ReadFromFormatInfo & info, std::shared_ptr file_iterator, - size_t processing_id, size_t max_block_size, ContextPtr local_context); bool hasDependencies(const StorageID & table_id); bool streamToViews(); void threadFunc(); - - void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); - void checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata); }; } - -#endif diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 11e2a2fc5e7..52b6674c93d 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -208,19 +208,9 @@ struct SelectQueryInfo bool need_aggregate = false; PrewhereInfoPtr prewhere_info; - /// Generated by pre-run optimization with StorageDummy. - /// Currently it's used to support StorageMerge PREWHERE optimization. - PrewhereInfoPtr optimized_prewhere_info; - /// If query has aggregate functions bool has_aggregates = false; - /// If query has any filter and no arrayJoin before filter. Used by skipping FINAL - /// Skipping FINAL algorithm will output the original chunk and a column indices of - /// selected rows. If query has filter and doesn't have array join before any filter, - /// we can merge the indices with the first filter in FilterTransform later. - bool has_filters_and_no_array_join_before_filter = false; - ClusterPtr getCluster() const { return !optimized_cluster ? cluster : optimized_cluster; } bool settings_limit_offset_done = false; diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp new file mode 100644 index 00000000000..757136fdf42 --- /dev/null +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -0,0 +1,201 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String part_name, ColumnStatisticsPtr stats) +{ + if (part_statistics.contains(part_name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "part {} has been added in column {}", part_name, stats->columnName()); + part_statistics[part_name] = stats; +} + +Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(Float64 val, Float64 rows) const +{ + if (part_statistics.empty()) + return default_normal_cond_factor * rows; + Float64 result = 0; + Float64 part_rows = 0; + for (const auto & [key, estimator] : part_statistics) + { + result += estimator->estimateLess(val); + part_rows += estimator->rowCount(); + } + return result * rows / part_rows; +} + +Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreater(Float64 val, Float64 rows) const +{ + return rows - estimateLess(val, rows); +} + +Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(Float64 val, Float64 rows) const +{ + if (part_statistics.empty()) + { + if (val < - threshold || val > threshold) + return default_normal_cond_factor * rows; + else + return default_good_cond_factor * rows; + } + Float64 result = 0; + Float64 partial_cnt = 0; + for (const auto & [key, estimator] : part_statistics) + { + result += estimator->estimateEqual(val); + partial_cnt += estimator->rowCount(); + } + return result * rows / partial_cnt; +} + +/// second return value represents how many columns in the node. +static std::pair tryToExtractSingleColumn(const RPNBuilderTreeNode & node) +{ + if (node.isConstant()) + { + return {}; + } + + if (!node.isFunction()) + { + auto column_name = node.getColumnName(); + return {column_name, 1}; + } + + auto function_node = node.toFunctionNode(); + size_t arguments_size = function_node.getArgumentsSize(); + std::pair result; + for (size_t i = 0; i < arguments_size; ++i) + { + auto function_argument = function_node.getArgumentAt(i); + auto subresult = tryToExtractSingleColumn(function_argument); + if (subresult.second == 0) /// the subnode contains 0 column + continue; + else if (subresult.second > 1) /// the subnode contains more than 1 column + return subresult; + else if (result.second == 0 || result.first == subresult.first) /// subnodes contain same column. + result = subresult; + else + return {"", 2}; + } + return result; +} + +std::pair ConditionSelectivityEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const +{ + if (!node.isFunction()) + return {}; + + auto function_node = node.toFunctionNode(); + if (function_node.getArgumentsSize() != 2) + return {}; + + String function_name = function_node.getFunctionName(); + + auto lhs_argument = function_node.getArgumentAt(0); + auto rhs_argument = function_node.getArgumentAt(1); + + auto lhs_argument_column_name = lhs_argument.getColumnName(); + auto rhs_argument_column_name = rhs_argument.getColumnName(); + + bool lhs_argument_is_column = column_name == (lhs_argument_column_name); + bool rhs_argument_is_column = column_name == (rhs_argument_column_name); + + bool lhs_argument_is_constant = lhs_argument.isConstant(); + bool rhs_argument_is_constant = rhs_argument.isConstant(); + + RPNBuilderTreeNode * constant_node = nullptr; + + if (lhs_argument_is_column && rhs_argument_is_constant) + constant_node = &rhs_argument; + else if (lhs_argument_is_constant && rhs_argument_is_column) + constant_node = &lhs_argument; + else + return {}; + + Field output_value; + DataTypePtr output_type; + if (!constant_node->tryGetConstant(output_value, output_type)) + return {}; + + const auto type = output_value.getType(); + Float64 value; + if (type == Field::Types::Int64) + value = output_value.get(); + else if (type == Field::Types::UInt64) + value = output_value.get(); + else if (type == Field::Types::Float64) + value = output_value.get(); + else + return {}; + return std::make_pair(function_name, value); +} + +Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode & node) const +{ + auto result = tryToExtractSingleColumn(node); + if (result.second != 1) + { + return default_unknown_cond_factor; + } + String col = result.first; + auto it = column_estimators.find(col); + + /// If there the estimator of the column is not found or there are no data at all, + /// we use dummy estimation. + bool dummy = total_rows == 0; + ColumnSelectivityEstimator estimator; + if (it != column_estimators.end()) + { + estimator = it->second; + } + else + { + dummy = true; + } + auto [op, val] = extractBinaryOp(node, col); + if (op == "equals") + { + if (dummy) + { + if (val < - threshold || 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; +} + +void ConditionSelectivityEstimator::merge(String part_name, UInt64 part_rows, ColumnStatisticsPtr column_stat) +{ + if (!part_names.contains(part_name)) + { + total_rows += part_rows; + part_names.insert(part_name); + } + if (column_stat != nullptr) + column_estimators[column_stat->columnName()].merge(part_name, column_stat); +} + +} diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h new file mode 100644 index 00000000000..f0599742276 --- /dev/null +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -0,0 +1,50 @@ +#pragma once + +#include + +namespace DB +{ + +class RPNBuilderTreeNode; + +/// It estimates the selectivity of a condition. +class ConditionSelectivityEstimator +{ +private: + friend class ColumnStatistics; + struct ColumnSelectivityEstimator + { + /// We store the part_name and part_statistics. + /// then simply get selectivity for every part_statistics and combine them. + std::map part_statistics; + + void merge(String part_name, ColumnStatisticsPtr stats); + + Float64 estimateLess(Float64 val, Float64 rows) const; + + Float64 estimateGreater(Float64 val, Float64 rows) const; + + Float64 estimateEqual(Float64 val, Float64 rows) 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; + + UInt64 total_rows = 0; + std::set part_names; + std::map column_estimators; + std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; + +public: + /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... + /// Right now we only support simple condition like col = val / col < val + Float64 estimateRowCount(const RPNBuilderTreeNode & node) const; + + void merge(String part_name, UInt64 part_rows, ColumnStatisticsPtr column_stat); +}; + +} diff --git a/src/Storages/Statistics/Estimator.cpp b/src/Storages/Statistics/Estimator.cpp deleted file mode 100644 index e272014c1c2..00000000000 --- a/src/Storages/Statistics/Estimator.cpp +++ /dev/null @@ -1,137 +0,0 @@ -#include -#include - -namespace DB -{ - -/// second return value represents how many columns in the node. -static std::pair tryToExtractSingleColumn(const RPNBuilderTreeNode & node) -{ - if (node.isConstant()) - { - return {}; - } - - if (!node.isFunction()) - { - auto column_name = node.getColumnName(); - return {column_name, 1}; - } - - auto function_node = node.toFunctionNode(); - size_t arguments_size = function_node.getArgumentsSize(); - std::pair result; - for (size_t i = 0; i < arguments_size; ++i) - { - auto function_argument = function_node.getArgumentAt(i); - auto subresult = tryToExtractSingleColumn(function_argument); - if (subresult.second == 0) /// the subnode contains 0 column - continue; - else if (subresult.second > 1) /// the subnode contains more than 1 column - return subresult; - else if (result.second == 0 || result.first == subresult.first) /// subnodes contain same column. - result = subresult; - else - return {"", 2}; - } - return result; -} - -std::pair ConditionEstimator::extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const -{ - if (!node.isFunction()) - return {}; - - auto function_node = node.toFunctionNode(); - if (function_node.getArgumentsSize() != 2) - return {}; - - std::string function_name = function_node.getFunctionName(); - - auto lhs_argument = function_node.getArgumentAt(0); - auto rhs_argument = function_node.getArgumentAt(1); - - auto lhs_argument_column_name = lhs_argument.getColumnName(); - auto rhs_argument_column_name = rhs_argument.getColumnName(); - - bool lhs_argument_is_column = column_name == (lhs_argument_column_name); - bool rhs_argument_is_column = column_name == (rhs_argument_column_name); - - bool lhs_argument_is_constant = lhs_argument.isConstant(); - bool rhs_argument_is_constant = rhs_argument.isConstant(); - - RPNBuilderTreeNode * constant_node = nullptr; - - if (lhs_argument_is_column && rhs_argument_is_constant) - constant_node = &rhs_argument; - else if (lhs_argument_is_constant && rhs_argument_is_column) - constant_node = &lhs_argument; - else - return {}; - - Field output_value; - DataTypePtr output_type; - if (!constant_node->tryGetConstant(output_value, output_type)) - return {}; - - const auto type = output_value.getType(); - Float64 value; - if (type == Field::Types::Int64) - value = output_value.get(); - else if (type == Field::Types::UInt64) - value = output_value.get(); - else if (type == Field::Types::Float64) - value = output_value.get(); - else - return {}; - return std::make_pair(function_name, value); -} - -Float64 ConditionEstimator::estimateSelectivity(const RPNBuilderTreeNode & node) const -{ - auto result = tryToExtractSingleColumn(node); - if (result.second != 1) - { - return default_unknown_cond_factor; - } - String col = result.first; - auto it = column_estimators.find(col); - - /// If there the estimator of the column is not found or there are no data at all, - /// we use dummy estimation. - bool dummy = total_count == 0; - ColumnEstimator estimator; - if (it != column_estimators.end()) - { - estimator = it->second; - } - else - { - dummy = true; - } - auto [op, val] = extractBinaryOp(node, col); - if (op == "equals") - { - if (val < -threshold || val > threshold) - return default_normal_cond_factor; - else - return default_good_cond_factor; - } - else if (op == "less" || op == "lessThan") - { - if (dummy) - return default_normal_cond_factor; - return estimator.estimateLess(val) / total_count; - } - else if (op == "greater" || op == "greaterThan") - { - if (dummy) - return default_normal_cond_factor; - return estimator.estimateGreater(val) / total_count; - } - else - return default_unknown_cond_factor; -} - - -} diff --git a/src/Storages/Statistics/Estimator.h b/src/Storages/Statistics/Estimator.h deleted file mode 100644 index 903bb57eb80..00000000000 --- a/src/Storages/Statistics/Estimator.h +++ /dev/null @@ -1,111 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class RPNBuilderTreeNode; - -/// It estimates the selectivity of a condition. -class ConditionEstimator -{ -private: - - 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; - - UInt64 total_count = 0; - - /// Minimum estimator for values in a part. It can contains multiple types of statistics. - /// But right now we only have tdigest; - struct PartColumnEstimator - { - UInt64 part_count = 0; - - std::shared_ptr tdigest; - - void merge(StatisticPtr statistic) - { - UInt64 cur_part_count = statistic->count(); - if (part_count == 0) - part_count = cur_part_count; - - if (typeid_cast(statistic.get())) - { - tdigest = std::static_pointer_cast(statistic); - } - } - - Float64 estimateLess(Float64 val) const - { - if (tdigest != nullptr) - return tdigest -> estimateLess(val); - return part_count * default_normal_cond_factor; - } - - Float64 estimateGreator(Float64 val) const - { - if (tdigest != nullptr) - return part_count - tdigest -> estimateLess(val); - return part_count * default_normal_cond_factor; - } - }; - - /// An estimator for a column consists of several PartColumnEstimator. - /// We simply get selectivity for every part estimator and combine the result. - struct ColumnEstimator - { - std::map estimators; - - void merge(std::string part_name, StatisticPtr statistic) - { - estimators[part_name].merge(statistic); - } - - Float64 estimateLess(Float64 val) const - { - if (estimators.empty()) - return default_normal_cond_factor; - Float64 result = 0; - for (const auto & [key, estimator] : estimators) - result += estimator.estimateLess(val); - return result; - } - - Float64 estimateGreater(Float64 val) const - { - if (estimators.empty()) - return default_normal_cond_factor; - Float64 result = 0; - for (const auto & [key, estimator] : estimators) - result += estimator.estimateGreator(val); - return result; - } - }; - - std::map column_estimators; - /// std::optional extractSingleColumn(const RPNBuilderTreeNode & node) const; - std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const std::string & column_name) const; - -public: - ConditionEstimator() = default; - - /// TODO: Support the condition consists of CNF/DNF like (cond1 and cond2) or (cond3) ... - /// Right now we only support simple condition like col = val / col < val - Float64 estimateSelectivity(const RPNBuilderTreeNode & node) const; - - void merge(std::string part_name, UInt64 part_count, StatisticPtr statistic) - { - total_count += part_count; - if (statistic != nullptr) - column_estimators[statistic->columnName()].merge(part_name, statistic); - } -}; - - -} diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 6619eac19dc..fed0bd61c03 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,11 +1,14 @@ #include #include -#include #include -#include +#include +#include +#include #include #include +#include +#include #include namespace DB @@ -15,39 +18,133 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; - extern const int ILLEGAL_STATISTIC; } -void MergeTreeStatisticsFactory::registerCreator(StatisticType stat_type, Creator creator) +/// Version / bitmask of statistics / data of statistics / +enum StatisticsFileVersion : UInt16 { - if (!creators.emplace(stat_type, std::move(creator)).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistic creator type {} is not unique", stat_type); -} + V0 = 0, +}; -void MergeTreeStatisticsFactory::registerValidator(StatisticType stat_type, Validator validator) +IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) {} + +ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_) + : stats_desc(stats_desc_), rows(0) { - if (!validators.emplace(stat_type, std::move(validator)).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistic validator type {} is not unique", stat_type); - } -StatisticPtr TDigestCreator(const StatisticDescription & stat) +void ColumnStatistics::update(const ColumnPtr & column) { - return StatisticPtr(new TDigestStatistic(stat)); + rows += column->size(); + for (const auto & iter : stats) + { + iter.second->update(column); + } } -void TDigestValidator(const StatisticDescription &, DataTypePtr data_type) +Float64 ColumnStatistics::estimateLess(Float64 val) const { - data_type = removeNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTIC, "TDigest does not support type {}", data_type->getName()); + if (stats.contains(StatisticsType::TDigest)) + return std::static_pointer_cast(stats.at(StatisticsType::TDigest))->estimateLess(val); + return rows * ConditionSelectivityEstimator::default_normal_cond_factor; } +Float64 ColumnStatistics::estimateGreater(Float64 val) const +{ + return rows - estimateLess(val); +} + +Float64 ColumnStatistics::estimateEqual(Float64 val) const +{ + if (stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) + { + auto uniq_static = std::static_pointer_cast(stats.at(StatisticsType::Uniq)); + /// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) + /// for every bucket. + if (uniq_static->getCardinality() < 2048) + { + auto tdigest_static = std::static_pointer_cast(stats.at(StatisticsType::TDigest)); + return tdigest_static->estimateEqual(val); + } + } + if (val < - ConditionSelectivityEstimator::threshold || val > ConditionSelectivityEstimator::threshold) + return rows * ConditionSelectivityEstimator::default_normal_cond_factor; + else + return rows * ConditionSelectivityEstimator::default_good_cond_factor; +} + +void ColumnStatistics::serialize(WriteBuffer & buf) +{ + writeIntBinary(V0, buf); + UInt64 stat_types_mask = 0; + for (const auto & [type, _]: stats) + stat_types_mask |= 1 << UInt8(type); + writeIntBinary(stat_types_mask, buf); + /// We write some basic statistics + writeIntBinary(rows, buf); + /// We write complex statistics + for (const auto & [type, stat_ptr]: stats) + stat_ptr->serialize(buf); +} + +void ColumnStatistics::deserialize(ReadBuffer &buf) +{ + UInt16 version; + readIntBinary(version, buf); + if (version != V0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown file format version: {}", version); + + UInt64 stat_types_mask = 0; + readIntBinary(stat_types_mask, buf); + readIntBinary(rows, buf); + for (auto it = stats.begin(); it != stats.end();) + { + if (!(stat_types_mask & 1 << UInt8(it->first))) + { + stats.erase(it++); + } + else + { + it->second->deserialize(buf); + ++it; + } + } +} + +String ColumnStatistics::getFileName() const +{ + return STATS_FILE_PREFIX + columnName(); +} + +const String & ColumnStatistics::columnName() const +{ + return stats_desc.column_name; +} + +UInt64 ColumnStatistics::rowCount() const +{ + return rows; +} + +void MergeTreeStatisticsFactory::registerCreator(StatisticsType stats_type, Creator creator) +{ + if (!creators.emplace(stats_type, std::move(creator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics creator type {} is not unique", stats_type); +} + +void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Validator validator) +{ + if (!validators.emplace(stats_type, std::move(validator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics validator type {} is not unique", stats_type); + +} MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { - registerCreator(TDigest, TDigestCreator); - registerValidator(TDigest, TDigestValidator); + registerCreator(StatisticsType::TDigest, TDigestCreator); + registerCreator(StatisticsType::Uniq, UniqCreator); + registerValidator(StatisticsType::TDigest, TDigestValidator); + registerValidator(StatisticsType::Uniq, UniqValidator); } MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance() @@ -56,33 +153,42 @@ MergeTreeStatisticsFactory & MergeTreeStatisticsFactory::instance() return instance; } -void MergeTreeStatisticsFactory::validate(const StatisticDescription & stat, DataTypePtr data_type) const +void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const { - auto it = validators.find(stat.type); - if (it == validators.end()) + for (const auto & [type, desc] : stats.types_to_desc) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown Statistic type '{}'", stat.type); + auto it = validators.find(type); + if (it == validators.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown Statistic type '{}'", type); + } + it->second(desc, data_type); } - it->second(stat, data_type); } -StatisticPtr MergeTreeStatisticsFactory::get(const StatisticDescription & stat) const +ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescription & stats) const { - auto it = creators.find(stat.type); - if (it == creators.end()) + ColumnStatisticsPtr column_stat = std::make_shared(stats); + for (const auto & [type, desc] : stats.types_to_desc) { - throw Exception(ErrorCodes::INCORRECT_QUERY, - "Unknown Statistic type '{}'. Available types: tdigest", stat.type); + auto it = creators.find(type); + if (it == creators.end()) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Unknown Statistic type '{}'. Available types: tdigest, uniq", type); + } + auto stat_ptr = (it->second)(desc, stats.data_type); + column_stat->stats[type] = stat_ptr; } - return std::make_shared(stat); + return column_stat; } -Statistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription & columns) const +ColumnsStatistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription & columns) const { - Statistics result; + ColumnsStatistics result; for (const auto & col : columns) - if (col.stat) - result.push_back(get(*col.stat)); + if (!col.statistics.empty()) + result.push_back(get(col.statistics)); return result; } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index e6d9666ce1c..2ab1337af02 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -1,12 +1,8 @@ #pragma once -#include #include -#include - #include -#include #include #include #include @@ -14,38 +10,23 @@ #include -/// this is for user-defined statistic. -constexpr auto STAT_FILE_PREFIX = "statistic_"; -constexpr auto STAT_FILE_SUFFIX = ".stat"; - namespace DB { -class IStatistic; -using StatisticPtr = std::shared_ptr; -using Statistics = std::vector; +/// this is for user-defined statistic. +constexpr auto STATS_FILE_PREFIX = "statistics_"; +constexpr auto STATS_FILE_SUFFIX = ".stats"; -/// Statistic contains the distribution of values in a column. -/// right now we support -/// - tdigest -class IStatistic +/// Statistics describe properties of the values in the column, +/// e.g. how many unique values exist, +/// what are the N most frequent values, +/// how frequent is a value V, etc. +class IStatistics { public: - explicit IStatistic(const StatisticDescription & stat_) - : stat(stat_) - { - } - virtual ~IStatistic() = default; + explicit IStatistics(const SingleStatisticsDescription & stat_); - String getFileName() const - { - return STAT_FILE_PREFIX + columnName(); - } - - const String & columnName() const - { - return stat.column_name; - } + virtual ~IStatistics() = default; virtual void serialize(WriteBuffer & buf) = 0; @@ -53,40 +34,68 @@ public: virtual void update(const ColumnPtr & column) = 0; - virtual UInt64 count() = 0; - protected: + SingleStatisticsDescription stat; +}; - StatisticDescription stat; +using StatisticsPtr = std::shared_ptr; +class ColumnStatistics +{ +public: + explicit ColumnStatistics(const ColumnStatisticsDescription & stats_); + void serialize(WriteBuffer & buf); + void deserialize(ReadBuffer & buf); + String getFileName() const; + + const String & columnName() const; + + UInt64 rowCount() const; + + void update(const ColumnPtr & column); + + Float64 estimateLess(Float64 val) const; + + Float64 estimateGreater(Float64 val) const; + + Float64 estimateEqual(Float64 val) const; + +private: + + friend class MergeTreeStatisticsFactory; + ColumnStatisticsDescription stats_desc; + std::map stats; + UInt64 rows; /// the number of rows of the column }; class ColumnsDescription; +using ColumnStatisticsPtr = std::shared_ptr; +using ColumnsStatistics = std::vector; class MergeTreeStatisticsFactory : private boost::noncopyable { public: static MergeTreeStatisticsFactory & instance(); - void validate(const StatisticDescription & stat, DataTypePtr data_type) const; + void validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const; - using Creator = std::function; + using Creator = std::function; - using Validator = std::function; + using Validator = std::function; - StatisticPtr get(const StatisticDescription & stat) const; + ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const; - Statistics getMany(const ColumnsDescription & columns) const; + ColumnsStatistics getMany(const ColumnsDescription & columns) const; - void registerCreator(StatisticType type, Creator creator); - void registerValidator(StatisticType type, Validator validator); + void registerCreator(StatisticsType type, Creator creator); + void registerValidator(StatisticsType type, Validator validator); protected: MergeTreeStatisticsFactory(); private: - using Creators = std::unordered_map; - using Validators = std::unordered_map; + using Creators = std::unordered_map; + using Validators = std::unordered_map; Creators creators; Validators validators; }; diff --git a/src/Storages/Statistics/TDigestStatistic.cpp b/src/Storages/Statistics/TDigestStatistic.cpp deleted file mode 100644 index efb4282d203..00000000000 --- a/src/Storages/Statistics/TDigestStatistic.cpp +++ /dev/null @@ -1,38 +0,0 @@ -#include - -namespace DB -{ - -Float64 TDigestStatistic::estimateLess(Float64 val) const -{ - return data.getCountLessThan(val); -} - -void TDigestStatistic::serialize(WriteBuffer & buf) -{ - data.serialize(buf); -} - -void TDigestStatistic::deserialize(ReadBuffer & buf) -{ - data.deserialize(buf); -} - -void TDigestStatistic::update(const ColumnPtr & column) -{ - size_t size = column->size(); - - for (size_t i = 0; i < size; ++i) - { - /// TODO: support more types. - Float64 value = column->getFloat64(i); - data.add(value, 1); - } -} - -UInt64 TDigestStatistic::count() -{ - return static_cast(data.count); -} - -} diff --git a/src/Storages/Statistics/TDigestStatistic.h b/src/Storages/Statistics/TDigestStatistic.h deleted file mode 100644 index 295b5f69900..00000000000 --- a/src/Storages/Statistics/TDigestStatistic.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/// TDigestStatistic is a kind of histogram. -class TDigestStatistic : public IStatistic -{ - QuantileTDigest data; -public: - explicit TDigestStatistic(const StatisticDescription & stat_) : IStatistic(stat_) - { - } - - Float64 estimateLess(Float64 val) const; - - void serialize(WriteBuffer & buf) override; - - void deserialize(ReadBuffer & buf) override; - - void update(const ColumnPtr & column) override; - - UInt64 count() override; -}; - -} diff --git a/src/Storages/Statistics/TDigestStatistics.cpp b/src/Storages/Statistics/TDigestStatistics.cpp new file mode 100644 index 00000000000..aa5662c979d --- /dev/null +++ b/src/Storages/Statistics/TDigestStatistics.cpp @@ -0,0 +1,60 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_STATISTICS; +} + +TDigestStatistics::TDigestStatistics(const SingleStatisticsDescription & stat_): + IStatistics(stat_) +{ +} + +Float64 TDigestStatistics::estimateLess(Float64 val) const +{ + return data.getCountLessThan(val); +} + +Float64 TDigestStatistics::estimateEqual(Float64 val) const +{ + return data.getCountEqual(val); +} + +void TDigestStatistics::serialize(WriteBuffer & buf) +{ + data.serialize(buf); +} + +void TDigestStatistics::deserialize(ReadBuffer & buf) +{ + data.deserialize(buf); +} + +void TDigestStatistics::update(const ColumnPtr & column) +{ + size_t size = column->size(); + + for (size_t i = 0; i < size; ++i) + { + /// TODO: support more types. + Float64 value = column->getFloat64(i); + data.add(value, 1); + } +} + +StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +{ + return std::make_shared(stat); +} + +void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +{ + data_type = removeNullable(data_type); + if (!data_type->isValueRepresentedByNumber()) + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' does not support type {}", data_type->getName()); +} + +} diff --git a/src/Storages/Statistics/TDigestStatistics.h b/src/Storages/Statistics/TDigestStatistics.h new file mode 100644 index 00000000000..7c361b8751f --- /dev/null +++ b/src/Storages/Statistics/TDigestStatistics.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + +namespace DB +{ + + +/// TDigestStatistic is a kind of histogram. +class TDigestStatistics : public IStatistics +{ +public: + explicit TDigestStatistics(const SingleStatisticsDescription & stat_); + + Float64 estimateLess(Float64 val) const; + + Float64 estimateEqual(Float64 val) const; + + void serialize(WriteBuffer & buf) override; + + void deserialize(ReadBuffer & buf) override; + + void update(const ColumnPtr & column) override; +private: + QuantileTDigest data; +}; + +StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); + +} diff --git a/src/Storages/Statistics/UniqStatistics.cpp b/src/Storages/Statistics/UniqStatistics.cpp new file mode 100644 index 00000000000..fc748e769ca --- /dev/null +++ b/src/Storages/Statistics/UniqStatistics.cpp @@ -0,0 +1,66 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_STATISTICS; +} + +UniqStatistics::UniqStatistics(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type) + : IStatistics(stat_) +{ + arena = std::make_unique(); + AggregateFunctionProperties properties; + collector = AggregateFunctionFactory::instance().get("uniq", NullsAction::IGNORE_NULLS, {data_type}, Array(), properties); + data = arena->alignedAlloc(collector->sizeOfData(), collector->alignOfData()); + collector->create(data); +} + +UniqStatistics::~UniqStatistics() +{ + collector->destroy(data); +} + +UInt64 UniqStatistics::getCardinality() +{ + auto column = DataTypeUInt64().createColumn(); + collector->insertResultInto(data, *column, nullptr); + return column->getUInt(0); +} + +void UniqStatistics::serialize(WriteBuffer & buf) +{ + collector->serialize(data, buf); +} + +void UniqStatistics::deserialize(ReadBuffer & buf) +{ + collector->deserialize(data, buf); +} + +void UniqStatistics::update(const ColumnPtr & column) +{ + /// TODO(hanfei): For low cardinality, it's very slow to convert to full column. We can read the dictionary directly. + /// Here we intend to avoid crash in CI. + auto col_ptr = column->convertToFullColumnIfLowCardinality(); + const IColumn * raw_ptr = col_ptr.get(); + collector->addBatchSinglePlace(0, column->size(), data, &(raw_ptr), nullptr); +} + +void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +{ + data_type = removeNullable(data_type); + if (!data_type->isValueRepresentedByNumber()) + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' does not support type {}", data_type->getName()); +} + +StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +{ + return std::make_shared(stat, data_type); +} + +} diff --git a/src/Storages/Statistics/UniqStatistics.h b/src/Storages/Statistics/UniqStatistics.h new file mode 100644 index 00000000000..0d86a6e458a --- /dev/null +++ b/src/Storages/Statistics/UniqStatistics.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class UniqStatistics : public IStatistics +{ +public: + UniqStatistics(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); + + ~UniqStatistics() override; + + UInt64 getCardinality(); + + void serialize(WriteBuffer & buf) override; + + void deserialize(ReadBuffer & buf) override; + + void update(const ColumnPtr & column) override; + +private: + + std::unique_ptr arena; + AggregateFunctionPtr collector; + AggregateDataPtr data; + +}; + +StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); + +} diff --git a/src/Storages/Statistics/tests/gtest_stats.cpp b/src/Storages/Statistics/tests/gtest_stats.cpp index 45f8271be97..f94f310be56 100644 --- a/src/Storages/Statistics/tests/gtest_stats.cpp +++ b/src/Storages/Statistics/tests/gtest_stats.cpp @@ -1,6 +1,6 @@ #include -#include +#include TEST(Statistics, TDigestLessThan) { diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 7d4226f2fbe..f10fb78f933 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -1,14 +1,16 @@ +#include + #include #include #include #include -#include +#include #include #include +#include #include #include #include -#include #include #include @@ -19,93 +21,187 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_QUERY; + extern const int ILLEGAL_STATISTICS; extern const int LOGICAL_ERROR; }; -StatisticDescription & StatisticDescription::operator=(const StatisticDescription & other) +SingleStatisticsDescription & SingleStatisticsDescription::operator=(const SingleStatisticsDescription & other) { if (this == &other) return *this; type = other.type; - column_name = other.column_name; ast = other.ast ? other.ast->clone() : nullptr; return *this; } -StatisticDescription & StatisticDescription::operator=(StatisticDescription && other) noexcept +SingleStatisticsDescription & SingleStatisticsDescription::operator=(SingleStatisticsDescription && other) noexcept { if (this == &other) return *this; - type = std::exchange(other.type, StatisticType{}); - column_name = std::move(other.column_name); + type = std::exchange(other.type, StatisticsType{}); ast = other.ast ? other.ast->clone() : nullptr; other.ast.reset(); return *this; } -StatisticType stringToType(String type) +static StatisticsType stringToStatisticsType(String type) { if (type == "tdigest") - return TDigest; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. We only support statistic type `tdigest` right now.", type); + return StatisticsType::TDigest; + if (type == "uniq") + return StatisticsType::Uniq; + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } -String StatisticDescription::getTypeName() const +String SingleStatisticsDescription::getTypeName() const { - if (type == TDigest) - return "tdigest"; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. We only support statistic type `tdigest` right now.", type); -} - -std::vector StatisticDescription::getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) -{ - const auto * stat_definition = definition_ast->as(); - if (!stat_definition) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create statistic from non ASTStatisticDeclaration AST"); - - std::vector stats; - stats.reserve(stat_definition->columns->children.size()); - for (const auto & column_ast : stat_definition->columns->children) + switch (type) { - StatisticDescription stat; - stat.type = stringToType(Poco::toLower(stat_definition->type)); - String column_name = column_ast->as().name(); + case StatisticsType::TDigest: + return "TDigest"; + case StatisticsType::Uniq: + return "Uniq"; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); + } +} - if (!columns.hasPhysical(column_name)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", column_name); +SingleStatisticsDescription::SingleStatisticsDescription(StatisticsType type_, ASTPtr ast_) + : type(type_), ast(ast_) +{} - const auto & column = columns.getPhysical(column_name); - stat.column_name = column.name; - stat.ast = makeASTFunction("STATISTIC", std::make_shared(stat_definition->type)); - stats.push_back(stat); +bool SingleStatisticsDescription::operator==(const SingleStatisticsDescription & other) const +{ + return type == other.type; +} + +bool ColumnStatisticsDescription::operator==(const ColumnStatisticsDescription & other) const +{ + return types_to_desc == other.types_to_desc; +} + +bool ColumnStatisticsDescription::empty() const +{ + return types_to_desc.empty(); +} + +bool ColumnStatisticsDescription::contains(const String & stat_type) const +{ + return types_to_desc.contains(stringToStatisticsType(stat_type)); +} + +void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & other, const String & merging_column_name, DataTypePtr merging_column_type, bool if_not_exists) +{ + chassert(merging_column_type); + + if (column_name.empty()) + { + column_name = merging_column_name; + data_type = merging_column_type; } - if (stats.empty()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list"); + for (const auto & [stats_type, stats_desc]: other.types_to_desc) + { + if (!if_not_exists && types_to_desc.contains(stats_type)) + { + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name); + } + else if (!types_to_desc.contains(stats_type)) + types_to_desc.emplace(stats_type, stats_desc); + } +} +void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other) +{ + if (other.column_name != column_name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", column_name, other.column_name); + + types_to_desc = other.types_to_desc; +} + +void ColumnStatisticsDescription::clear() +{ + types_to_desc.clear(); +} + +std::vector ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) +{ + const auto * stat_definition_ast = definition_ast->as(); + if (!stat_definition_ast) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot cast AST to ASTSingleStatisticsDeclaration"); + + StatisticsTypeDescMap statistics_types; + for (const auto & stat_ast : stat_definition_ast->types->children) + { + String stat_type_name = stat_ast->as().name; + auto stat_type = stringToStatisticsType(Poco::toLower(stat_type_name)); + if (statistics_types.contains(stat_type)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistics type {} was specified more than once", stat_type_name); + SingleStatisticsDescription stat(stat_type, stat_ast->clone()); + + statistics_types.emplace(stat.type, stat); + } + + std::vector result; + result.reserve(stat_definition_ast->columns->children.size()); + + for (const auto & column_ast : stat_definition_ast->columns->children) + { + ColumnStatisticsDescription stats; + String physical_column_name = column_ast->as().name(); + + if (!columns.hasPhysical(physical_column_name)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name); + + const auto & column = columns.getPhysical(physical_column_name); + stats.column_name = column.name; + stats.types_to_desc = statistics_types; + result.push_back(stats); + } + + if (result.empty()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistics column list is not allowed."); + + return result; +} + +ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type) +{ + const auto & stat_type_list_ast = column.statistics_desc->as().arguments; + if (stat_type_list_ast->children.empty()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column)); + ColumnStatisticsDescription stats; + stats.column_name = column.name; + for (const auto & ast : stat_type_list_ast->children) + { + const auto & stat_type = ast->as().name; + + SingleStatisticsDescription stat(stringToStatisticsType(Poco::toLower(stat_type)), ast->clone()); + if (stats.types_to_desc.contains(stat.type)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type); + stats.types_to_desc.emplace(stat.type, std::move(stat)); + } + stats.data_type = data_type; return stats; } -String queryToString(const IAST & query); - -StatisticDescription StatisticDescription::getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column) +ASTPtr ColumnStatisticsDescription::getAST() const { - const auto & stat_type_list_ast = column.stat_type->as().arguments; - if (stat_type_list_ast->children.size() != 1) - throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect only one statistic type for column {}", queryToString(column)); - - const auto & stat_type = stat_type_list_ast->children[0]->as().name; - - StatisticDescription stat; - stat.type = stringToType(Poco::toLower(stat_type)); - stat.column_name = column.name; - stat.ast = column.stat_type; - - return stat; + auto function_node = std::make_shared(); + function_node->name = "STATISTICS"; + function_node->arguments = std::make_shared(); + for (const auto & [type, desc] : types_to_desc) + { + if (desc.ast == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown ast"); + function_node->arguments->children.push_back(desc.ast); + } + function_node->children.push_back(function_node->arguments); + return function_node; } } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index b571fa31e9d..4862fb79d45 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -1,45 +1,66 @@ #pragma once +#include #include #include + #include namespace DB { -enum StatisticType +enum class StatisticsType : UInt8 { TDigest = 0, + Uniq = 1, + + Max = 63, }; -class ColumnsDescription; - -struct StatisticDescription +struct SingleStatisticsDescription { - /// the type of statistic, right now it's only tdigest. - StatisticType type; - - /// Names of statistic columns - String column_name; + StatisticsType type; ASTPtr ast; String getTypeName() const; - StatisticDescription() = default; - StatisticDescription(const StatisticDescription & other) { *this = other; } - StatisticDescription & operator=(const StatisticDescription & other); - StatisticDescription(StatisticDescription && other) noexcept { *this = std::move(other); } - StatisticDescription & operator=(StatisticDescription && other) noexcept; + SingleStatisticsDescription() = delete; + SingleStatisticsDescription(StatisticsType type_, ASTPtr ast_); - bool operator==(const StatisticDescription & other) const - { - return type == other.type && column_name == other.column_name; - } + SingleStatisticsDescription(const SingleStatisticsDescription & other) { *this = other; } + SingleStatisticsDescription & operator=(const SingleStatisticsDescription & other); + SingleStatisticsDescription(SingleStatisticsDescription && other) noexcept { *this = std::move(other); } + SingleStatisticsDescription & operator=(SingleStatisticsDescription && other) noexcept; - static StatisticDescription getStatisticFromColumnDeclaration(const ASTColumnDeclaration & column); + bool operator==(const SingleStatisticsDescription & other) const; +}; - static std::vector getStatisticsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); +class ColumnsDescription; + +struct ColumnStatisticsDescription +{ + bool operator==(const ColumnStatisticsDescription & other) const; + + bool empty() const; + + bool contains(const String & stat_type) const; + + void merge(const ColumnStatisticsDescription & other, const String & column_name, DataTypePtr column_type, bool if_not_exists); + + void assign(const ColumnStatisticsDescription & other); + + void clear(); + + ASTPtr getAST() const; + + static std::vector fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); + static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type); + + using StatisticsTypeDescMap = std::map; + StatisticsTypeDescMap types_to_desc; + String column_name; + DataTypePtr data_type; }; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index fbb40f8b79f..5048ef4788e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -904,11 +904,13 @@ void StorageDistributed::read( [my_custom_key_ast = std::move(custom_key_ast), column_description = this->getInMemoryMetadataPtr()->columns, custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, + custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, + custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, context = local_context, replica_count = modified_query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr { return getCustomKeyFilterForParallelReplica( - replica_count, replica_num - 1, my_custom_key_ast, custom_key_type, column_description, context); + replica_count, replica_num - 1, my_custom_key_ast, {custom_key_type, custom_key_range_lower, custom_key_range_upper}, column_description, context); }; } } @@ -1986,9 +1988,18 @@ void registerStorageDistributed(StorageFactory & factory) bool StorageDistributed::initializeDiskOnConfigChange(const std::set & new_added_disks) { - if (!data_volume) + if (!storage_policy || !data_volume) return true; + auto new_storage_policy = getContext()->getStoragePolicy(storage_policy->getName()); + auto new_data_volume = new_storage_policy->getVolume(0); + if (new_storage_policy->getVolumes().size() > 1) + LOG_WARNING(log, "Storage policy for Distributed table has multiple volumes. " + "Only {} volume will be used to store data. Other will be ignored.", data_volume->getName()); + + std::atomic_store(&storage_policy, new_storage_policy); + std::atomic_store(&data_volume, new_data_volume); + for (auto & disk : data_volume->getDisks()) { if (new_added_disks.contains(disk->getName())) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6744159d5dc..16c248f1b7b 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1341,6 +1341,7 @@ Chunk StorageFileSource::generate() chassert(file_enumerator); current_path = fmt::format("{}::{}", archive_reader->getPath(), *filename_override); current_file_size = file_enumerator->getFileInfo().uncompressed_size; + current_file_last_modified = file_enumerator->getFileInfo().last_modified; if (need_only_count && tryGetCountFromCache(current_archive_stat)) continue; @@ -1370,6 +1371,7 @@ Chunk StorageFileSource::generate() struct stat file_stat; file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); current_file_size = file_stat.st_size; + current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtime); if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; @@ -1436,8 +1438,15 @@ Chunk StorageFileSource::generate() progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); /// Enrich with virtual columns. - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = current_path, + .size = current_file_size, + .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), + .last_modified = current_file_last_modified + }); + return chunk; } @@ -1814,7 +1823,6 @@ private: { writer->finalize(); writer->flush(); - write_buf->finalize(); } catch (...) { @@ -1822,12 +1830,14 @@ private: release(); throw; } + + write_buf->finalize(); } void release() { writer.reset(); - write_buf->finalize(); + write_buf.reset(); } StorageMetadataPtr metadata_snapshot; diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 37da59c3664..ac094aeb489 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -279,6 +279,7 @@ private: FilesIteratorPtr files_iterator; String current_path; std::optional current_file_size; + std::optional current_file_last_modified; struct stat current_archive_stat; std::optional filename_override; Block sample_block; diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index cdbade51695..2f850c76465 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -50,6 +50,12 @@ namespace ErrorCodes namespace { +struct GenerateRandomState +{ + std::atomic add_total_rows = 0; +}; +using GenerateRandomStatePtr = std::shared_ptr; + void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false) { size_t size = limit * size_of_type; @@ -267,6 +273,9 @@ ColumnPtr fillColumnWithRandomData( case TypeIndex::Tuple: { auto elements = typeid_cast(type.get())->getElements(); + if (elements.empty()) + return ColumnTuple::create(limit); + const size_t tuple_size = elements.size(); Columns tuple_columns(tuple_size); @@ -529,10 +538,24 @@ ColumnPtr fillColumnWithRandomData( class GenerateSource : public ISource { public: - GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) + GenerateSource( + UInt64 block_size_, + UInt64 max_array_length_, + UInt64 max_string_length_, + UInt64 random_seed_, + Block block_header_, + ContextPtr context_, + GenerateRandomStatePtr state_) : ISource(Nested::flattenNested(prepareBlockToFill(block_header_))) - , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) - , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} + , block_size(block_size_) + , max_array_length(max_array_length_) + , max_string_length(max_string_length_) + , block_to_fill(std::move(block_header_)) + , rng(random_seed_) + , context(context_) + , shared_state(state_) + { + } String getName() const override { return "GenerateRandom"; } @@ -546,7 +569,15 @@ protected: columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); columns = Nested::flattenNested(block_to_fill.cloneWithColumns(columns)).getColumns(); - return {std::move(columns), block_size}; + + UInt64 total_rows = shared_state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + auto chunk = Chunk{std::move(columns), block_size}; + progress(chunk.getNumRows(), chunk.bytes()); + + return chunk; } private: @@ -558,6 +589,7 @@ private: pcg64 rng; ContextPtr context; + GenerateRandomStatePtr shared_state; static Block & prepareBlockToFill(Block & block) { @@ -645,9 +677,6 @@ Pipe StorageGenerateRandom::read( { 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) @@ -676,16 +705,24 @@ Pipe StorageGenerateRandom::read( } } + UInt64 query_limit = query_info.limit; + if (query_limit && num_streams * max_block_size > query_limit) + { + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); + } + Pipes pipes; + pipes.reserve(num_streams); + /// Will create more seed values for each source from initial seed. pcg64 generate(random_seed); + auto shared_state = std::make_shared(query_info.limit); + for (UInt64 i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context); - - if (i == 0 && query_info.limit) - source->addTotalRowsApprox(query_info.limit); - + auto source = std::make_shared( + max_block_size, max_array_length, max_string_length, generate(), block_header, context, shared_state); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4c678a1228b..ed3f43367dd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -34,9 +34,10 @@ #include #include #include -#include #include #include +#include +#include #include #include #include @@ -402,10 +403,14 @@ ReadFromMerge::ReadFromMerge( { } -void ReadFromMerge::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) +void ReadFromMerge::addFilter(FilterDAGInfo filter) { - SourceStepWithFilter::updatePrewhereInfo(prewhere_info_value); - common_header = applyPrewhereActions(common_header, prewhere_info); + output_stream->header = FilterTransform::transformHeader( + output_stream->header, + filter.actions.get(), + filter.column_name, + filter.do_remove_column); + pushed_down_filters.push_back(std::move(filter)); } void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -435,21 +440,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu Names column_names_as_aliases; Aliases aliases; - Names real_column_names = column_names; - if (child_plan.row_policy_data_opt) - child_plan.row_policy_data_opt->extendNames(real_column_names); - - auto modified_query_info = getModifiedQueryInfo(modified_context, table, nested_storage_snaphsot, real_column_names, column_names_as_aliases, aliases); - - auto source_pipeline = createSources( - child_plan.plan, - nested_storage_snaphsot, - modified_query_info, - common_processed_stage, - common_header, - child_plan.table_aliases, - child_plan.row_policy_data_opt, - table); + auto source_pipeline = buildPipeline(child_plan, common_processed_stage); if (source_pipeline && source_pipeline->initialized()) { @@ -567,10 +558,8 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ if (sampling_requested && !storage->supportsSampling()) throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table {} doesn't support sampling", storage->getStorageID().getNameForLogs()); - res.emplace_back(); - - auto & aliases = res.back().table_aliases; - auto & row_policy_data_opt = res.back().row_policy_data_opt; + Aliases aliases; + RowPolicyDataOpt row_policy_data_opt; auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, modified_context); @@ -649,7 +638,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ } - res.back().plan = createPlanForTable( + auto child = createPlanForTable( nested_storage_snaphsot, modified_query_info, common_processed_stage, @@ -659,9 +648,32 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ row_policy_data_opt, modified_context, current_streams); - res.back().plan.addInterpreterContext(modified_context); - } + child.plan.addInterpreterContext(modified_context); + if (child.plan.isInitialized()) + { + addVirtualColumns(child, modified_query_info, common_processed_stage, table); + + /// Subordinary tables could have different but convertible types, like numeric types of different width. + /// We must return streams with structure equals to structure of Merge table. + convertAndFilterSourceStream(common_header, modified_query_info, nested_storage_snaphsot, aliases, row_policy_data_opt, context, child); + + for (const auto & filter_info : pushed_down_filters) + { + auto filter_step = std::make_unique( + child.plan.getCurrentDataStream(), + filter_info.actions->clone(), + filter_info.column_name, + filter_info.do_remove_column); + + child.plan.addStep(std::move(filter_step)); + } + + child.plan.optimize(QueryPlanOptimizationSettings::fromContext(modified_context)); + } + + res.emplace_back(std::move(child)); + } return res; } @@ -876,8 +888,6 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo const StorageID current_storage_id = storage->getStorageID(); SelectQueryInfo modified_query_info = query_info; - if (modified_query_info.optimized_prewhere_info && !modified_query_info.prewhere_info) - modified_query_info.prewhere_info = modified_query_info.optimized_prewhere_info; if (modified_query_info.planner_context) modified_query_info.planner_context = std::make_shared(modified_context, modified_query_info.planner_context); @@ -1019,31 +1029,101 @@ bool recursivelyApplyToReadingSteps(QueryPlan::Node * node, const std::function< return ok; } -QueryPipelineBuilderPtr ReadFromMerge::createSources( - QueryPlan & plan, - const StorageSnapshotPtr & storage_snapshot_, +void ReadFromMerge::addVirtualColumns( + ChildPlan & child, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, - const Block & header, - const Aliases & aliases, - const RowPolicyDataOpt & row_policy_data_opt, - const StorageWithLockAndName & storage_with_lock, - bool concat_streams) const + const StorageWithLockAndName & storage_with_lock) const { - if (!plan.isInitialized()) - return std::make_unique(); - - QueryPipelineBuilderPtr builder; - - const auto & [database_name, storage, _, table_name] = storage_with_lock; + const auto & [database_name, _, storage, table_name] = storage_with_lock; bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; - auto storage_stage - = storage->getQueryProcessingStage(context, processed_stage, storage_snapshot_, modified_query_info); - builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + /// Add virtual columns if we don't already have them. - if (processed_stage > storage_stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) + Block plan_header = child.plan.getCurrentDataStream().header; + + if (allow_experimental_analyzer) + { + String table_alias = modified_query_info.query_tree->as()->getJoinTree()->as()->getAlias(); + + String database_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_database" : table_alias + "._database"; + String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; + + if (has_database_virtual_column && common_header.has(database_column) + && child.stage == QueryProcessingStage::FetchColumns && !plan_header.has(database_column)) + { + ColumnWithTypeAndName column; + column.name = database_column; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; + } + + if (has_table_virtual_column && common_header.has(table_column) + && child.stage == QueryProcessingStage::FetchColumns && !plan_header.has(table_column)) + { + ColumnWithTypeAndName column; + column.name = table_column; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; + } + } + else + { + if (has_database_virtual_column && common_header.has("_database") && !plan_header.has("_database")) + { + ColumnWithTypeAndName column; + column.name = "_database"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(database_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; + } + + if (has_table_virtual_column && common_header.has("_table") && !plan_header.has("_table")) + { + ColumnWithTypeAndName column; + column.name = "_table"; + column.type = std::make_shared(std::make_shared()); + column.column = column.type->createColumnConst(0, Field(table_name)); + + auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), adding_column_dag); + child.plan.addStep(std::move(expression_step)); + plan_header = child.plan.getCurrentDataStream().header; + } + } +} + +QueryPipelineBuilderPtr ReadFromMerge::buildPipeline( + ChildPlan & child, + QueryProcessingStage::Enum processed_stage) const +{ + if (!child.plan.isInitialized()) + return nullptr; + + auto optimisation_settings = QueryPlanOptimizationSettings::fromContext(context); + /// All optimisations will be done at plans creation + optimisation_settings.optimize_plan = false; + auto builder = child.plan.buildQueryPipeline(optimisation_settings, BuildQueryPipelineSettings::fromContext(context)); + + if (!builder->initialized()) + return builder; + + bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer; + if (processed_stage > child.stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns)) { /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, @@ -1052,99 +1132,10 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( builder->addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } - if (builder->initialized()) - { - if (concat_streams && builder->getNumStreams() > 1) - { - // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. - // Using concat instead. - builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); - } - - /// Add virtual columns if we don't already have them. - - Block pipe_header = builder->getHeader(); - - if (allow_experimental_analyzer) - { - String table_alias = modified_query_info.query_tree->as()->getJoinTree()->as()->getAlias(); - - String database_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_database" : table_alias + "._database"; - String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; - - if (has_database_virtual_column && common_header.has(database_column) - && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(database_column)) - { - ColumnWithTypeAndName column; - column.name = database_column; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - - if (has_table_virtual_column && common_header.has(table_column) - && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(table_column)) - { - ColumnWithTypeAndName column; - column.name = table_column; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - } - else - { - if (has_database_virtual_column && common_header.has("_database") && !pipe_header.has("_database")) - { - ColumnWithTypeAndName column; - column.name = "_database"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(database_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - - if (has_table_virtual_column && common_header.has("_table") && !pipe_header.has("_table")) - { - ColumnWithTypeAndName column; - column.name = "_table"; - column.type = std::make_shared(std::make_shared()); - column.column = column.type->createColumnConst(0, Field(table_name)); - - auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); - auto adding_column_actions = std::make_shared( - std::move(adding_column_dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); - builder->addSimpleTransform([&](const Block & stream_header) - { return std::make_shared(stream_header, adding_column_actions); }); - } - } - - /// Subordinary tables could have different but convertible types, like numeric types of different width. - /// We must return streams with structure equals to structure of Merge table. - convertAndFilterSourceStream( - header, modified_query_info, storage_snapshot_, aliases, row_policy_data_opt, context, *builder, storage_stage); - } - return builder; } -QueryPlan ReadFromMerge::createPlanForTable( +ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( const StorageSnapshotPtr & storage_snapshot_, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, @@ -1181,35 +1172,14 @@ QueryPlan ReadFromMerge::createPlanForTable( if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot_->metadata->getColumns().getAllPhysical()).name); - StorageView * view = dynamic_cast(storage.get()); - if (!view || allow_experimental_analyzer) - { - storage->read(plan, - real_column_names, - storage_snapshot_, - modified_query_info, - modified_context, - processed_stage, - max_block_size, - UInt32(streams_num)); - } - else - { - /// For view storage, we need to rewrite the `modified_query_info.view_query` to optimize read. - /// The most intuitive way is to use InterpreterSelectQuery. - - /// Intercept the settings - modified_context->setSetting("max_threads", streams_num); - modified_context->setSetting("max_streams_to_max_threads_ratio", 1); - modified_context->setSetting("max_block_size", max_block_size); - - InterpreterSelectQuery interpreter(modified_query_info.query, - modified_context, - storage, - view->getInMemoryMetadataPtr(), - SelectQueryOptions(processed_stage)); - interpreter.buildQueryPlan(plan); - } + storage->read(plan, + real_column_names, + storage_snapshot_, + modified_query_info, + modified_context, + processed_stage, + max_block_size, + UInt32(streams_num)); if (!plan.isInitialized()) return {}; @@ -1248,7 +1218,7 @@ QueryPlan ReadFromMerge::createPlanForTable( } } - return plan; + return ChildPlan{std::move(plan), storage_stage}; } ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter_ptr, @@ -1306,12 +1276,10 @@ void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) step->addFilter(actions_dag, filter_column_name); } -void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPipelineBuilder & builder) const +void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const { - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, filter_actions, filter_column_name, true /* remove filter column */); - }); + auto filter_step = std::make_unique(plan.getCurrentDataStream(), actions_dag, filter_column_name, true /* remove filter column */); + plan.addStep(std::move(filter_step)); } StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables( @@ -1490,13 +1458,12 @@ void ReadFromMerge::convertAndFilterSourceStream( const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, ContextPtr local_context, - QueryPipelineBuilder & builder, - QueryProcessingStage::Enum processed_stage) + ChildPlan & child) { - Block before_block_header = builder.getHeader(); + Block before_block_header = child.plan.getCurrentDataStream().header; auto storage_sample_block = snapshot->metadata->getSampleBlock(); - auto pipe_columns = builder.getHeader().getNamesAndTypesList(); + auto pipe_columns = before_block_header.getNamesAndTypesList(); if (local_context->getSettingsRef().allow_experimental_analyzer) { @@ -1519,13 +1486,8 @@ void ReadFromMerge::convertAndFilterSourceStream( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); - - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, actions); - }); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + child.plan.addStep(std::move(expression_step)); } } else @@ -1539,37 +1501,26 @@ void ReadFromMerge::convertAndFilterSourceStream( auto dag = std::make_shared(pipe_columns); auto actions_dag = expression_analyzer.getActionsDAG(true, false); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, actions); - }); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), actions_dag); + child.plan.addStep(std::move(expression_step)); } } ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name; if (local_context->getSettingsRef().allow_experimental_analyzer - && (processed_stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) + && (child.stage != QueryProcessingStage::FetchColumns || dynamic_cast(&snapshot->storage) != nullptr)) convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position; if (row_policy_data_opt) - { - row_policy_data_opt->addFilterTransform(builder); - } + row_policy_data_opt->addFilterTransform(child.plan); - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), + auto convert_actions_dag = ActionsDAG::makeConvertingActions(child.plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); - auto actions = std::make_shared( - std::move(convert_actions_dag), - ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header, actions); - }); + auto expression_step = std::make_unique(child.plan.getCurrentDataStream(), convert_actions_dag); + child.plan.addStep(std::move(expression_step)); } const ReadFromMerge::StorageListWithLocks & ReadFromMerge::getSelectedTables() @@ -1606,29 +1557,14 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) return true; } -void ReadFromMerge::applyFilters(const QueryPlan & plan, const ActionDAGNodes & added_filter_nodes) const -{ - auto apply_filters = [&added_filter_nodes](ReadFromMergeTree & read_from_merge_tree) - { - for (const auto & node : added_filter_nodes.nodes) - read_from_merge_tree.addFilterFromParentStep(node); - - read_from_merge_tree.SourceStepWithFilter::applyFilters(); - return true; - }; - - recursivelyApplyToReadingSteps(plan.getRootNode(), apply_filters); -} - void ReadFromMerge::applyFilters(ActionDAGNodes added_filter_nodes) { + for (const auto & filter_info : pushed_down_filters) + added_filter_nodes.nodes.push_back(&filter_info.actions->findInOutputs(filter_info.column_name)); + SourceStepWithFilter::applyFilters(added_filter_nodes); filterTablesAndCreateChildrenPlans(); - - for (const auto & child_plan : *child_plans) - if (child_plan.plan.isInitialized()) - applyFilters(child_plan.plan, added_filter_nodes); } QueryPlanRawPtrs ReadFromMerge::getChildPlans() diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 735c8711a63..94b34256d02 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -165,7 +165,7 @@ public: QueryPlanRawPtrs getChildPlans() override; - void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) override; + void addFilter(FilterDAGInfo filter); private: const size_t required_max_block_size; @@ -221,7 +221,7 @@ private: /// Create explicit filter transform to exclude /// rows that are not conform to row level policy - void addFilterTransform(QueryPipelineBuilder &) const; + void addFilterTransform(QueryPlan &) const; private: std::string filter_column_name; // complex filter, may contain logic operations @@ -235,21 +235,21 @@ private: struct ChildPlan { QueryPlan plan; - Aliases table_aliases; - RowPolicyDataOpt row_policy_data_opt; + QueryProcessingStage::Enum stage; }; /// Store read plan for each child table. /// It's needed to guarantee lifetime for child steps to be the same as for this step (mainly for EXPLAIN PIPELINE). std::optional> child_plans; + /// Store filters pushed down from query plan optimization. Filters are added on top of child plans. + std::vector pushed_down_filters; + std::vector createChildrenPlans(SelectQueryInfo & query_info_) const; void filterTablesAndCreateChildrenPlans(); - void applyFilters(const QueryPlan & plan, const ActionDAGNodes & added_filter_nodes) const; - - QueryPlan createPlanForTable( + ChildPlan createPlanForTable( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, QueryProcessingStage::Enum processed_stage, @@ -260,16 +260,15 @@ private: ContextMutablePtr modified_context, size_t streams_num) const; - QueryPipelineBuilderPtr createSources( - QueryPlan & plan, - const StorageSnapshotPtr & storage_snapshot, + void addVirtualColumns( + ChildPlan & child, SelectQueryInfo & modified_query_info, QueryProcessingStage::Enum processed_stage, - const Block & header, - const Aliases & aliases, - const RowPolicyDataOpt & row_policy_data_opt, - const StorageWithLockAndName & storage_with_lock, - bool concat_streams = false) const; + const StorageWithLockAndName & storage_with_lock) const; + + QueryPipelineBuilderPtr buildPipeline( + ChildPlan & child, + QueryProcessingStage::Enum processed_stage) const; static void convertAndFilterSourceStream( const Block & header, @@ -278,15 +277,12 @@ private: const Aliases & aliases, const RowPolicyDataOpt & row_policy_data_opt, ContextPtr context, - QueryPipelineBuilder & builder, - QueryProcessingStage::Enum processed_stage); + ChildPlan & child); StorageMerge::StorageListWithLocks getSelectedTables( ContextPtr query_context, bool filter_by_database_virtual_column, bool filter_by_table_virtual_column) const; - - // static VirtualColumnsDescription createVirtuals(StoragePtr first_table); }; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 27a76f4f21d..9352f772ce1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -213,36 +213,13 @@ void StorageMergeTree::read( size_t max_block_size, size_t num_streams) { - if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + const auto & settings = local_context->getSettingsRef(); + /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here + if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree + && !settings.allow_experimental_analyzer) { - ASTPtr modified_query_ast; - Block header; - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); - rewriteJoinToGlobalJoin(modified_query_tree, local_context); - modified_query_tree = buildQueryTreeForShard(query_info.planner_context, modified_query_tree); - header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - } - else - { - const auto table_id = getStorageID(); - modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, - getStorageID(), - header, - processed_stage, - modified_query_ast, - local_context, - query_info.storage_limits); + query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); } else { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e18e66d7af9..db58d0081c6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5461,7 +5461,8 @@ void StorageReplicatedMergeTree::read( /// For this you have to synchronously go to ZooKeeper. if (settings.select_sequential_consistency) readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); - else if (local_context->canUseParallelReplicasOnInitiator()) + /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here + else if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); else readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); @@ -5493,36 +5494,8 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( ContextPtr local_context, QueryProcessingStage::Enum processed_stage) { - ASTPtr modified_query_ast; - Block header; - const auto table_id = getStorageID(); - - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); - rewriteJoinToGlobalJoin(modified_query_tree, local_context); - modified_query_tree = buildQueryTreeForShard(query_info.planner_context, modified_query_tree); - - header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - } - else - { - modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, - table_id, - header, - processed_stage, - modified_query_ast, - local_context, - query_info.storage_limits); + query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); } void StorageReplicatedMergeTree::readLocalImpl( diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp deleted file mode 100644 index b767805f637..00000000000 --- a/src/Storages/StorageS3Settings.cpp +++ /dev/null @@ -1,315 +0,0 @@ -#include - -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INVALID_SETTING_VALUE; -} - -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings, bool validate_settings) -{ - updateFromSettings(settings, false); - if (validate_settings) - validate(); -} - -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix, - bool validate_settings) - : PartUploadSettings(settings, validate_settings) -{ - String key = config_prefix + "." + setting_name_prefix; - strict_upload_part_size = config.getUInt64(key + "strict_upload_part_size", strict_upload_part_size); - min_upload_part_size = config.getUInt64(key + "min_upload_part_size", min_upload_part_size); - max_upload_part_size = config.getUInt64(key + "max_upload_part_size", max_upload_part_size); - upload_part_size_multiply_factor = config.getUInt64(key + "upload_part_size_multiply_factor", upload_part_size_multiply_factor); - upload_part_size_multiply_parts_count_threshold = config.getUInt64(key + "upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); - max_inflight_parts_for_one_file = config.getUInt64(key + "max_inflight_parts_for_one_file", max_inflight_parts_for_one_file); - max_part_number = config.getUInt64(key + "max_part_number", max_part_number); - max_single_part_upload_size = config.getUInt64(key + "max_single_part_upload_size", max_single_part_upload_size); - max_single_operation_copy_size = config.getUInt64(key + "max_single_operation_copy_size", max_single_operation_copy_size); - - /// This configuration is only applicable to s3. Other types of object storage are not applicable or have different meanings. - storage_class_name = config.getString(config_prefix + ".s3_storage_class", storage_class_name); - storage_class_name = Poco::toUpperInPlace(storage_class_name); - - if (validate_settings) - validate(); -} - -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedCollection & collection) -{ - strict_upload_part_size = collection.getOrDefault("strict_upload_part_size", strict_upload_part_size); - min_upload_part_size = collection.getOrDefault("min_upload_part_size", min_upload_part_size); - max_single_part_upload_size = collection.getOrDefault("max_single_part_upload_size", max_single_part_upload_size); - upload_part_size_multiply_factor = collection.getOrDefault("upload_part_size_multiply_factor", upload_part_size_multiply_factor); - upload_part_size_multiply_parts_count_threshold = collection.getOrDefault("upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); - max_inflight_parts_for_one_file = collection.getOrDefault("max_inflight_parts_for_one_file", max_inflight_parts_for_one_file); - - /// This configuration is only applicable to s3. Other types of object storage are not applicable or have different meanings. - storage_class_name = collection.getOrDefault("s3_storage_class", storage_class_name); - storage_class_name = Poco::toUpperInPlace(storage_class_name); - - validate(); -} - -void S3Settings::RequestSettings::PartUploadSettings::updateFromSettings(const Settings & settings, bool if_changed) -{ - if (!if_changed || settings.s3_strict_upload_part_size.changed) - strict_upload_part_size = settings.s3_strict_upload_part_size; - - if (!if_changed || settings.s3_min_upload_part_size.changed) - min_upload_part_size = settings.s3_min_upload_part_size; - - if (!if_changed || settings.s3_max_upload_part_size.changed) - max_upload_part_size = settings.s3_max_upload_part_size; - - if (!if_changed || settings.s3_upload_part_size_multiply_factor.changed) - upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; - - if (!if_changed || settings.s3_upload_part_size_multiply_parts_count_threshold.changed) - upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; - - if (!if_changed || settings.s3_max_inflight_parts_for_one_file.changed) - max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; - - if (!if_changed || settings.s3_max_single_part_upload_size.changed) - max_single_part_upload_size = settings.s3_max_single_part_upload_size; -} - -void S3Settings::RequestSettings::PartUploadSettings::validate() -{ - static constexpr size_t min_upload_part_size_limit = 5 * 1024 * 1024; - if (strict_upload_part_size && strict_upload_part_size < min_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting strict_upload_part_size has invalid value {} which is less than the s3 API limit {}", - ReadableSize(strict_upload_part_size), ReadableSize(min_upload_part_size_limit)); - - if (min_upload_part_size < min_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting min_upload_part_size has invalid value {} which is less than the s3 API limit {}", - ReadableSize(min_upload_part_size), ReadableSize(min_upload_part_size_limit)); - - static constexpr size_t max_upload_part_size_limit = 5ull * 1024 * 1024 * 1024; - if (max_upload_part_size > max_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_upload_part_size has invalid value {} which is greater than the s3 API limit {}", - ReadableSize(max_upload_part_size), ReadableSize(max_upload_part_size_limit)); - - if (max_single_part_upload_size > max_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_single_part_upload_size has invalid value {} which is grater than the s3 API limit {}", - ReadableSize(max_single_part_upload_size), ReadableSize(max_upload_part_size_limit)); - - if (max_single_operation_copy_size > max_upload_part_size_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_single_operation_copy_size has invalid value {} which is grater than the s3 API limit {}", - ReadableSize(max_single_operation_copy_size), ReadableSize(max_upload_part_size_limit)); - - if (max_upload_part_size < min_upload_part_size) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_upload_part_size ({}) can't be less than setting min_upload_part_size {}", - ReadableSize(max_upload_part_size), ReadableSize(min_upload_part_size)); - - if (!upload_part_size_multiply_factor) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting upload_part_size_multiply_factor cannot be zero"); - - if (!upload_part_size_multiply_parts_count_threshold) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting upload_part_size_multiply_parts_count_threshold cannot be zero"); - - if (!max_part_number) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_part_number cannot be zero"); - - static constexpr size_t max_part_number_limit = 10000; - if (max_part_number > max_part_number_limit) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_part_number has invalid value {} which is grater than the s3 API limit {}", - ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); - - size_t maybe_overflow; - if (common::mulOverflow(max_upload_part_size, upload_part_size_multiply_factor, maybe_overflow)) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting upload_part_size_multiply_factor is too big ({}). " - "Multiplication to max_upload_part_size ({}) will cause integer overflow", - ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); - - std::unordered_set storage_class_names {"STANDARD", "INTELLIGENT_TIERING"}; - if (!storage_class_name.empty() && !storage_class_names.contains(storage_class_name)) - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Setting storage_class has invalid value {} which only supports STANDARD and INTELLIGENT_TIERING", - storage_class_name); - - /// TODO: it's possible to set too small limits. We can check that max possible object size is not too small. -} - - -S3Settings::RequestSettings::RequestSettings(const Settings & settings, bool validate_settings) - : upload_settings(settings, validate_settings) -{ - updateFromSettingsImpl(settings, false); -} - -S3Settings::RequestSettings::RequestSettings(const NamedCollection & collection) - : upload_settings(collection) -{ - max_single_read_retries = collection.getOrDefault("max_single_read_retries", max_single_read_retries); - max_connections = collection.getOrDefault("max_connections", max_connections); - list_object_keys_size = collection.getOrDefault("list_object_keys_size", list_object_keys_size); - allow_native_copy = collection.getOrDefault("allow_native_copy", allow_native_copy); - throw_on_zero_files_match = collection.getOrDefault("throw_on_zero_files_match", throw_on_zero_files_match); -} - -S3Settings::RequestSettings::RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix, - bool validate_settings) - : upload_settings(config, config_prefix, settings, setting_name_prefix, validate_settings) -{ - String key = config_prefix + "." + setting_name_prefix; - max_single_read_retries = config.getUInt64(key + "max_single_read_retries", settings.s3_max_single_read_retries); - max_connections = config.getUInt64(key + "max_connections", settings.s3_max_connections); - check_objects_after_upload = config.getBool(key + "check_objects_after_upload", settings.s3_check_objects_after_upload); - list_object_keys_size = config.getUInt64(key + "list_object_keys_size", settings.s3_list_object_keys_size); - allow_native_copy = config.getBool(key + "allow_native_copy", allow_native_copy); - throw_on_zero_files_match = config.getBool(key + "throw_on_zero_files_match", settings.s3_throw_on_zero_files_match); - retry_attempts = config.getUInt64(key + "retry_attempts", settings.s3_retry_attempts); - request_timeout_ms = config.getUInt64(key + "request_timeout_ms", settings.s3_request_timeout_ms); - - /// NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, - /// which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = config.getUInt64(key + "max_get_rps", settings.s3_max_get_rps)) - { - size_t default_max_get_burst = settings.s3_max_get_burst - ? settings.s3_max_get_burst - : (Throttler::default_burst_seconds * max_get_rps); - - size_t max_get_burst = config.getUInt64(key + "max_get_burst", default_max_get_burst); - - get_request_throttler = std::make_shared(max_get_rps, max_get_burst); - } - if (UInt64 max_put_rps = config.getUInt64(key + "max_put_rps", settings.s3_max_put_rps)) - { - size_t default_max_put_burst = settings.s3_max_put_burst - ? settings.s3_max_put_burst - : (Throttler::default_burst_seconds * max_put_rps); - - size_t max_put_burst = config.getUInt64(key + "max_put_burst", default_max_put_burst); - - put_request_throttler = std::make_shared(max_put_rps, max_put_burst); - } -} - -void S3Settings::RequestSettings::updateFromSettingsImpl(const Settings & settings, bool if_changed) -{ - if (!if_changed || settings.s3_max_single_read_retries.changed) - max_single_read_retries = settings.s3_max_single_read_retries; - - if (!if_changed || settings.s3_max_connections.changed) - max_connections = settings.s3_max_connections; - - if (!if_changed || settings.s3_check_objects_after_upload.changed) - check_objects_after_upload = settings.s3_check_objects_after_upload; - - if (!if_changed || settings.s3_max_unexpected_write_error_retries.changed) - max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; - - if (!if_changed || settings.s3_list_object_keys_size.changed) - list_object_keys_size = settings.s3_list_object_keys_size; - - if ((!if_changed || settings.s3_max_get_rps.changed || settings.s3_max_get_burst.changed) && settings.s3_max_get_rps) - get_request_throttler = std::make_shared( - settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); - - if ((!if_changed || settings.s3_max_put_rps.changed || settings.s3_max_put_burst.changed) && settings.s3_max_put_rps) - put_request_throttler = std::make_shared( - settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); - - if (!if_changed || settings.s3_throw_on_zero_files_match.changed) - throw_on_zero_files_match = settings.s3_throw_on_zero_files_match; - - if (!if_changed || settings.s3_retry_attempts.changed) - retry_attempts = settings.s3_retry_attempts; - - if (!if_changed || settings.s3_request_timeout_ms.changed) - request_timeout_ms = settings.s3_request_timeout_ms; -} - -void S3Settings::RequestSettings::updateFromSettingsIfChanged(const Settings & settings) -{ - updateFromSettingsImpl(settings, true); - upload_settings.updateFromSettings(settings, true); -} - -void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings) -{ - std::lock_guard lock(mutex); - s3_settings.clear(); - if (!config.has(config_elem)) - return; - - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(config_elem, config_keys); - - for (const String & key : config_keys) - { - if (config.has(config_elem + "." + key + ".endpoint")) - { - auto endpoint = config.getString(config_elem + "." + key + ".endpoint"); - auto auth_settings = S3::AuthSettings::loadFromConfig(config_elem + "." + key, config); - S3Settings::RequestSettings request_settings(config, config_elem + "." + key, settings); - - s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); - } - } -} - -std::optional StorageS3Settings::getSettings(const String & endpoint, const String & user, bool ignore_user) const -{ - std::lock_guard lock(mutex); - auto next_prefix_setting = s3_settings.upper_bound(endpoint); - - /// Linear time algorithm may be replaced with logarithmic with prefix tree map. - for (auto possible_prefix_setting = next_prefix_setting; possible_prefix_setting != s3_settings.begin();) - { - std::advance(possible_prefix_setting, -1); - const auto & [endpoint_prefix, settings] = *possible_prefix_setting; - if (endpoint.starts_with(endpoint_prefix) && (ignore_user || settings.auth_settings.canBeUsedByUser(user))) - return possible_prefix_setting->second; - } - - return {}; -} - -} diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h deleted file mode 100644 index c3bc8aa6ed6..00000000000 --- a/src/Storages/StorageS3Settings.h +++ /dev/null @@ -1,122 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace Poco::Util -{ -class AbstractConfiguration; -} - -namespace DB -{ - -struct Settings; -class NamedCollection; - -struct S3Settings -{ - struct RequestSettings - { - struct PartUploadSettings - { - size_t strict_upload_part_size = 0; - size_t min_upload_part_size = 16 * 1024 * 1024; - size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t upload_part_size_multiply_factor = 2; - size_t upload_part_size_multiply_parts_count_threshold = 500; - size_t max_inflight_parts_for_one_file = 20; - size_t max_part_number = 10000; - size_t max_single_part_upload_size = 32 * 1024 * 1024; - size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; - String storage_class_name; - - void updateFromSettings(const Settings & settings, bool if_changed); - void validate(); - - private: - PartUploadSettings() = default; - explicit PartUploadSettings(const Settings & settings, bool validate_settings = true); - explicit PartUploadSettings(const NamedCollection & collection); - PartUploadSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix = {}, - bool validate_settings = true); - - friend struct RequestSettings; - }; - - private: - PartUploadSettings upload_settings = {}; - - public: - size_t max_single_read_retries = 4; - size_t max_connections = 1024; - bool check_objects_after_upload = false; - size_t max_unexpected_write_error_retries = 4; - size_t list_object_keys_size = 1000; - ThrottlerPtr get_request_throttler; - ThrottlerPtr put_request_throttler; - size_t retry_attempts = 10; - size_t request_timeout_ms = 30000; - bool allow_native_copy = true; - - bool throw_on_zero_files_match = false; - - const PartUploadSettings & getUploadSettings() const { return upload_settings; } - PartUploadSettings & getUploadSettings() { return upload_settings; } - - void setStorageClassName(const String & storage_class_name) { upload_settings.storage_class_name = storage_class_name; } - - RequestSettings() = default; - explicit RequestSettings(const Settings & settings, bool validate_settings = true); - explicit RequestSettings(const NamedCollection & collection); - - /// What's the setting_name_prefix, and why do we need it? - /// There are (at least) two config sections where s3 settings can be specified: - /// * settings for s3 disk (clickhouse/storage_configuration/disks) - /// * settings for s3 storage (clickhouse/s3), which are also used for backups - /// Even though settings are the same, in case of s3 disk they are prefixed with "s3_" - /// ("s3_max_single_part_upload_size"), but in case of s3 storage they are not - /// ( "max_single_part_upload_size"). Why this happened is a complete mystery to me. - RequestSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Settings & settings, - String setting_name_prefix = {}, - bool validate_settings = true); - - void updateFromSettingsIfChanged(const Settings & settings); - - private: - void updateFromSettingsImpl(const Settings & settings, bool if_changed); - }; - - S3::AuthSettings auth_settings; - RequestSettings request_settings; -}; - -/// Settings for the StorageS3. -class StorageS3Settings -{ -public: - void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings); - - std::optional getSettings(const String & endpoint, const String & user, bool ignore_user = false) const; - -private: - mutable std::mutex mutex; - std::map s3_settings; -}; - -} diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 272f771194d..f8424bc3d1b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -411,7 +411,12 @@ Chunk StorageURLSource::generate() if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, curr_uri.getPath(), current_file_size); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = curr_uri.getPath(), + .size = current_file_size + }); return chunk; } @@ -457,7 +462,7 @@ std::pair> StorageURLSource: const auto settings = context_->getSettings(); - auto proxy_config = getProxyConfiguration(http_method); + auto proxy_config = getProxyConfiguration(request_uri.getScheme()); try { @@ -543,10 +548,11 @@ StorageURLSink::StorageURLSink( std::string content_type = FormatFactory::instance().getContentType(format, context, format_settings); std::string content_encoding = toContentEncodingName(compression_method); - auto proxy_config = getProxyConfiguration(http_method); + auto poco_uri = Poco::URI(uri); + auto proxy_config = getProxyConfiguration(poco_uri.getScheme()); auto write_buffer = std::make_unique( - HTTPConnectionGroupType::STORAGE, Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config + HTTPConnectionGroupType::STORAGE, poco_uri, http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config ); const auto & settings = context->getSettingsRef(); @@ -603,7 +609,6 @@ void StorageURLSink::finalize() { writer->finalize(); writer->flush(); - write_buf->finalize(); } catch (...) { @@ -611,12 +616,14 @@ void StorageURLSink::finalize() release(); throw; } + + write_buf->finalize(); } void StorageURLSink::release() { writer.reset(); - write_buf->finalize(); + write_buf.reset(); } class PartitionedStorageURLSink : public PartitionedSink @@ -1327,6 +1334,7 @@ std::optional IStorageURLBase::tryGetLastModificationTime( .withBufSize(settings.max_read_buffer_size) .withRedirects(settings.max_http_get_redirects) .withHeaders(headers) + .withProxy(proxy_config) .create(credentials); return buf->tryGetLastModificationTime(); diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index cb8d5caa50c..160c8d6270e 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -54,6 +54,10 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co if (auto database_cluster = replicated->tryGetCluster()) writeCluster(res_columns, {name_and_database.first, database_cluster}, replicated->tryGetAreReplicasActive(database_cluster)); + + if (auto database_cluster = replicated->tryGetAllGroupsCluster()) + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); } } } diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 49da1eba9ec..8dd8d3b6154 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -298,7 +298,7 @@ private: ClientInfo::Interface client_info_interface; size_t db_table_num = 0; size_t total_tables; - std::shared_ptr access; + std::shared_ptr access; bool need_to_check_access_for_tables; String query_id; std::chrono::milliseconds lock_acquire_timeout; diff --git a/src/Storages/System/StorageSystemDashboards.cpp b/src/Storages/System/StorageSystemDashboards.cpp index 9682fbc74a1..5faa37d951e 100644 --- a/src/Storages/System/StorageSystemDashboards.cpp +++ b/src/Storages/System/StorageSystemDashboards.cpp @@ -212,6 +212,20 @@ FROM merge('system', '^asynchronous_metric_log') WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition' GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview" }, + { "title", "Concurrent network connections" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, + sum(CurrentMetric_TCPConnection) AS TCP_Connections, + sum(CurrentMetric_MySQLConnection) AS MySQL_Connections, + sum(CurrentMetric_HTTPConnection) AS HTTP_Connections +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t +ORDER BY t WITH FILL STEP {rounding:UInt32} )EOQ") } }, /// Default dashboard for ClickHouse Cloud @@ -349,6 +363,11 @@ ORDER BY t WITH FILL STEP {rounding:UInt32} { "dashboard", "Cloud overview" }, { "title", "Network send bytes/sec" }, { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM (\n SELECT event_time, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview" }, + { "title", "Concurrent network connections" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(TCP_Connections), max(MySQL_Connections), max(HTTP_Connections) FROM (SELECT event_time, sum(CurrentMetric_TCPConnection) AS TCP_Connections, sum(CurrentMetric_MySQLConnection) AS MySQL_Connections, sum(CurrentMetric_HTTPConnection) AS HTTP_Connections FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } } }; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 0836560dff0..e98ea155f30 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -33,7 +33,7 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte { const auto & access = context->getAccess(); - NamedCollectionUtils::loadIfNot(); + NamedCollectionFactory::instance().loadIfNot(); auto collections = NamedCollectionFactory::instance().getAll(); for (const auto & [name, collection] : collections) diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index a6bb7da2b6e..637182067f2 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -45,29 +45,27 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co { for (const auto & [zookeeper_path, metadata] : S3QueueMetadataFactory::instance().getAll()) { - for (const auto & [file_name, file_status] : metadata->getFileStateses()) + for (const auto & [file_name, file_status] : metadata->getFileStatuses()) { size_t i = 0; res_columns[i++]->insert(zookeeper_path); res_columns[i++]->insert(file_name); - std::lock_guard lock(file_status->metadata_lock); - res_columns[i++]->insert(file_status->processed_rows.load()); - res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); + res_columns[i++]->insert(magic_enum::enum_name(file_status->state.load())); if (file_status->processing_start_time) - res_columns[i++]->insert(file_status->processing_start_time); + res_columns[i++]->insert(file_status->processing_start_time.load()); else res_columns[i++]->insertDefault(); if (file_status->processing_end_time) - res_columns[i++]->insert(file_status->processing_end_time); + res_columns[i++]->insert(file_status->processing_end_time.load()); else res_columns[i++]->insertDefault(); ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); - res_columns[i++]->insert(file_status->last_exception); + res_columns[i++]->insert(file_status->getException()); } } } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 651ca815420..339a59e88a5 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -12,7 +12,6 @@ #include #include #include -#include "Common/Scheduler/ResourceRequest.h" namespace DB diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 2e848f68850..ef10b2f45da 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -81,7 +81,10 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context {"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, {"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, {"index_uncompressed_cache_size", {std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, - {"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}} + {"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + + {"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}}, + {"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}} }; if (context->areBackgroundExecutorsInitialized()) diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index a48b109fbbe..09a2bb5d963 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,9 @@ namespace struct ZerosState { + explicit ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; + std::atomic add_total_rows = 0; }; using ZerosStatePtr = std::shared_ptr; @@ -42,10 +44,13 @@ protected: auto column_ptr = column; size_t column_size = column_ptr->size(); - if (state) + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + if (limit) { auto generated_rows = state->num_generated_rows.fetch_add(column_size, std::memory_order_acquire); - if (generated_rows >= limit) return {}; @@ -103,36 +108,25 @@ Pipe StorageSystemZeros::read( { storage_snapshot->check(column_names); - bool use_multiple_streams = multithreaded; + UInt64 query_limit = limit ? *limit : 0; + if (query_info.limit) + query_limit = query_limit ? std::min(query_limit, query_info.limit) : query_info.limit; - if (limit && *limit < max_block_size) - { - max_block_size = static_cast(*limit); - use_multiple_streams = false; - } + if (query_limit && query_limit < max_block_size) + max_block_size = query_limit; - if (!use_multiple_streams) + if (!multithreaded) num_streams = 1; + else if (query_limit && num_streams * max_block_size > query_limit) + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); + + ZerosStatePtr state = std::make_shared(query_limit); Pipe res; - - ZerosStatePtr state; - - if (limit) - state = std::make_shared(); - for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); - - if (i == 0) - { - if (limit) - source->addTotalRowsApprox(*limit); - else if (query_info.limit) - source->addTotalRowsApprox(query_info.limit); - } - + auto source = std::make_shared(max_block_size, query_limit, state); res.addSource(std::move(source)); } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 6e7ea32ee59..f831465277d 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -166,8 +166,14 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType { ExpressionAndSets result; auto ttl_string = queryToString(ast); - auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns); - ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context); + auto context_copy = Context::createCopy(context); + /// FIXME All code here will work with old analyzer, however for TTL + /// with subqueries it's possible that new analyzer will be enabled in ::read method + /// of underlying storage when all other parts of infra are not ready for it + /// (built with old analyzer). + context_copy->setSetting("allow_experimental_analyzer", Field{0}); + auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns); + ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); const auto * col = &dag->findInOutputs(ast->getColumnName()); @@ -177,7 +183,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType dag->getOutputs() = {col}; dag->removeUnusedActions(); - result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context)); + result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); return result; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index cec55cefda2..778c9e13adb 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -111,7 +112,7 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) NameSet getVirtualNamesForFileLikeStorage() { - return {"_path", "_file", "_size"}; + return {"_path", "_file", "_size", "_time"}; } VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) @@ -129,6 +130,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_path", std::make_shared(std::make_shared())); add_virtual("_file", std::make_shared(std::make_shared())); add_virtual("_size", makeNullable(std::make_shared())); + add_virtual("_time", makeNullable(std::make_shared())); return desc; } @@ -187,32 +189,40 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename) +void addRequestedFileLikeStorageVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, + VirtualsForFileLikeStorage virtual_values) { for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), path)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), virtual_values.path)->convertToFullColumnIfConst()); } else if (virtual_column.name == "_file") { - if (filename) + if (virtual_values.filename) { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *filename)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), (*virtual_values.filename))->convertToFullColumnIfConst()); } else { - size_t last_slash_pos = path.find_last_of('/'); - auto filename_from_path = path.substr(last_slash_pos + 1); + size_t last_slash_pos = virtual_values.path.find_last_of('/'); + auto filename_from_path = virtual_values.path.substr(last_slash_pos + 1); chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), filename_from_path)->convertToFullColumnIfConst()); } } else if (virtual_column.name == "_size") { - if (size) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *size)->convertToFullColumnIfConst()); + if (virtual_values.size) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *virtual_values.size)->convertToFullColumnIfConst()); + else + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); + } + else if (virtual_column.name == "_time") + { + if (virtual_values.last_modified) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), virtual_values.last_modified->epochTime())->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 62f2e4855b5..fbfbdd6c6cc 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -68,8 +68,18 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa sources = std::move(filtered_sources); } -void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr); +struct VirtualsForFileLikeStorage +{ + const String & path; + std::optional size { std::nullopt }; + const String * filename { nullptr }; + std::optional last_modified { std::nullopt }; + +}; + +void addRequestedFileLikeStorageVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, + VirtualsForFileLikeStorage virtual_values); } } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8bca1c97aad..a6e7ce301a0 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include #include #include @@ -633,7 +633,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) }); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); @@ -1532,7 +1532,7 @@ void StorageWindowView::writeIntoWindowView( builder = select_block.buildQueryPipeline(); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, local_context->getSettingsRef().min_insert_block_size_rows, local_context->getSettingsRef().min_insert_block_size_bytes); diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 131712e750a..ed378169381 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include @@ -290,7 +290,7 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, size_t min_block_size_rows = mutable_context->getSettingsRef().min_external_table_block_size_rows; size_t min_block_size_bytes = mutable_context->getSettingsRef().min_external_table_block_size_bytes; - auto squashing = std::make_shared(builder->getHeader(), min_block_size_rows, min_block_size_bytes); + auto squashing = std::make_shared(builder->getHeader(), min_block_size_rows, min_block_size_bytes); builder->resize(1); builder->addTransform(std::move(squashing)); 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 7e2d393c3d1..6765e112bb9 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -368,17 +368,21 @@ TEST(TransformQueryForExternalDatabase, Null) check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NULL", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)", + R"(SELECT "field" FROM "test"."table" WHERE 1 = 0)"); check(state, 1, {"field"}, "SELECT field FROM table WHERE field IS NOT NULL", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)", + R"(SELECT "field" FROM "test"."table")"); check(state, 1, {"field"}, "SELECT field FROM table WHERE isNull(field)", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NULL)", + R"(SELECT "field" FROM "test"."table" WHERE 1 = 0)"); check(state, 1, {"field"}, "SELECT field FROM table WHERE isNotNull(field)", - R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)"); + R"(SELECT "field" FROM "test"."table" WHERE "field" IS NOT NULL)", + R"(SELECT "field" FROM "test"."table")"); } TEST(TransformQueryForExternalDatabase, ToDate) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 5f798158a41..1f7357b6494 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -9,3 +9,5 @@ 01287_max_execution_speed # Check after ConstantNode refactoring 02154_parser_backtracking +02944_variant_as_common_type +02942_variant_cast diff --git a/tests/ci/artifacts_helper.py b/tests/ci/artifacts_helper.py index 37abf0bdefb..503ba2e1ec4 100644 --- a/tests/ci/artifacts_helper.py +++ b/tests/ci/artifacts_helper.py @@ -15,7 +15,7 @@ from github.Commit import Commit from build_download_helper import download_build_with_progress from commit_status_helper import post_commit_status from compress_files import SUFFIX, compress_fast, decompress_fast -from env_helper import CI, RUNNER_TEMP, S3_BUILDS_BUCKET +from env_helper import IS_CI, RUNNER_TEMP, S3_BUILDS_BUCKET from git_helper import SHA_REGEXP from report import FOOTER_HTML_TEMPLATE, HEAD_HTML_TEMPLATE, SUCCESS from s3_helper import S3Helper @@ -131,7 +131,7 @@ class ArtifactsHelper: post_commit_status(commit, SUCCESS, url, "Artifacts for workflow", "Artifacts") def _regenerate_index(self) -> None: - if CI: + if IS_CI: files = self._get_s3_objects() else: files = self._get_local_s3_objects() diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index b88a9476a6d..8bc0f51dfc7 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -6,7 +6,7 @@ import subprocess import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH @@ -14,6 +14,7 @@ from pr_info import PRInfo from report import FAIL, FAILURE, OK, SUCCESS, JobReport, TestResult from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI IMAGE_NAME = "clickhouse/fuzzer" @@ -64,7 +65,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) if not urls: raise ValueError("No build URLs found") diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index d41fdaf05ff..71b18572938 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -7,7 +7,7 @@ import sys from pathlib import Path from typing import List, Sequence, Tuple -from ci_config import JobNames +from ci_config import CI from ci_utils import normalize_string from env_helper import TEMP_PATH from functional_test_check import NO_CHANGES_MSG @@ -92,16 +92,19 @@ def main(): logging.basicConfig(level=logging.INFO) # args = parse_args() stopwatch = Stopwatch() - jobs_to_validate = [JobNames.STATELESS_TEST_RELEASE, JobNames.INTEGRATION_TEST] + jobs_to_validate = [ + CI.JobNames.STATELESS_TEST_RELEASE, + CI.JobNames.INTEGRATION_TEST, + ] functional_job_report_file = Path(TEMP_PATH) / "functional_test_job_report.json" integration_job_report_file = Path(TEMP_PATH) / "integration_test_job_report.json" jobs_report_files = { - JobNames.STATELESS_TEST_RELEASE: functional_job_report_file, - JobNames.INTEGRATION_TEST: integration_job_report_file, + CI.JobNames.STATELESS_TEST_RELEASE: functional_job_report_file, + CI.JobNames.INTEGRATION_TEST: integration_job_report_file, } jobs_scripts = { - JobNames.STATELESS_TEST_RELEASE: "functional_test_check.py", - JobNames.INTEGRATION_TEST: "integration_test_check.py", + CI.JobNames.STATELESS_TEST_RELEASE: "functional_test_check.py", + CI.JobNames.INTEGRATION_TEST: "integration_test_check.py", } for test_job in jobs_to_validate: diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 260b77b0ee5..39f34ed9ccf 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -9,7 +9,7 @@ from pathlib import Path from typing import Tuple import docker_images_helper -from ci_config import CI_CONFIG, BuildConfig +from ci_config import CI from env_helper import REPO_COPY, S3_BUILDS_BUCKET, TEMP_PATH from git_helper import Git from lambda_shared_package.lambda_shared.pr import Labels @@ -27,7 +27,7 @@ IMAGE_NAME = "clickhouse/binary-builder" BUILD_LOG_NAME = "build_log.log" -def _can_export_binaries(build_config: BuildConfig) -> bool: +def _can_export_binaries(build_config: CI.BuildConfig) -> bool: if build_config.package_type != "deb": return False if build_config.sanitizer != "": @@ -38,7 +38,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: def get_packager_cmd( - build_config: BuildConfig, + build_config: CI.BuildConfig, packager_path: Path, output_path: Path, build_version: str, @@ -147,7 +147,8 @@ def main(): stopwatch = Stopwatch() build_name = args.build_name - build_config = CI_CONFIG.build_config[build_name] + build_config = CI.JOB_CONFIGS[build_name].build_config + assert build_config temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 0f6c8e5aa8a..8482abb26e0 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -10,11 +10,25 @@ from typing import Any, Callable, List, Optional, Union import requests -import get_robot_token as grt # we need an updated ROBOT_TOKEN -from ci_config import CI_CONFIG +from ci_config import CI + +try: + # A work around for scripts using this downloading module without required deps + import get_robot_token as grt # we need an updated ROBOT_TOKEN +except ImportError: + + class grt: # type: ignore + ROBOT_TOKEN = None + + @staticmethod + def get_best_robot_token() -> str: + return "" + DOWNLOAD_RETRIES_COUNT = 5 +logger = logging.getLogger(__name__) + class DownloadException(Exception): pass @@ -30,7 +44,7 @@ def get_with_retries( sleep: int = 3, **kwargs: Any, ) -> requests.Response: - logging.info( + logger.info( "Getting URL with %i tries and sleep %i in between: %s", retries, sleep, url ) exc = Exception("A placeholder to satisfy typing and avoid nesting") @@ -42,7 +56,7 @@ def get_with_retries( return response except Exception as e: if i + 1 < retries: - logging.info("Exception '%s' while getting, retry %i", e, i + 1) + logger.info("Exception '%s' while getting, retry %i", e, i + 1) time.sleep(sleep) exc = e @@ -63,15 +77,10 @@ def get_gh_api( """ def set_auth_header(): - if "headers" in kwargs: - if "Authorization" not in kwargs["headers"]: - kwargs["headers"][ - "Authorization" - ] = f"Bearer {grt.get_best_robot_token()}" - else: - kwargs["headers"] = { - "Authorization": f"Bearer {grt.get_best_robot_token()}" - } + headers = kwargs.get("headers", {}) + if "Authorization" not in headers: + headers["Authorization"] = f"Bearer {grt.get_best_robot_token()}" + kwargs["headers"] = headers if grt.ROBOT_TOKEN is not None: set_auth_header() @@ -96,7 +105,7 @@ def get_gh_api( ) try_auth = e.response.status_code == 404 if (ratelimit_exceeded or try_auth) and not token_is_set: - logging.warning( + logger.warning( "Received rate limit exception, setting the auth header and retry" ) set_auth_header() @@ -107,39 +116,35 @@ def get_gh_api( exc = e if try_cnt < retries: - logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) + logger.info("Exception '%s' while getting, retry %i", exc, try_cnt) time.sleep(sleep) - raise APIException("Unable to request data from GH API") from exc - - -def get_build_name_for_check(check_name: str) -> str: - return CI_CONFIG.test_configs[check_name].required_build + raise APIException(f"Unable to request data from GH API: {url}") from exc def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]: for root, _, files in os.walk(reports_path): for file in files: if file.endswith(f"_{build_name}.json"): - logging.info("Found build report json %s for %s", file, build_name) + logger.info("Found build report json %s for %s", file, build_name) with open( os.path.join(root, file), "r", encoding="utf-8" ) as file_handler: build_report = json.load(file_handler) return build_report["build_urls"] # type: ignore - logging.info("A build report is not found for %s", build_name) + logger.info("A build report is not found for %s", build_name) return [] def download_build_with_progress(url: str, path: Path) -> None: - logging.info("Downloading from %s to temp path %s", url, path) + logger.info("Downloading from %s to temp path %s", url, path) for i in range(DOWNLOAD_RETRIES_COUNT): try: response = get_with_retries(url, retries=1, stream=True) total_length = int(response.headers.get("content-length", 0)) if path.is_file() and total_length and path.stat().st_size == total_length: - logging.info( + logger.info( "The file %s already exists and have a proper size %s", path, total_length, @@ -148,14 +153,14 @@ def download_build_with_progress(url: str, path: Path) -> None: with open(path, "wb") as f: if total_length == 0: - logging.info( + logger.info( "No content-length, will download file without progress" ) f.write(response.content) else: dl = 0 - logging.info("Content length is %ld bytes", total_length) + logger.info("Content length is %ld bytes", total_length) for data in response.iter_content(chunk_size=4096): dl += len(data) f.write(data) @@ -170,8 +175,8 @@ def download_build_with_progress(url: str, path: Path) -> None: except Exception as e: if sys.stdout.isatty(): sys.stdout.write("\n") - if os.path.exists(path): - os.remove(path) + if path.exists(): + path.unlink() if i + 1 < DOWNLOAD_RETRIES_COUNT: time.sleep(3) @@ -182,7 +187,7 @@ def download_build_with_progress(url: str, path: Path) -> None: if sys.stdout.isatty(): sys.stdout.write("\n") - logging.info("Downloading finished") + logger.info("Downloading finished") def download_builds( @@ -191,7 +196,7 @@ def download_builds( for url in build_urls: if filter_fn(url): fname = os.path.basename(url.replace("%2B", "+").replace("%20", " ")) - logging.info("Will download %s to %s", fname, result_path) + logger.info("Will download %s to %s", fname, result_path) download_build_with_progress(url, result_path / fname) @@ -201,9 +206,9 @@ def download_builds_filter( result_path: Path, filter_fn: Callable[[str], bool] = lambda _: True, ) -> None: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) - logging.info("The build report for %s contains the next URLs: %s", build_name, urls) + logger.info("The build report for %s contains the next URLs: %s", build_name, urls) if not urls: raise DownloadException("No build URLs found") @@ -238,9 +243,9 @@ def download_clickhouse_binary( def get_clickhouse_binary_url( check_name: str, reports_path: Union[Path, str] ) -> Optional[str]: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) - logging.info("The build report for %s contains the next URLs: %s", build_name, urls) + logger.info("The build report for %s contains the next URLs: %s", build_name, urls) for url in urls: check_url = url if "?" in check_url: diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 1d734fbb3f8..664f6a7cbb9 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -1,4 +1,5 @@ #!/usr/bin/env python3 +import argparse import json import logging import os @@ -6,7 +7,6 @@ import sys from pathlib import Path from typing import List -from ci_config import CI_CONFIG, Build from env_helper import ( GITHUB_JOB_URL, GITHUB_REPOSITORY, @@ -14,7 +14,7 @@ from env_helper import ( REPORT_PATH, TEMP_PATH, CI_CONFIG_PATH, - CI, + IS_CI, ) from pr_info import PRInfo from report import ( @@ -25,8 +25,10 @@ from report import ( JobReport, create_build_html_report, get_worst_status, + FAILURE, ) from stopwatch import Stopwatch +from ci_config import CI # Old way to read the neads_data NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "") @@ -46,16 +48,13 @@ def main(): "\n ".join(p.as_posix() for p in reports_path.rglob("*.json")), ) - build_check_name = sys.argv[1] + build_check_name = CI.JobNames.BUILD_CHECK pr_info = PRInfo() - builds_for_check = CI_CONFIG.get_builds_for_report( - build_check_name, - release=pr_info.is_release, - backport=pr_info.head_ref.startswith("backport/"), - ) - if CI: + args = parse_args() + + if (CI_CONFIG_PATH or IS_CI) and not args.reports: # In CI only specific builds might be manually selected, or some wf does not build all builds. # Filtering @builds_for_check to verify only builds that are present in the current CI workflow with open(CI_CONFIG_PATH, encoding="utf-8") as jfd: @@ -64,8 +63,12 @@ def main(): ci_config["jobs_data"]["jobs_to_skip"] + ci_config["jobs_data"]["jobs_to_do"] ) - builds_for_check = [job for job in builds_for_check if job in all_ci_jobs] - print(f"NOTE: following build reports will be accounted: [{builds_for_check}]") + builds_for_check = [job for job in CI.BuildNames if job in all_ci_jobs] + print(f"NOTE: following build reports will be checked: [{builds_for_check}]") + else: + builds_for_check = parse_args().reports + for job in builds_for_check: + assert job in CI.BuildNames, "Builds must be known build job names" required_builds = len(builds_for_check) missing_builds = 0 @@ -77,8 +80,8 @@ def main(): build_name, pr_info.number, pr_info.head_ref ) if not build_result: - if build_name == Build.FUZZERS: - logging.info("Build [%s] is missing - skip", Build.FUZZERS) + if build_name == CI.BuildNames.FUZZERS: + logging.info("Build [%s] is missing - skip", CI.BuildNames.FUZZERS) continue logging.warning("Build results for %s is missing", build_name) build_result = BuildResult.missing_result("missing") @@ -132,17 +135,16 @@ def main(): # Check if there are no builds at all, do not override bad status if summary_status == SUCCESS: if missing_builds: - summary_status = PENDING + summary_status = FAILURE elif ok_groups == 0: summary_status = ERROR - addition = "" - if missing_builds: - addition = ( - f" ({required_builds - missing_builds} of {required_builds} builds are OK)" - ) + description = "" - description = f"{ok_groups}/{total_groups} artifact groups are OK{addition}" + if missing_builds: + description = f"{missing_builds} of {required_builds} builds are missing." + + description += f" {ok_groups}/{total_groups} artifact groups are OK" JobReport( description=description, @@ -158,5 +160,16 @@ def main(): sys.exit(1) +def parse_args(): + parser = argparse.ArgumentParser("Generates overall build report") + + parser.add_argument( + "--reports", + nargs="+", + help="List of build reports to check", + ) + return parser.parse_args() + + if __name__ == "__main__": main() diff --git a/tests/ci/cache_utils.py b/tests/ci/cache_utils.py index a0692f4eff2..5a295fc66ca 100644 --- a/tests/ci/cache_utils.py +++ b/tests/ci/cache_utils.py @@ -197,7 +197,6 @@ class CargoCache(Cache): logging.info("Cache for Cargo.lock md5 %s will be uploaded", self.lock_hash) self._force_upload_cache = True self.directory.mkdir(parents=True, exist_ok=True) - return def upload(self): self._upload(f"{self.PREFIX}/{self.archive_name}", self._force_upload_cache) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py new file mode 100755 index 00000000000..fcb61d3f605 --- /dev/null +++ b/tests/ci/changelog.py @@ -0,0 +1,455 @@ +#!/usr/bin/env python3 +# In our CI this script runs in style-test containers + +import argparse +import logging +import re +from datetime import date, timedelta +from pathlib import Path +from subprocess import DEVNULL +from typing import Any, Dict, List, Optional, TextIO + +import tqdm # type: ignore +from github.GithubException import RateLimitExceededException, UnknownObjectException +from github.NamedUser import NamedUser +from thefuzz.fuzz import ratio # type: ignore + +from cache_utils import GitHubCache +from env_helper import TEMP_PATH +from git_helper import git_runner, is_shallow +from github_helper import GitHub, PullRequest, PullRequests, Repository +from s3_helper import S3Helper +from version_helper import ( + FILE_WITH_VERSION_PATH, + get_abs_path, + get_version_from_repo, + get_version_from_tag, +) + +# This array gives the preferred category order, and is also used to +# normalize category names. +# Categories are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there +# updated accordingly +categories_preferred_order = ( + "Backward Incompatible Change", + "New Feature", + "Performance Improvement", + "Improvement", + "Critical Bug Fix", + "Bug Fix", + "Build/Testing/Packaging Improvement", + "Other", +) + +FROM_REF = "" +TO_REF = "" +SHA_IN_CHANGELOG = [] # type: List[str] +gh = GitHub(create_cache_dir=False) +runner = git_runner + + +class Description: + def __init__( + self, number: int, user: NamedUser, html_url: str, entry: str, category: str + ): + self.number = number + self.html_url = html_url + self.user = gh.get_user_cached(user._rawData["login"]) # type: ignore + self.entry = entry + self.category = category + + @property + def formatted_entry(self) -> str: + # Substitute issue links. + # 1) issue number w/o markdown link + entry = re.sub( + r"([^[])#([0-9]{4,})", + r"\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)", + self.entry, + ) + # 2) issue URL w/o markdown link + # including #issuecomment-1 or #event-12 + entry = re.sub( + r"([^(])(https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})[-#a-z0-9]*)", + r"\1[#\3](\2)", + entry, + ) + # It's possible that we face a secondary rate limit. + # In this case we should sleep until we get it + while True: + try: + user_name = self.user.name if self.user.name else self.user.login + break + except UnknownObjectException: + user_name = self.user.login + break + except RateLimitExceededException: + gh.sleep_on_rate_limit() + return ( + f"* {entry} [#{self.number}]({self.html_url}) " + f"([{user_name}]({self.user.html_url}))." + ) + + # Sort PR descriptions by numbers + def __eq__(self, other: Any) -> bool: + if not isinstance(self, type(other)): + raise NotImplementedError + return bool(self.number == other.number) + + def __lt__(self, other: "Description") -> bool: + return self.number < other.number + + +def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: + descriptions = {} # type: Dict[str, List[Description]] + repos = {} # type: Dict[str, Repository] + for pr in prs: + # See https://github.com/PyGithub/PyGithub/issues/2202, + # obj._rawData doesn't spend additional API requests + # We'll save some requests + # pylint: disable=protected-access + repo_name = pr._rawData["base"]["repo"]["full_name"] + # pylint: enable=protected-access + if repo_name not in repos: + repos[repo_name] = pr.base.repo + in_changelog = False + merge_commit = pr.merge_commit_sha + if merge_commit is None: + logging.warning("PR %s does not have merge-commit, skipping", pr.number) + continue + + in_changelog = merge_commit in SHA_IN_CHANGELOG + if in_changelog: + desc = generate_description(pr, repos[repo_name]) + if desc: + if desc.category not in descriptions: + descriptions[desc.category] = [] + descriptions[desc.category].append(desc) + + for descs in descriptions.values(): + descs.sort() + + return descriptions + + +def parse_args() -> argparse.Namespace: + parser = argparse.ArgumentParser( + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + description="Generate a changelog in Markdown format between given tags. " + "It fetches all tags and unshallow the git repository automatically", + ) + parser.add_argument( + "-v", + "--verbose", + action="count", + default=0, + help="set the script verbosity, could be used multiple", + ) + parser.add_argument( + "--debug-helpers", + action="store_true", + help="add debug logging for git_helper and github_helper", + ) + parser.add_argument( + "--output", + type=argparse.FileType("w"), + default="-", + help="output file for changelog", + ) + parser.add_argument( + "--repo", + default="ClickHouse/ClickHouse", + help="a repository to query for pull-requests from GitHub", + ) + parser.add_argument( + "--jobs", + type=int, + default=10, + help="number of jobs to get pull-requests info from GitHub API", + ) + parser.add_argument( + "--gh-user-or-token", + help="user name or GH token to authenticate", + ) + parser.add_argument( + "--gh-password", + help="a password that should be used when user is given", + ) + parser.add_argument( + "--with-testing-tags", + action="store_true", + help="by default '*-testing' tags are ignored, this argument enables them too", + ) + parser.add_argument( + "--from", + dest="from_ref", + help="git ref for a starting point of changelog, by default is calculated " + "automatically to match a previous tag in history", + ) + parser.add_argument( + "to_ref", + metavar="TO_REF", + help="git ref for the changelog end", + ) + args = parser.parse_args() + return args + + +# This function mirrors the PR description checks in ClickhousePullRequestTrigger. +# Returns None if the PR should not be mentioned in changelog. +def generate_description(item: PullRequest, repo: Repository) -> Optional[Description]: + backport_number = item.number + if item.head.ref.startswith("backport/"): + branch_parts = item.head.ref.split("/") + if len(branch_parts) == 3: + try: + item = gh.get_pull_cached(repo, int(branch_parts[-1])) + except Exception as e: + logging.warning("unable to get backpoted PR, exception: %s", e) + else: + logging.warning( + "The branch %s doesn't match backport template, using PR %s as is", + item.head.ref, + item.number, + ) + description = item.body + # Don't skip empty lines because they delimit parts of description + lines = [x.strip() for x in (description.split("\n") if description else [])] + lines = [re.sub(r"\s+", " ", ln) for ln in lines] + + category = "" + entry = "" + + if lines: + i = 0 + while i < len(lines): + if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): + i += 1 + if i >= len(lines): + break + # Can have one empty line between header and the category itself. + # Filter it out. + if not lines[i]: + i += 1 + if i >= len(lines): + break + category = re.sub(r"^[-*\s]*", "", lines[i]) + i += 1 + elif re.match( + r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] + ): + i += 1 + # Can have one empty line between header and the entry itself. + # Filter it out. + if i < len(lines) and not lines[i]: + i += 1 + # All following lines until empty one are the changelog entry. + entry_lines = [] + while i < len(lines) and lines[i]: + entry_lines.append(lines[i]) + i += 1 + entry = " ".join(entry_lines) + else: + i += 1 + + # Remove excessive bullets from the entry. + if re.match(r"^[\-\*] ", entry): + entry = entry[2:] + + # Better style. + if re.match(r"^[a-z]", entry): + entry = entry.capitalize() + + if not category: + # Shouldn't happen, because description check in CI should catch such PRs. + # Fall through, so that it shows up in output and the user can fix it. + category = "NO CL CATEGORY" + + # Filter out documentations changelog before not-for-changelog + if re.match( + r"(?i)doc", + category, + ): + return None + + # Filter out the PR categories that are not for changelog. + if re.search( + r"(?i)((non|in|not|un)[-\s]*significant)|" + r"(not[ ]*for[ ]*changelog)|" + r"(changelog[ ]*entry[ ]*is[ ]*not[ ]*required)", + category, + ): + category = "NOT FOR CHANGELOG / INSIGNIFICANT" + entry = item.title + + # Normalize bug fixes + if re.match( + r"(?i)bug\Wfix", + category, + ): + category = "Bug Fix (user-visible misbehavior in an official stable release)" + + if backport_number != item.number: + entry = f"Backported in #{backport_number}: {entry}" + + if not entry: + # Shouldn't happen, because description check in CI should catch such PRs. + category = "NO CL ENTRY" + entry = "NO CL ENTRY: '" + item.title + "'" + + entry = entry.strip() + if entry[-1] != ".": + entry += "." + + for c in categories_preferred_order: + if ratio(category.lower(), c.lower()) >= 90: + category = c + break + + return Description(item.number, item.user, item.html_url, entry, category) + + +def write_changelog( + fd: TextIO, descriptions: Dict[str, List[Description]], year: int +) -> None: + to_commit = runner(f"git rev-parse {TO_REF}^{{}}")[:11] + from_commit = runner(f"git rev-parse {FROM_REF}^{{}}")[:11] + fd.write( + f"---\nsidebar_position: 1\nsidebar_label: {year}\n---\n\n" + f"# {year} Changelog\n\n" + f"### ClickHouse release {TO_REF} ({to_commit}) FIXME " + f"as compared to {FROM_REF} ({from_commit})\n\n" + ) + + seen_categories = [] # type: List[str] + for category in categories_preferred_order: + if category in descriptions: + seen_categories.append(category) + fd.write(f"#### {category}\n") + for desc in descriptions[category]: + fd.write(f"{desc.formatted_entry}\n") + + fd.write("\n") + + for category in sorted(descriptions): + if category not in seen_categories: + fd.write(f"#### {category}\n\n") + for desc in descriptions[category]: + fd.write(f"{desc.formatted_entry}\n") + + fd.write("\n") + + +def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool) -> None: + global FROM_REF, TO_REF + TO_REF = to_ref + + # Check TO_REF + runner.run(f"git rev-parse {TO_REF}") + + # Check from_ref + if from_ref is not None: + runner.run(f"git rev-parse {FROM_REF}") + FROM_REF = from_ref + return + + # Get the cmake/autogenerated_versions.txt from FROM_REF to read the version + # If the previous tag is greater than version in the FROM_REF, + # then we need to add it to tags_to_exclude + temp_cmake = "tests/ci/tmp/autogenerated_versions.txt" + cmake_version = get_abs_path(temp_cmake) + cmake_version.write_text(runner(f"git show {TO_REF}:{FILE_WITH_VERSION_PATH}")) + to_ref_version = get_version_from_repo(cmake_version) + # Get all tags pointing to TO_REF + excluded_tags = runner.run(f"git tag --points-at '{TO_REF}^{{}}'").split("\n") + logging.info("All tags pointing to %s:\n%s", TO_REF, excluded_tags) + if not with_testing_tags: + excluded_tags.append("*-testing") + while not from_ref: + exclude = " ".join([f"--exclude='{tag}'" for tag in excluded_tags]) + from_ref_tag = runner(f"git describe --abbrev=0 --tags {exclude} '{TO_REF}'") + from_ref_version = get_version_from_tag(from_ref_tag) + if from_ref_version <= to_ref_version: + from_ref = from_ref_tag + break + excluded_tags.append(from_ref_tag) + + cmake_version.unlink() + FROM_REF = from_ref + + +def set_sha_in_changelog(): + global SHA_IN_CHANGELOG + SHA_IN_CHANGELOG = runner.run( + f"git log --format=format:%H {FROM_REF}..{TO_REF}" + ).split("\n") + + +def get_year(prs: PullRequests) -> int: + if not prs: + return date.today().year + return max(pr.created_at.year for pr in prs) + + +def main(): + log_levels = [logging.WARN, logging.INFO, logging.DEBUG] + args = parse_args() + logging.basicConfig( + format="%(asctime)s %(levelname)-8s [%(filename)s:%(lineno)d]:\n%(message)s", + level=log_levels[min(args.verbose, 2)], + ) + if args.debug_helpers: + logging.getLogger("github_helper").setLevel(logging.DEBUG) + logging.getLogger("git_helper").setLevel(logging.DEBUG) + + # Get the full repo + if is_shallow(): + logging.info("Unshallow repository") + runner.run("git fetch --unshallow", stderr=DEVNULL) + logging.info("Fetching all tags") + runner.run("git fetch --tags", stderr=DEVNULL) + + check_refs(args.from_ref, args.to_ref, args.with_testing_tags) + set_sha_in_changelog() + + logging.info("Using %s..%s as changelog interval", FROM_REF, TO_REF) + + # use merge-base commit as a starting point, if used ref in another branch + base_commit = runner.run(f"git merge-base '{FROM_REF}^{{}}' '{TO_REF}^{{}}'") + # Get starting and ending dates for gathering PRs + # Add one day after and before to mitigate TZ possible issues + # `tag^{}` format gives commit ref when we have annotated tags + # format %cs gives a committer date, works better for cherry-picked commits + from_date = runner.run(f"git log -1 --format=format:%cs '{base_commit}'") + to_date = runner.run(f"git log -1 --format=format:%cs '{TO_REF}^{{}}'") + merged = ( + date.fromisoformat(from_date) - timedelta(1), + date.fromisoformat(to_date) + timedelta(1), + ) + + # Get all PRs for the given time frame + global gh + gh = GitHub( + args.gh_user_or_token, + args.gh_password, + create_cache_dir=False, + per_page=100, + pool_size=args.jobs, + ) + temp_path = Path(TEMP_PATH) + gh_cache = GitHubCache(gh.cache_path, temp_path, S3Helper()) + gh_cache.download() + query = f"type:pr repo:{args.repo} is:merged" + prs = gh.get_pulls_from_search( + query=query, merged=merged, sort="created", progress_func=tqdm.tqdm + ) + + descriptions = get_descriptions(prs) + changelog_year = get_year(prs) + + write_changelog(args.output, descriptions, changelog_year) + gh_cache.upload() + + +if __name__ == "__main__": + main() diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index e470621e2c5..459be12ada0 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -127,13 +127,11 @@ close it. to_pop.append(i) elif pr.head.ref.startswith(f"backport/{self.name}"): self.backport_pr = pr + self._backported = True to_pop.append(i) else: assert False, f"BUG! Invalid PR's branch [{pr.head.ref}]" - # Cherry-pick or backport PR found, set @backported flag for current release branch - self._backported = True - for i in reversed(to_pop): # Going from the tail to keep the order and pop greater index first prs.pop(i) @@ -218,6 +216,7 @@ close it. self.name, self.pr.number, ) + self._backported = True return except CalledProcessError: # There are most probably conflicts, they'll be resolved in PR @@ -247,7 +246,6 @@ close it. self.cherrypick_pr.add_to_labels(Labels.PR_CRITICAL_BUGFIX) elif Labels.PR_BUGFIX in [label.name for label in self.pr.labels]: self.cherrypick_pr.add_to_labels(Labels.PR_BUGFIX) - self._backported = True self._assign_new_pr(self.cherrypick_pr) # update cherrypick PR to get the state for PR.mergable self.cherrypick_pr.update() @@ -359,10 +357,10 @@ class Backport: self._fetch_from = fetch_from self.dry_run = dry_run - self.must_create_backport_label = ( - Labels.MUST_BACKPORT + self.must_create_backport_labels = ( + [Labels.MUST_BACKPORT] if self._repo_name == self._fetch_from - else Labels.MUST_BACKPORT_CLOUD + else [Labels.MUST_BACKPORT_CLOUD, Labels.MUST_BACKPORT] ) self.backport_created_label = ( Labels.PR_BACKPORTS_CREATED @@ -468,7 +466,7 @@ class Backport: query_args = { "query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", "label": ",".join( - self.labels_to_backport + [self.must_create_backport_label] + self.labels_to_backport + self.must_create_backport_labels ), "merged": [since_date, tomorrow], } @@ -492,7 +490,10 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if self.must_create_backport_label in pr_labels: + if ( + any(label in pr_labels for label in self.must_create_backport_labels) + or self._repo_name != self._fetch_from + ): branches = [ ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in self.release_branches @@ -531,9 +532,9 @@ class Backport: for br in branches: br.process(self.dry_run) - for br in branches: - assert br.backported, f"BUG! backport to branch [{br}] failed" - self.mark_pr_backported(pr) + if all(br.backported for br in branches): + # And check it after the running + self.mark_pr_backported(pr) def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 949d17f777f..4e34e6b6135 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -3,23 +3,19 @@ import concurrent.futures import json import logging import os -import random import re import subprocess import sys -import time -from copy import deepcopy -from dataclasses import asdict, dataclass -from enum import Enum +from dataclasses import dataclass from pathlib import Path -from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union +from typing import Any, Dict, List, Optional import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames, StatusNames +from ci_config import CI from ci_metadata import CiMetadata -from ci_utils import GHActions, is_hex, normalize_string +from ci_utils import GHActions, normalize_string from clickhouse_helper import ( CiLogsCredentials, ClickHouseHelper, @@ -35,18 +31,15 @@ from commit_status_helper import ( get_commit, post_commit_status, set_status_comment, + get_commit_filtered_statuses, ) -from digest_helper import DockerDigester, JobDigester +from digest_helper import DockerDigester from env_helper import ( - CI, - CI_CONFIG_PATH, + IS_CI, GITHUB_JOB_API_URL, GITHUB_REPOSITORY, GITHUB_RUN_ID, - GITHUB_RUN_URL, REPO_COPY, - REPORT_PATH, - S3_BUILDS_BUCKET, TEMP_PATH, ) from get_robot_token import get_best_robot_token @@ -58,946 +51,13 @@ from report import ERROR, FAILURE, PENDING, SUCCESS, BuildResult, JobReport, Tes from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_cache import CiCache +from ci_settings import CiSettings from version_helper import get_version_from_repo # pylint: disable=too-many-lines -@dataclass -class PendingState: - updated_at: float - run_url: str - - -class CiCache: - """ - CI cache is a bunch of records. Record is a file stored under special location on s3. - The file name has a format: - - _[]--___.ci - - RECORD_TYPE: - SUCCESSFUL - for successful jobs - PENDING - for pending jobs - - ATTRIBUTES: - release - for jobs being executed on the release branch including master branch (not a PR branch) - """ - - _S3_CACHE_PREFIX = "CI_cache_v1" - _CACHE_BUILD_REPORT_PREFIX = "build_report" - _RECORD_FILE_EXTENSION = ".ci" - _LOCAL_CACHE_PATH = Path(TEMP_PATH) / "ci_cache" - _ATTRIBUTE_RELEASE = "release" - # divider symbol 1 - _DIV1 = "--" - # divider symbol 2 - _DIV2 = "_" - assert _DIV1 != _DIV2 - - class RecordType(Enum): - SUCCESSFUL = "successful" - PENDING = "pending" - FAILED = "failed" - - @dataclass - class Record: - record_type: "CiCache.RecordType" - job_name: str - job_digest: str - batch: int - num_batches: int - release_branch: bool - file: str = "" - - def to_str_key(self): - """other fields must not be included in the hash str""" - return "_".join( - [self.job_name, self.job_digest, str(self.batch), str(self.num_batches)] - ) - - class JobType(Enum): - DOCS = "DOCS" - SRCS = "SRCS" - - @classmethod - def is_docs_job(cls, job_name: str) -> bool: - return job_name == JobNames.DOCS_CHECK - - @classmethod - def is_srcs_job(cls, job_name: str) -> bool: - return not cls.is_docs_job(job_name) - - @classmethod - def get_type_by_name(cls, job_name: str) -> "CiCache.JobType": - res = cls.SRCS - if cls.is_docs_job(job_name): - res = cls.DOCS - elif cls.is_srcs_job(job_name): - res = cls.SRCS - else: - assert False - return res - - def __init__( - self, - s3: S3Helper, - job_digests: Dict[str, str], - ): - self.s3 = s3 - self.job_digests = job_digests - self.cache_s3_paths = { - job_type: f"{self._S3_CACHE_PREFIX}/{job_type.value}-{self._get_digest_for_job_type(self.job_digests, job_type)}/" - for job_type in self.JobType - } - self.s3_record_prefixes = { - record_type: record_type.value for record_type in self.RecordType - } - self.records: Dict["CiCache.RecordType", Dict[str, "CiCache.Record"]] = { - record_type: {} for record_type in self.RecordType - } - - self.cache_updated = False - self.cache_data_fetched = True - if not self._LOCAL_CACHE_PATH.exists(): - self._LOCAL_CACHE_PATH.mkdir(parents=True, exist_ok=True) - - def _get_digest_for_job_type( - self, job_digests: Dict[str, str], job_type: JobType - ) -> str: - if job_type == self.JobType.DOCS: - res = job_digests[JobNames.DOCS_CHECK] - elif job_type == self.JobType.SRCS: - # any build type job has the same digest - pick up Build.PACKAGE_RELEASE or Build.PACKAGE_ASAN as a failover - # Build.PACKAGE_RELEASE may not exist in the list if we have reduced CI pipeline - if Build.PACKAGE_RELEASE in job_digests: - res = job_digests[Build.PACKAGE_RELEASE] - elif Build.PACKAGE_ASAN in job_digests: - # failover, if failover does not work - fix it! - res = job_digests[Build.PACKAGE_ASAN] - else: - assert False, "BUG, no build job in digest' list" - else: - assert False, "BUG, New JobType? - please update func" - return res - - def _get_record_file_name( - self, - record_type: RecordType, - job_name: str, - batch: int, - num_batches: int, - release_branch: bool, - ) -> str: - prefix = self.s3_record_prefixes[record_type] - prefix_extended = ( - self._DIV2.join([prefix, self._ATTRIBUTE_RELEASE]) - if release_branch - else prefix - ) - assert self._DIV1 not in job_name, f"Invalid job name {job_name}" - job_name = self._DIV2.join( - [job_name, self.job_digests[job_name], str(batch), str(num_batches)] - ) - file_name = self._DIV1.join([prefix_extended, job_name]) - file_name += self._RECORD_FILE_EXTENSION - return file_name - - def _get_record_s3_path(self, job_name: str) -> str: - return self.cache_s3_paths[self.JobType.get_type_by_name(job_name)] - - def _parse_record_file_name( - self, record_type: RecordType, file_name: str - ) -> Optional["CiCache.Record"]: - # validate filename - if ( - not file_name.endswith(self._RECORD_FILE_EXTENSION) - or not len(file_name.split(self._DIV1)) == 2 - ): - print("ERROR: wrong file name format") - return None - - file_name = file_name.removesuffix(self._RECORD_FILE_EXTENSION) - release_branch = False - - prefix_extended, job_suffix = file_name.split(self._DIV1) - record_type_and_attribute = prefix_extended.split(self._DIV2) - - # validate filename prefix - failure = False - if not 0 < len(record_type_and_attribute) <= 2: - print("ERROR: wrong file name prefix") - failure = True - if ( - len(record_type_and_attribute) > 1 - and record_type_and_attribute[1] != self._ATTRIBUTE_RELEASE - ): - print("ERROR: wrong record attribute") - failure = True - if record_type_and_attribute[0] != self.s3_record_prefixes[record_type]: - print("ERROR: wrong record type") - failure = True - if failure: - return None - - if ( - len(record_type_and_attribute) > 1 - and record_type_and_attribute[1] == self._ATTRIBUTE_RELEASE - ): - release_branch = True - - job_properties = job_suffix.split(self._DIV2) - job_name, job_digest, batch, num_batches = ( - self._DIV2.join(job_properties[:-3]), - job_properties[-3], - int(job_properties[-2]), - int(job_properties[-1]), - ) - - if not is_hex(job_digest): - print("ERROR: wrong record job digest") - return None - - record = self.Record( - record_type, - job_name, - job_digest, - batch, - num_batches, - release_branch, - file="", - ) - return record - - def print_status(self): - for record_type in self.RecordType: - GHActions.print_in_group( - f"Cache records: [{record_type}]", list(self.records[record_type]) - ) - return self - - @staticmethod - def dump_run_config(indata: Dict[str, Any]) -> None: - assert indata - assert CI_CONFIG_PATH - with open(CI_CONFIG_PATH, "w", encoding="utf-8") as json_file: - json.dump(indata, json_file, indent=2) - - def update(self): - """ - Pulls cache records from s3. Only records name w/o content. - """ - for record_type in self.RecordType: - prefix = self.s3_record_prefixes[record_type] - cache_list = self.records[record_type] - for job_type in self.JobType: - path = self.cache_s3_paths[job_type] - records = self.s3.list_prefix(f"{path}{prefix}", S3_BUILDS_BUCKET) - records = [record.split("/")[-1] for record in records] - for file in records: - record = self._parse_record_file_name( - record_type=record_type, file_name=file - ) - if not record: - print(f"ERROR: failed to parse cache record [{file}]") - continue - if ( - record.job_name not in self.job_digests - or self.job_digests[record.job_name] != record.job_digest - ): - # skip records we are not interested in - continue - - if record.to_str_key() not in cache_list: - cache_list[record.to_str_key()] = record - self.cache_data_fetched = False - elif ( - not cache_list[record.to_str_key()].release_branch - and record.release_branch - ): - # replace a non-release record with a release one - cache_list[record.to_str_key()] = record - self.cache_data_fetched = False - - self.cache_updated = True - return self - - def fetch_records_data(self): - """ - Pulls CommitStatusData for all cached jobs from s3 - """ - if not self.cache_updated: - self.update() - - if self.cache_data_fetched: - # there are no records without fetched data - no need to fetch - return self - - # clean up - for file in self._LOCAL_CACHE_PATH.glob("*.ci"): - file.unlink() - - # download all record files - for job_type in self.JobType: - path = self.cache_s3_paths[job_type] - for record_type in self.RecordType: - prefix = self.s3_record_prefixes[record_type] - _ = self.s3.download_files( - bucket=S3_BUILDS_BUCKET, - s3_path=f"{path}{prefix}", - file_suffix=self._RECORD_FILE_EXTENSION, - local_directory=self._LOCAL_CACHE_PATH, - ) - - # validate we have files for all records and save file names meanwhile - for record_type in self.RecordType: - record_list = self.records[record_type] - for _, record in record_list.items(): - record_file_name = self._get_record_file_name( - record_type, - record.job_name, - record.batch, - record.num_batches, - record.release_branch, - ) - assert ( - self._LOCAL_CACHE_PATH / record_file_name - ).is_file(), f"BUG. Record file must be present: {self._LOCAL_CACHE_PATH / record_file_name}" - record.file = record_file_name - - self.cache_data_fetched = True - return self - - def exist( - self, - record_type: "CiCache.RecordType", - job: str, - batch: int, - num_batches: int, - release_branch: bool, - ) -> bool: - if not self.cache_updated: - self.update() - record_key = self.Record( - record_type, - job, - self.job_digests[job], - batch, - num_batches, - release_branch, - ).to_str_key() - res = record_key in self.records[record_type] - if release_branch: - return res and self.records[record_type][record_key].release_branch - else: - return res - - def push( - self, - record_type: "CiCache.RecordType", - job: str, - batches: Union[int, Sequence[int]], - num_batches: int, - status: Union[CommitStatusData, PendingState], - release_branch: bool = False, - ) -> None: - """ - Pushes a cache record (CommitStatusData) - @release_branch adds "release" attribute to a record - """ - if isinstance(batches, int): - batches = [batches] - for batch in batches: - record_file = self._LOCAL_CACHE_PATH / self._get_record_file_name( - record_type, job, batch, num_batches, release_branch - ) - record_s3_path = self._get_record_s3_path(job) - if record_type == self.RecordType.SUCCESSFUL: - assert isinstance(status, CommitStatusData) - status.dump_to_file(record_file) - elif record_type == self.RecordType.FAILED: - assert isinstance(status, CommitStatusData) - status.dump_to_file(record_file) - elif record_type == self.RecordType.PENDING: - assert isinstance(status, PendingState) - with open(record_file, "w", encoding="utf-8") as json_file: - json.dump(asdict(status), json_file) - else: - assert False - - _ = self.s3.upload_file( - bucket=S3_BUILDS_BUCKET, - file_path=record_file, - s3_path=record_s3_path + record_file.name, - ) - record = self.Record( - record_type, - job, - self.job_digests[job], - batch, - num_batches, - release_branch, - file=record_file.name, - ) - if ( - record.release_branch - or record.to_str_key() not in self.records[record_type] - ): - self.records[record_type][record.to_str_key()] = record - - def get( - self, record_type: "CiCache.RecordType", job: str, batch: int, num_batches: int - ) -> Optional[Union[CommitStatusData, PendingState]]: - """ - Gets a cache record data for a job, or None if a cache miss - """ - - if not self.cache_data_fetched: - self.fetch_records_data() - - record_key = self.Record( - record_type, - job, - self.job_digests[job], - batch, - num_batches, - release_branch=False, - ).to_str_key() - - if record_key not in self.records[record_type]: - return None - - record_file_name = self.records[record_type][record_key].file - - res = CommitStatusData.load_from_file( - self._LOCAL_CACHE_PATH / record_file_name - ) # type: CommitStatusData - - return res - - def delete( - self, - record_type: "CiCache.RecordType", - job: str, - batch: int, - num_batches: int, - release_branch: bool, - ) -> None: - """ - deletes record from the cache - """ - raise NotImplementedError("Let's try make cache push-and-read-only") - # assert ( - # record_type == self.RecordType.PENDING - # ), "FIXME: delete is supported for pending records only" - # record_file_name = self._get_record_file_name( - # self.RecordType.PENDING, - # job, - # batch, - # num_batches, - # release_branch=release_branch, - # ) - # record_s3_path = self._get_record_s3_path(job) - # self.s3.delete_file_from_s3(S3_BUILDS_BUCKET, record_s3_path + record_file_name) - - # record_key = self.Record( - # record_type, - # job, - # self.job_digests[job], - # batch, - # num_batches, - # release_branch=False, - # ).to_str_key() - - # if record_key in self.records[record_type]: - # del self.records[record_type][record_key] - - def is_successful( - self, job: str, batch: int, num_batches: int, release_branch: bool - ) -> bool: - """ - checks if a given job have already been done successfully - """ - return self.exist( - self.RecordType.SUCCESSFUL, job, batch, num_batches, release_branch - ) - - def is_failed( - self, job: str, batch: int, num_batches: int, release_branch: bool - ) -> bool: - """ - checks if a given job have already been done with failure - """ - return self.exist( - self.RecordType.FAILED, job, batch, num_batches, release_branch - ) - - def is_pending( - self, job: str, batch: int, num_batches: int, release_branch: bool - ) -> bool: - """ - check pending record in the cache for a given job - @release_branch - checks that "release" attribute is set for a record - """ - if self.is_successful( - job, batch, num_batches, release_branch - ) or self.is_failed(job, batch, num_batches, release_branch): - return False - - return self.exist( - self.RecordType.PENDING, job, batch, num_batches, release_branch - ) - - def push_successful( - self, - job: str, - batch: int, - num_batches: int, - job_status: CommitStatusData, - release_branch: bool = False, - ) -> None: - """ - Pushes a cache record (CommitStatusData) - @release_branch adds "release" attribute to a record - """ - self.push( - self.RecordType.SUCCESSFUL, - job, - [batch], - num_batches, - job_status, - release_branch, - ) - - def push_failed( - self, - job: str, - batch: int, - num_batches: int, - job_status: CommitStatusData, - release_branch: bool = False, - ) -> None: - """ - Pushes a cache record of type Failed (CommitStatusData) - @release_branch adds "release" attribute to a record - """ - self.push( - self.RecordType.FAILED, - job, - [batch], - num_batches, - job_status, - release_branch, - ) - - def push_pending( - self, job: str, batches: List[int], num_batches: int, release_branch: bool - ) -> None: - """ - pushes pending record for a job to the cache - """ - pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL) - self.push( - self.RecordType.PENDING, - job, - batches, - num_batches, - pending_state, - release_branch, - ) - - def get_successful( - self, job: str, batch: int, num_batches: int - ) -> Optional[CommitStatusData]: - """ - Gets a cache record (CommitStatusData) for a job, or None if a cache miss - """ - res = self.get(self.RecordType.SUCCESSFUL, job, batch, num_batches) - assert res is None or isinstance(res, CommitStatusData) - return res - - def delete_pending( - self, job: str, batch: int, num_batches: int, release_branch: bool - ) -> None: - """ - deletes pending record from the cache - """ - self.delete(self.RecordType.PENDING, job, batch, num_batches, release_branch) - - def download_build_reports(self, file_prefix: str = "") -> List[str]: - """ - not an ideal class for this method, - but let it be as we store build reports in CI cache directory on s3 - and CiCache knows where exactly - - @file_prefix allows filtering out reports by git head_ref - """ - report_path = Path(REPORT_PATH) - report_path.mkdir(exist_ok=True, parents=True) - path = ( - self._get_record_s3_path(Build.PACKAGE_RELEASE) - + self._CACHE_BUILD_REPORT_PREFIX - ) - if file_prefix: - path += "_" + file_prefix - reports_files = self.s3.download_files( - bucket=S3_BUILDS_BUCKET, - s3_path=path, - file_suffix=".json", - local_directory=report_path, - ) - return reports_files - - def upload_build_report(self, build_result: BuildResult) -> str: - result_json_path = build_result.write_json(Path(TEMP_PATH)) - s3_path = ( - self._get_record_s3_path(Build.PACKAGE_RELEASE) + result_json_path.name - ) - return self.s3.upload_file( - bucket=S3_BUILDS_BUCKET, file_path=result_json_path, s3_path=s3_path - ) - - def await_jobs( - self, jobs_with_params: Dict[str, Dict[str, Any]], is_release_branch: bool - ) -> Dict[str, List[int]]: - """ - await pending jobs to be finished - @jobs_with_params - jobs to await. {JOB_NAME: {"batches": [BATCHES...], "num_batches": NUM_BATCHES}} - returns successfully finished jobs: {JOB_NAME: [BATCHES...]} - """ - if not jobs_with_params: - return {} - poll_interval_sec = 300 - # TIMEOUT * MAX_ROUNDS_TO_WAIT must be less than 6h (GH job timeout) with a room for rest RunConfig work - TIMEOUT = 3000 # 50 min - MAX_ROUNDS_TO_WAIT = 6 - MAX_JOB_NUM_TO_WAIT = 3 - await_finished: Dict[str, List[int]] = {} - round_cnt = 0 - while ( - len(jobs_with_params) > MAX_JOB_NUM_TO_WAIT - and round_cnt < MAX_ROUNDS_TO_WAIT - ): - round_cnt += 1 - GHActions.print_in_group( - f"Wait pending jobs, round [{round_cnt}/{MAX_ROUNDS_TO_WAIT}]:", - list(jobs_with_params), - ) - # this is initial approach to wait pending jobs: - # start waiting for the next TIMEOUT seconds if there are more than X(=4) jobs to wait - # wait TIMEOUT seconds in rounds. Y(=5) is the max number of rounds - expired_sec = 0 - start_at = int(time.time()) - while expired_sec < TIMEOUT and jobs_with_params: - time.sleep(poll_interval_sec) - self.update() - jobs_with_params_copy = deepcopy(jobs_with_params) - for job_name in jobs_with_params: - num_batches = jobs_with_params[job_name]["num_batches"] - job_config = CI_CONFIG.get_job_config(job_name) - for batch in jobs_with_params[job_name]["batches"]: - if self.is_pending( - job_name, - batch, - num_batches, - release_branch=is_release_branch - and job_config.required_on_release_branch, - ): - continue - print( - f"Job [{job_name}_[{batch}/{num_batches}]] is not pending anymore" - ) - - # some_job_ready = True - jobs_with_params_copy[job_name]["batches"].remove(batch) - if not jobs_with_params_copy[job_name]["batches"]: - del jobs_with_params_copy[job_name] - - if not self.is_successful( - job_name, - batch, - num_batches, - release_branch=is_release_branch - and job_config.required_on_release_branch, - ): - print( - f"NOTE: Job [{job_name}:{batch}] finished but no success - remove from awaiting list, do not add to ready" - ) - continue - if job_name in await_finished: - await_finished[job_name].append(batch) - else: - await_finished[job_name] = [batch] - jobs_with_params = jobs_with_params_copy - expired_sec = int(time.time()) - start_at - print( - f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]" - ) - if await_finished: - GHActions.print_in_group( - f"Finished jobs, round [{round_cnt}]:", - [f"{job}:{batches}" for job, batches in await_finished.items()], - ) - GHActions.print_in_group( - "Remaining jobs:", - [f"{job}:{params['batches']}" for job, params in jobs_with_params.items()], - ) - return await_finished - - -@dataclass -class CiOptions: - # job will be included in the run if any keyword from the list matches job name - include_keywords: Optional[List[str]] = None - # job will be excluded in the run if any keyword from the list matches job name - exclude_keywords: Optional[List[str]] = None - - # list of specified preconfigured ci sets to run - ci_sets: Optional[List[str]] = None - # list of specified jobs to run - ci_jobs: Optional[List[str]] = None - - # batches to run for all multi-batch jobs - job_batches: Optional[List[int]] = None - - do_not_test: bool = False - no_ci_cache: bool = False - upload_all: bool = False - no_merge_commit: bool = False - - def as_dict(self) -> Dict[str, Any]: - return asdict(self) - - @staticmethod - def create_from_run_config(run_config: Dict[str, Any]) -> "CiOptions": - return CiOptions(**run_config["ci_options"]) - - @staticmethod - def create_from_pr_message( - debug_message: Optional[str], update_from_api: bool - ) -> "CiOptions": - """ - Creates CiOptions instance based on tags found in PR body and/or commit message - @commit_message - may be provided directly for debugging purposes, otherwise it will be retrieved from git. - """ - res = CiOptions() - pr_info = PRInfo() - if ( - not pr_info.is_pr and not debug_message - ): # if commit_message is provided it's test/debug scenario - do not return - # CI options can be configured in PRs only - # if debug_message is provided - it's a test - return res - message = debug_message or GitRunner(set_cwd_to_git_root=True).run( - f"{GIT_PREFIX} log {pr_info.sha} --format=%B -n 1" - ) - - # CI setting example we need to match with re: - # - [x] Exclude: All with TSAN, MSAN, UBSAN, Coverage - pattern = r"(#|- \[x\] + Exclude: All with TSAN, MSAN, UBSAN, Coverage + pattern = r"(#|- \[x\] + do not test (only style check) +- [x] Woolen Wolfdog CI - [x] disable merge-commit (no merge from master before tests) - [ ] disable CI cache (job reuse) @@ -54,10 +55,18 @@ _TEST_JOB_LIST = [ "Fast test", "package_release", "package_asan", + "package_aarch64", + "package_release_coverage", + "package_debug", + "package_tsan", + "package_msan", + "package_ubsan", + "binary_release", + "fuzzers", "Docker server image", "Docker keeper image", - "Install packages (amd64)", - "Install packages (arm64)", + "Install packages (release)", + "Install packages (aarch64)", "Stateless tests (debug)", "Stateless tests (release)", "Stateless tests (coverage)", @@ -112,15 +121,15 @@ _TEST_JOB_LIST = [ "AST fuzzer (ubsan)", "ClickHouse Keeper Jepsen", "ClickHouse Server Jepsen", - "Performance Comparison", - "Performance Comparison Aarch64", + "Performance Comparison (release)", + "Performance Comparison (aarch64)", "Sqllogic test (release)", "SQLancer (release)", "SQLancer (debug)", "SQLTest", - "Compatibility check (amd64)", + "Compatibility check (release)", "Compatibility check (aarch64)", - "ClickBench (amd64)", + "ClickBench (release)", "ClickBench (aarch64)", "libFuzzer tests", "ClickHouse build check", @@ -129,24 +138,28 @@ _TEST_JOB_LIST = [ "Bugfix validation", ] +_TEST_JOB_LIST_2 = ["Style check", "Fast test", "fuzzers"] + class TestCIOptions(unittest.TestCase): def test_pr_body_parsing(self): - ci_options = CiOptions.create_from_pr_message( + ci_options = CiSettings.create_from_pr_message( _TEST_BODY_1, update_from_api=False ) self.assertFalse(ci_options.do_not_test) self.assertFalse(ci_options.no_ci_cache) self.assertTrue(ci_options.no_merge_commit) + self.assertTrue(ci_options.woolen_wolfdog) self.assertEqual(ci_options.ci_sets, ["ci_set_non_required"]) self.assertCountEqual(ci_options.include_keywords, ["foo", "foo_bar"]) self.assertCountEqual(ci_options.exclude_keywords, ["foo", "foo_bar"]) def test_options_applied(self): self.maxDiff = None - ci_options = CiOptions.create_from_pr_message( + ci_options = CiSettings.create_from_pr_message( _TEST_BODY_2, update_from_api=False ) + self.assertFalse(ci_options.woolen_wolfdog) self.assertCountEqual( ci_options.include_keywords, ["integration", "foo_bar", "stateless", "azure"], @@ -155,24 +168,40 @@ class TestCIOptions(unittest.TestCase): ci_options.exclude_keywords, ["tsan", "foobar", "aarch64", "analyzer", "s3_storage", "coverage"], ) - jobs_to_do = list(_TEST_JOB_LIST) - jobs_to_skip = [] - job_params = { - "Stateless tests (azure, asan)": { - "batches": list(range(3)), - "num_batches": 3, - "run_by_ci_option": True, - } + + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST } - jobs_to_do, jobs_to_skip, job_params = ci_options.apply( - jobs_to_do, jobs_to_skip, job_params, PRInfo() + jobs_configs[ + "fuzzers" + ].run_by_label = ( + "TEST_LABEL" # check "fuzzers" appears in the result due to the label + ) + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( + True # still must be included as it's set with include keywords + ) + filtered_jobs = list( + ci_options.apply( + jobs_configs, + is_release=False, + is_pr=True, + is_mq=False, + labels=["TEST_LABEL"], + ) ) self.assertCountEqual( - jobs_to_do, + filtered_jobs, [ "Style check", + "fuzzers", "package_release", "package_asan", + "package_debug", + "package_msan", + "package_ubsan", "Stateless tests (asan)", "Stateless tests (azure, asan)", "Stateless tests flaky check (asan)", @@ -187,54 +216,123 @@ class TestCIOptions(unittest.TestCase): ) def test_options_applied_2(self): + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST_2 + } + jobs_configs["Style check"].release_only = True + jobs_configs["Fast test"].pr_only = True + jobs_configs["fuzzers"].run_by_label = "TEST_LABEL" + # no settings are set + filtered_jobs = list( + CiSettings().apply( + jobs_configs, is_release=False, is_pr=False, is_mq=True, labels=[] + ) + ) + self.assertCountEqual( + filtered_jobs, + [ + "Fast test", + ], + ) + filtered_jobs = list( + CiSettings().apply( + jobs_configs, is_release=False, is_pr=True, is_mq=False, labels=[] + ) + ) + self.assertCountEqual( + filtered_jobs, + [ + "Fast test", + ], + ) + filtered_jobs = list( + CiSettings().apply( + jobs_configs, is_release=True, is_pr=False, is_mq=False, labels=[] + ) + ) + self.assertCountEqual( + filtered_jobs, + [ + "Style check", + ], + ) + + def test_options_applied_3(self): + ci_settings = CiSettings() + ci_settings.include_keywords = ["Style"] + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST_2 + } + jobs_configs["Style check"].release_only = True + jobs_configs["Fast test"].pr_only = True + # no settings are set + filtered_jobs = list( + ci_settings.apply( + jobs_configs, + is_release=False, + is_pr=True, + is_mq=False, + labels=["TEST_LABEL"], + ) + ) + self.assertCountEqual( + filtered_jobs, + [ + "Style check", + ], + ) + + ci_settings.include_keywords = ["Fast"] + filtered_jobs = list( + ci_settings.apply( + jobs_configs, + is_release=True, + is_pr=False, + is_mq=False, + labels=["TEST_LABEL"], + ) + ) + self.assertCountEqual( + filtered_jobs, + [ + "Style check", + ], + ) + + def test_options_applied_4(self): self.maxDiff = None - ci_options = CiOptions.create_from_pr_message( + ci_options = CiSettings.create_from_pr_message( _TEST_BODY_3, update_from_api=False ) self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) self.assertIsNone(ci_options.exclude_keywords) - jobs_to_do = list(_TEST_JOB_LIST) - jobs_to_skip = [] - job_params = {} - jobs_to_do, jobs_to_skip, job_params = ci_options.apply( - jobs_to_do, jobs_to_skip, job_params, PRInfo() + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST + } + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs["Integration tests (asan)"].release_only = True + filtered_jobs = list( + ci_options.apply( + jobs_configs, + is_release=False, + is_pr=True, + is_mq=False, + labels=["TEST_LABEL"], + ) ) self.assertCountEqual( - jobs_to_do, + filtered_jobs, [ "Style check", "Integration tests (asan, old analyzer)", "package_release", "Stateless tests (release, old analyzer, s3, DatabaseReplicated)", "package_asan", + "fuzzers", ], ) - - def test_options_applied_3(self): - self.maxDiff = None - ci_options = CiOptions.create_from_pr_message( - _TEST_BODY_4, update_from_api=False - ) - self.assertIsNone(ci_options.include_keywords, None) - self.assertIsNone(ci_options.exclude_keywords, None) - jobs_to_do = list(_TEST_JOB_LIST) - jobs_to_skip = [] - job_params = {} - - for job in _TEST_JOB_LIST: - if "Stateless" in job: - job_params[job] = { - "batches": list(range(3)), - "num_batches": 3, - "run_by_ci_option": "azure" in job, - } - else: - job_params[job] = {"run_by_ci_option": False} - - jobs_to_do, jobs_to_skip, job_params = ci_options.apply( - jobs_to_do, jobs_to_skip, job_params, PRInfo() - ) - self.assertNotIn( - "Stateless tests (azure, asan)", - jobs_to_do, - ) diff --git a/tests/ci/test_git.py b/tests/ci/test_git.py index 3aedd8a8dea..60cd95b6869 100644 --- a/tests/ci/test_git.py +++ b/tests/ci/test_git.py @@ -1,10 +1,11 @@ #!/usr/bin/env python -from unittest.mock import patch import os.path as p import unittest +from dataclasses import dataclass +from unittest.mock import patch -from git_helper import Git, Runner, CWD +from git_helper import CWD, Git, Runner, git_runner class TestRunner(unittest.TestCase): @@ -35,8 +36,10 @@ class TestRunner(unittest.TestCase): class TestGit(unittest.TestCase): def setUp(self): """we use dummy git object""" + # get the git_runner's cwd to set it properly before the Runner is patched + _ = git_runner.cwd run_patcher = patch("git_helper.Runner.run", return_value="") - self.run_mock = run_patcher.start() + run_mock = run_patcher.start() self.addCleanup(run_patcher.stop) update_patcher = patch("git_helper.Git.update") update_mock = update_patcher.start() @@ -44,15 +47,13 @@ class TestGit(unittest.TestCase): self.git = Git() update_mock.assert_called_once() self.git.run("test") - self.run_mock.assert_called_once() - self.git.new_branch = "NEW_BRANCH_NAME" - self.git.new_tag = "v21.12.333.22222-stable" + run_mock.assert_called_once() self.git.branch = "old_branch" self.git.sha = "" self.git.sha_short = "" self.git.latest_tag = "" - self.git.description = "" - self.git.commits_since_tag = 0 + self.git.commits_since_latest = 0 + self.git.commits_since_new = 0 def test_tags(self): self.git.new_tag = "v21.12.333.22222-stable" @@ -71,11 +72,30 @@ class TestGit(unittest.TestCase): setattr(self.git, tag_attr, tag) def test_tweak(self): - self.git.commits_since_tag = 0 - self.assertEqual(self.git.tweak, 1) - self.git.commits_since_tag = 2 - self.assertEqual(self.git.tweak, 2) - self.git.latest_tag = "v21.12.333.22222-testing" - self.assertEqual(self.git.tweak, 22224) - self.git.commits_since_tag = 0 - self.assertEqual(self.git.tweak, 22222) + # tweak for the latest tag + @dataclass + class TestCase: + tag: str + commits: int + tweak: int + + cases = ( + TestCase("", 0, 1), + TestCase("", 2, 2), + TestCase("v21.12.333.22222-stable", 0, 22222), + TestCase("v21.12.333.22222-stable", 2, 2), + TestCase("v21.12.333.22222-testing", 0, 22222), + TestCase("v21.12.333.22222-testing", 2, 22224), + ) + for tag, commits, tweak in ( + ("latest_tag", "commits_since_latest", "tweak"), + ("new_tag", "commits_since_new", "tweak_to_new"), + ): + for tc in cases: + setattr(self.git, tag, tc.tag) + setattr(self.git, commits, tc.commits) + self.assertEqual( + getattr(self.git, tweak), + tc.tweak, + f"Wrong tweak for tag {tc.tag} and commits {tc.commits} of {tag}", + ) diff --git a/tests/ci/test_version.py b/tests/ci/test_version.py index 978edcc093e..c4f12091ec0 100644 --- a/tests/ci/test_version.py +++ b/tests/ci/test_version.py @@ -2,8 +2,13 @@ import unittest from argparse import ArgumentTypeError +from dataclasses import dataclass +from pathlib import Path import version_helper as vh +from git_helper import Git + +CHV = vh.ClickHouseVersion class TestFunctions(unittest.TestCase): @@ -32,3 +37,55 @@ class TestFunctions(unittest.TestCase): for error_case in error_cases: with self.assertRaises(ArgumentTypeError): version = vh.version_arg(error_case[0]) + + def test_get_version_from_repo(self): + @dataclass + class TestCase: + latest_tag: str + commits_since_latest: int + new_tag: str + commits_since_new: int + expected: CHV + + cases = ( + TestCase( + "v24.6.1.1-new", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 415), + ), + TestCase( + "v24.6.1.1-testing", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 16), + ), + TestCase( + "v24.6.1.1-stable", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 15), + ), + TestCase( + "v24.5.1.1-stable", + 15, + "v24.4.1.2088-stable", + 415, + CHV(24, 5, 1, 54487, None, 15), + ), + ) + git = Git(True) + for tc in cases: + git.latest_tag = tc.latest_tag + git.commits_since_latest = tc.commits_since_latest + git.new_tag = tc.new_tag + git.commits_since_new = tc.commits_since_new + self.assertEqual( + vh.get_version_from_repo( + Path("tests/ci/tests/autogenerated_versions.txt"), git + ), + tc.expected, + ) diff --git a/tests/ci/tests/autogenerated_versions.txt b/tests/ci/tests/autogenerated_versions.txt new file mode 100644 index 00000000000..10028bf50c8 --- /dev/null +++ b/tests/ci/tests/autogenerated_versions.txt @@ -0,0 +1,12 @@ +# This variables autochanged by tests/ci/version_helper.py: + +# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, +# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. +SET(VERSION_REVISION 54487) +SET(VERSION_MAJOR 24) +SET(VERSION_MINOR 5) +SET(VERSION_PATCH 1) +SET(VERSION_GITHASH 70a1d3a63d47f0be077d67b8deb907230fc7cfb0) +SET(VERSION_DESCRIBE v24.5.1.1-testing) +SET(VERSION_STRING 24.5.1.1) +# end of autochange diff --git a/tests/ci/tmp/.gitignore b/tests/ci/tmp/.gitignore new file mode 100644 index 00000000000..72e8ffc0db8 --- /dev/null +++ b/tests/ci/tmp/.gitignore @@ -0,0 +1 @@ +* diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 800bfcf52c3..50263f6ebb6 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -1,7 +1,7 @@ #!/usr/bin/env python3 import logging -import os.path as p from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser, ArgumentTypeError +from pathlib import Path from typing import Any, Dict, Iterable, List, Literal, Optional, Set, Tuple, Union from git_helper import TWEAK, Git, get_tags, git_runner, removeprefix @@ -22,7 +22,7 @@ VERSIONS = Dict[str, Union[int, str]] VERSIONS_TEMPLATE = """# This variables autochanged by tests/ci/version_helper.py: -# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, +# NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. SET(VERSION_REVISION {revision}) SET(VERSION_MAJOR {major}) @@ -47,7 +47,7 @@ class ClickHouseVersion: patch: Union[int, str], revision: Union[int, str], git: Optional[Git], - tweak: Optional[str] = None, + tweak: Optional[Union[int, str]] = None, ): self._major = int(major) self._minor = int(minor) @@ -95,7 +95,7 @@ class ClickHouseVersion: if self._git is not None: self._git.update() return ClickHouseVersion( - self.major, self.minor, self.patch, self.revision, self._git, "1" + self.major, self.minor, self.patch, self.revision, self._git, 1 ) @property @@ -114,6 +114,10 @@ class ClickHouseVersion: def tweak(self) -> int: return self._tweak + @tweak.setter + def tweak(self, tweak: int) -> None: + self._tweak = tweak + @property def revision(self) -> int: return self._revision @@ -172,7 +176,7 @@ class ClickHouseVersion: self.patch, self.revision, self._git, - str(self.tweak), + self.tweak, ) try: copy.with_description(self.description) @@ -190,7 +194,9 @@ class ClickHouseVersion: and self.tweak == other.tweak ) - def __lt__(self, other: "ClickHouseVersion") -> bool: + def __lt__(self, other: Any) -> bool: + if not isinstance(self, type(other)): + return NotImplemented for part in ("major", "minor", "patch", "tweak"): if getattr(self, part) < getattr(other, part): return True @@ -220,10 +226,11 @@ ClickHouseVersions = List[ClickHouseVersion] class VersionType: LTS = "lts" + NEW = "new" PRESTABLE = "prestable" STABLE = "stable" TESTING = "testing" - VALID = (TESTING, PRESTABLE, STABLE, LTS) + VALID = (NEW, TESTING, PRESTABLE, STABLE, LTS) def validate_version(version: str) -> None: @@ -234,43 +241,56 @@ def validate_version(version: str) -> None: int(part) -def get_abs_path(path: str) -> str: - return p.abspath(p.join(git_runner.cwd, path)) +def get_abs_path(path: Union[Path, str]) -> Path: + return (Path(git_runner.cwd) / path).absolute() -def read_versions(versions_path: str = FILE_WITH_VERSION_PATH) -> VERSIONS: +def read_versions(versions_path: Union[Path, str] = FILE_WITH_VERSION_PATH) -> VERSIONS: versions = {} - path_to_file = get_abs_path(versions_path) - with open(path_to_file, "r", encoding="utf-8") as f: - for line in f: - line = line.strip() - if not line.startswith("SET("): - continue + for line in get_abs_path(versions_path).read_text(encoding="utf-8").splitlines(): + line = line.strip() + if not line.startswith("SET("): + continue - value = 0 # type: Union[int, str] - name, value = line[4:-1].split(maxsplit=1) - name = removeprefix(name, "VERSION_").lower() - try: - value = int(value) - except ValueError: - pass - versions[name] = value + value = 0 # type: Union[int, str] + name, value = line[4:-1].split(maxsplit=1) + name = removeprefix(name, "VERSION_").lower() + try: + value = int(value) + except ValueError: + pass + versions[name] = value return versions def get_version_from_repo( - versions_path: str = FILE_WITH_VERSION_PATH, + versions_path: Union[Path, str] = FILE_WITH_VERSION_PATH, git: Optional[Git] = None, ) -> ClickHouseVersion: + """Get a ClickHouseVersion from FILE_WITH_VERSION_PATH. When the `git` parameter is + present, a proper `tweak` version part is calculated for case if the latest tag has + a `new` type and greater than version in `FILE_WITH_VERSION_PATH`""" versions = read_versions(versions_path) - return ClickHouseVersion( + cmake_version = ClickHouseVersion( versions["major"], versions["minor"], versions["patch"], versions["revision"], git, ) + # Since 24.5 we have tags like v24.6.1.1-new, and we must check if the release + # branch already has it's own commit. It's necessary for a proper tweak version + if git is not None and git.latest_tag: + version_from_tag = get_version_from_tag(git.latest_tag) + if ( + version_from_tag.description == VersionType.NEW + and cmake_version < version_from_tag + ): + # We are in a new release branch without existing release. + # We should change the tweak version to a `tweak_to_new` + cmake_version.tweak = git.tweak_to_new + return cmake_version def get_version_from_string( @@ -350,15 +370,15 @@ def get_supported_versions( def update_cmake_version( version: ClickHouseVersion, - versions_path: str = FILE_WITH_VERSION_PATH, + versions_path: Union[Path, str] = FILE_WITH_VERSION_PATH, ) -> None: - path_to_file = get_abs_path(versions_path) - with open(path_to_file, "w", encoding="utf-8") as f: - f.write(VERSIONS_TEMPLATE.format_map(version.as_dict())) + get_abs_path(versions_path).write_text( + VERSIONS_TEMPLATE.format_map(version.as_dict()), encoding="utf-8" + ) def update_contributors( - relative_contributors_path: str = GENERATED_CONTRIBUTORS, + relative_contributors_path: Union[Path, str] = GENERATED_CONTRIBUTORS, force: bool = False, raise_error: bool = False, ) -> None: @@ -378,13 +398,11 @@ def update_contributors( ) contributors = [f' "{c}",' for c in contributors] - executer = p.relpath(p.realpath(__file__), git_runner.cwd) + executer = Path(__file__).relative_to(git_runner.cwd) content = CONTRIBUTORS_TEMPLATE.format( executer=executer, contributors="\n".join(contributors) ) - contributors_path = get_abs_path(relative_contributors_path) - with open(contributors_path, "w", encoding="utf-8") as cfd: - cfd.write(content) + get_abs_path(relative_contributors_path).write_text(content, encoding="utf-8") def update_version_local(version, version_type="testing"): diff --git a/tests/clickhouse-test b/tests/clickhouse-test index af203563d58..36870d59c3a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -808,10 +808,10 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "max_bytes_before_external_sort": threshold_generator( - 0.3, 0.5, 1, 10 * 1024 * 1024 * 1024 + 0.3, 0.5, 0, 10 * 1024 * 1024 * 1024 ), "max_bytes_before_external_group_by": threshold_generator( - 0.3, 0.5, 1, 10 * 1024 * 1024 * 1024 + 0.3, 0.5, 0, 10 * 1024 * 1024 * 1024 ), "max_bytes_before_remerge_sort": lambda: random.randint(1, 3000000000), "min_compress_block_size": lambda: random.randint(1, 1048576 * 3), @@ -850,6 +850,11 @@ class SettingsRandomizer: "merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": lambda: round( random.random(), 2 ), + "prefer_external_sort_block_bytes": lambda: random.choice([0, 1, 100000000]), + "cross_join_min_rows_to_compress": lambda: random.choice([0, 1, 100000000]), + "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]), } @staticmethod diff --git a/tests/config/config.d/rocksdb.xml b/tests/config/config.d/rocksdb.xml new file mode 100644 index 00000000000..a3790a3dc1d --- /dev/null +++ b/tests/config/config.d/rocksdb.xml @@ -0,0 +1,7 @@ + + + + DEBUG_LEVEL + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 6536683b6c2..e04392d893b 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -68,6 +68,7 @@ ln -sf $SRC_PATH/config.d/zero_copy_destructive_operations.xml $DEST_SERVER_PATH ln -sf $SRC_PATH/config.d/block_number.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 5739496cb50..4e4468fef77 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -110,10 +110,9 @@ class HDFSApi(object): logging.debug( "Stdout:\n{}\n".format(res.stdout.decode("utf-8")) ) - logging.debug("Env:\n{}\n".format(env)) raise Exception( "Command {} return non-zero code {}: {}".format( - args, res.returncode, res.stderr.decode("utf-8") + cmd, res.returncode, res.stderr.decode("utf-8") ) ) diff --git a/tests/integration/helpers/s3_url_proxy_tests_util.py b/tests/integration/helpers/s3_url_proxy_tests_util.py index 9059fda08ae..9a45855acb8 100644 --- a/tests/integration/helpers/s3_url_proxy_tests_util.py +++ b/tests/integration/helpers/s3_url_proxy_tests_util.py @@ -2,21 +2,35 @@ import os import time +ALL_HTTP_METHODS = {"POST", "PUT", "GET", "HEAD", "CONNECT"} + + def check_proxy_logs( - cluster, proxy_instance, protocol, bucket, http_methods={"POST", "PUT", "GET"} + cluster, proxy_instances, protocol, bucket, requested_http_methods ): for i in range(10): - logs = cluster.get_container_logs(proxy_instance) # Check with retry that all possible interactions with Minio are present - for http_method in http_methods: - if ( - logs.find(http_method + f" {protocol}://minio1:9001/root/data/{bucket}") - >= 0 - ): - return - time.sleep(1) - else: - assert False, f"{http_methods} method not found in logs of {proxy_instance}" + for http_method in ALL_HTTP_METHODS: + for proxy_instance in proxy_instances: + logs = cluster.get_container_logs(proxy_instance) + if ( + logs.find( + http_method + f" {protocol}://minio1:9001/root/data/{bucket}" + ) + >= 0 + ): + if http_method not in requested_http_methods: + assert ( + False + ), f"Found http method {http_method} for bucket {bucket} that should not be found in {proxy_instance} logs" + break + else: + if http_method in requested_http_methods: + assert ( + False + ), f"{http_method} method not found in logs of {proxy_instance} for bucket {bucket}" + + time.sleep(1) def wait_resolver(cluster): @@ -33,8 +47,8 @@ def wait_resolver(cluster): if response == "proxy1" or response == "proxy2": return time.sleep(i) - else: - assert False, "Resolver is not up" + + assert False, "Resolver is not up" # Runs simple proxy resolver in python env container. @@ -80,9 +94,43 @@ def perform_simple_queries(node, minio_endpoint): def simple_test(cluster, proxies, protocol, bucket): minio_endpoint = build_s3_endpoint(protocol, bucket) - node = cluster.instances[f"{bucket}"] + node = cluster.instances[bucket] perform_simple_queries(node, minio_endpoint) - for proxy in proxies: - check_proxy_logs(cluster, proxy, protocol, bucket) + check_proxy_logs(cluster, proxies, protocol, bucket, ["PUT", "GET", "HEAD"]) + + +def simple_storage_test(cluster, node, proxies, policy): + node.query( + """ + CREATE TABLE s3_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='{}' + """.format( + policy + ) + ) + node.query("INSERT INTO s3_test VALUES (0,'data'),(1,'data')") + assert ( + node.query("SELECT * FROM s3_test order by id FORMAT Values") + == "(0,'data'),(1,'data')" + ) + + node.query("DROP TABLE IF EXISTS s3_test SYNC") + + # not checking for POST because it is in a different format + check_proxy_logs(cluster, proxies, "http", policy, ["PUT", "GET"]) + + +def simple_test_assert_no_proxy(cluster, proxies, protocol, bucket): + minio_endpoint = build_s3_endpoint(protocol, bucket) + node = cluster.instances[bucket] + perform_simple_queries(node, minio_endpoint) + + # No HTTP method should be found in proxy logs if no proxy is active + empty_method_list = [] + check_proxy_logs(cluster, proxies, protocol, bucket, empty_method_list) diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index 2afbae340be..1c8c5c33a13 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -139,12 +139,18 @@ def assert_logs_contain_with_retry(instance, substring, retry_count=20, sleep_ti def exec_query_with_retry( - instance, query, retry_count=40, sleep_time=0.5, silent=False, settings={} + instance, + query, + retry_count=40, + sleep_time=0.5, + silent=False, + settings={}, + timeout=30, ): exception = None for cnt in range(retry_count): try: - res = instance.query(query, timeout=30, settings=settings) + res = instance.query(query, timeout=timeout, settings=settings) if not silent: logging.debug(f"Result of {query} on {cnt} try is {res}") break diff --git a/tests/integration/helpers/uclient.py b/tests/integration/helpers/uclient.py index 45c8b8f64e2..195eb52ffeb 100644 --- a/tests/integration/helpers/uclient.py +++ b/tests/integration/helpers/uclient.py @@ -8,7 +8,7 @@ sys.path.insert(0, os.path.join(CURDIR)) from . import uexpect -prompt = ":\) " +prompt = ":\\) " end_of_block = r".*\r\n.*\r\n" @@ -21,7 +21,7 @@ class client(object): self.client.eol("\r") self.client.logger(log, prefix=name) self.client.timeout(20) - self.client.expect("[#\$] ", timeout=2) + self.client.expect("[#\\$] ", timeout=2) self.client.send(command) def __enter__(self): diff --git a/tests/integration/test_access_control_on_cluster/test.py b/tests/integration/test_access_control_on_cluster/test.py index 8dbb87c67d8..b12add7ad3f 100644 --- a/tests/integration/test_access_control_on_cluster/test.py +++ b/tests/integration/test_access_control_on_cluster/test.py @@ -74,3 +74,18 @@ def test_grant_all_on_cluster(): assert ch2.query("SHOW GRANTS FOR Alex") == "GRANT ALL ON *.* TO Alex\n" ch1.query("DROP USER Alex ON CLUSTER 'cluster'") + + +def test_grant_current_database_on_cluster(): + ch1.query("CREATE DATABASE user_db ON CLUSTER 'cluster'") + ch1.query( + "CREATE USER IF NOT EXISTS test_user ON CLUSTER 'cluster' DEFAULT DATABASE user_db" + ) + ch1.query( + "GRANT SELECT ON user_db.* TO test_user ON CLUSTER 'cluster' WITH GRANT OPTION" + ) + ch1.query("GRANT CLUSTER ON *.* TO test_user ON CLUSTER 'cluster'") + + assert ch1.query("SHOW DATABASES", user="test_user") == "user_db\n" + ch1.query("GRANT SELECT ON * TO test_user ON CLUSTER 'cluster'", user="test_user") + assert ch1.query("SHOW DATABASES", user="test_user") == "user_db\n" diff --git a/tests/integration/test_manipulate_statistic/__init__.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py similarity index 100% rename from tests/integration/test_manipulate_statistic/__init__.py rename to tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..47e88730482 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py new file mode 100644 index 00000000000..a8f4ab05888 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -0,0 +1,73 @@ +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=["configs/asynchronous_metrics_update_period_s.xml"], + env_variables={"MALLOC_CONF": "background_thread:true,prof:true"}, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +# asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting +# asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and +# are available for querying during the test. +def test_asynchronous_metric_jemalloc_profile_active(started_cluster): + # default open + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + + res_o = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) + assert ( + res_o + == """Row 1: +────── +metric: jemalloc.prof.active +value: 1 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) + # disable + node1.query("SYSTEM JEMALLOC DISABLE PROFILE") + time.sleep(5) + res_t = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) + assert ( + res_t + == """Row 1: +────── +metric: jemalloc.prof.active +value: 0 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) + # enable + node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + time.sleep(5) + res_f = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) + assert ( + res_f + == """Row 1: +────── +metric: jemalloc.prof.active +value: 1 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) diff --git a/tests/integration/test_attach_partition_using_copy/test.py b/tests/integration/test_attach_partition_using_copy/test.py index e7163b1eb32..d5b07603dff 100644 --- a/tests/integration/test_attach_partition_using_copy/test.py +++ b/tests/integration/test_attach_partition_using_copy/test.py @@ -98,7 +98,8 @@ def create_destination_table(node, table_name, replicated): ) -def test_both_mergtree(start_cluster): +def test_both_mergetree(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", False) @@ -120,12 +121,13 @@ def test_both_mergtree(start_cluster): def test_all_replicated(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", True) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) - replica1.query("SYSTEM SYNC REPLICA destination") replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + replica2.query("SYSTEM SYNC REPLICA destination") assert_eq_with_retry( replica1, @@ -154,12 +156,13 @@ def test_all_replicated(start_cluster): def test_only_destination_replicated(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) - replica1.query("SYSTEM SYNC REPLICA destination") replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + replica2.query("SYSTEM SYNC REPLICA destination") assert_eq_with_retry( replica1, @@ -188,6 +191,7 @@ def test_only_destination_replicated(start_cluster): def test_not_work_on_different_disk(start_cluster): + cleanup([replica1, replica2]) # Replace and move should not work on replace create_source_table(replica1, "source", False) create_destination_table(replica2, "destination", False) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ef9e536976b..c67f63e3f6b 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -168,6 +168,32 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" +def test_restore_materialized_view_with_definer(): + instance.query("CREATE DATABASE test") + instance.query( + "CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s" + ) + instance.query("CREATE USER u1") + instance.query("GRANT SELECT ON *.* TO u1") + instance.query("GRANT INSERT ON *.* TO u1") + + instance.query( + """ + CREATE MATERIALIZED VIEW test.test_mv_1 (s String) + ENGINE = MergeTree ORDER BY s + DEFINER = u1 SQL SECURITY DEFINER + AS SELECT * FROM test.test_table + """ + ) + + backup_name = new_backup_name() + instance.query(f"BACKUP DATABASE test TO {backup_name}") + instance.query("DROP DATABASE test") + instance.query("DROP USER u1") + + instance.query(f"RESTORE DATABASE test FROM {backup_name}") + + @pytest.mark.parametrize( "engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"] ) @@ -1474,7 +1500,7 @@ def test_backup_all(exclude_system_log_tables): restore_settings = [] if not exclude_system_log_tables: restore_settings.append("allow_non_empty_tables=true") - restore_command = f"RESTORE ALL FROM {backup_name} {'SETTINGS '+ ', '.join(restore_settings) if restore_settings else ''}" + restore_command = f"RESTORE ALL FROM {backup_name} {'SETTINGS ' + ', '.join(restore_settings) if restore_settings else ''}" session_id = new_session_id() instance.http_query( diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index c9f20333654..cd0f2032559 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -161,13 +161,13 @@ def wait_for_fail_restore(node, restore_id): elif status == "RESTORING": assert_eq_with_retry( node, - f"SELECT status FROM system.backups WHERE id = '{backup_id}'", + f"SELECT status FROM system.backups WHERE id = '{restore_id}'", "RESTORE_FAILED", sleep_time=2, retry_count=50, ) error = node.query( - f"SELECT error FROM system.backups WHERE id == '{backup_id}'" + f"SELECT error FROM system.backups WHERE id == '{restore_id}'" ).rstrip("\n") assert re.search( "Cannot restore the table default.tbl because it already contains some data", diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index b8db4e005a4..f5d0c066600 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -187,7 +187,7 @@ def check_convert_all_dbs_to_atomic(): # 6 tables, MVs contain 2 rows (inner tables does not match regexp) assert "8\t{}\n".format(8 * len("atomic")) == node.query( - "SELECT count(), sum(n) FROM atomic.merge".format(db) + "SELECT count(), sum(n) FROM atomic.merge" ) node.query("DETACH TABLE ordinary.detached PERMANENTLY") diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 1cf5c3deb81..758dda655da 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -89,7 +89,7 @@ def test_aggregate_states(start_cluster): logging.info("Skipping %s", aggregate_function) skipped += 1 continue - logging.exception("Failed %s", function) + logging.exception("Failed %s", aggregate_function) failed += 1 continue diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 476f7c61b28..912fd3cc163 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -300,7 +300,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): LIMIT 1000000 SETTINGS s3_max_single_part_upload_size=100, - s3_min_upload_part_size=1000000, + s3_min_upload_part_size=100000, s3_check_objects_after_upload=0 """, query_id=insert_query_id, @@ -311,7 +311,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): ) assert create_multipart == 1 - assert upload_parts == 7 + assert upload_parts == 69 assert s3_errors == 3 broken_s3.setup_at_part_upload( @@ -512,6 +512,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( ), error +@pytest.mark.skip(reason="test is flaky, waiting ClickHouse/issues/64451") def test_query_is_canceled_with_inf_retries(cluster, broken_s3): node = cluster.instances["node_with_inf_s3_retries"] diff --git a/tests/integration/test_disk_over_web_server/configs/no_async_load.xml b/tests/integration/test_disk_over_web_server/configs/no_async_load.xml new file mode 100644 index 00000000000..1100874ad48 --- /dev/null +++ b/tests/integration/test_disk_over_web_server/configs/no_async_load.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 9f43ab73fa3..891ee8f00f5 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -11,13 +11,13 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "node1", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, use_old_analyzer=True, ) cluster.add_instance( "node2", - main_configs=["configs/storage_conf_web.xml"], + main_configs=["configs/storage_conf_web.xml", "configs/no_async_load.xml"], with_nginx=True, stay_alive=True, with_zookeeper=True, @@ -25,7 +25,7 @@ def cluster(): ) cluster.add_instance( "node3", - main_configs=["configs/storage_conf_web.xml"], + main_configs=["configs/storage_conf_web.xml", "configs/no_async_load.xml"], with_nginx=True, with_zookeeper=True, use_old_analyzer=True, @@ -33,7 +33,7 @@ def cluster(): cluster.add_instance( "node4", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, stay_alive=True, with_installed_binary=True, @@ -42,7 +42,7 @@ def cluster(): ) cluster.add_instance( "node5", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, use_old_analyzer=True, ) @@ -116,7 +116,7 @@ def test_usage(cluster, node_name): (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( - i, uuids[i], i, i + i, uuids[i] ) ) @@ -338,7 +338,7 @@ def test_page_cache(cluster): (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( - i, uuids[i], i, i + i, uuids[i] ) ) diff --git a/tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml b/tests/integration/test_hot_reload_storage_policy/configs/config.d/storage_configuration.xml similarity index 56% rename from tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml rename to tests/integration/test_hot_reload_storage_policy/configs/config.d/storage_configuration.xml index 466ecde137d..8940efb3301 100644 --- a/tests/integration/test_hot_reload_storage_policy/configs/storage_configuration.xml +++ b/tests/integration/test_hot_reload_storage_policy/configs/config.d/storage_configuration.xml @@ -4,18 +4,25 @@ /var/lib/clickhouse/disk0/ - - /var/lib/clickhouse/disk1/ - - + disk0 - + + + + + + localhost + 9000 + + + + \ No newline at end of file diff --git a/tests/integration/test_hot_reload_storage_policy/test.py b/tests/integration/test_hot_reload_storage_policy/test.py index 8654b0462e4..1d38f39d72c 100644 --- a/tests/integration/test_hot_reload_storage_policy/test.py +++ b/tests/integration/test_hot_reload_storage_policy/test.py @@ -10,11 +10,8 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -node0 = cluster.add_instance( - "node0", with_zookeeper=True, main_configs=["configs/storage_configuration.xml"] -) -node1 = cluster.add_instance( - "node1", with_zookeeper=True, main_configs=["configs/storage_configuration.xml"] +node = cluster.add_instance( + "node", main_configs=["configs/config.d/storage_configuration.xml"], stay_alive=True ) @@ -28,6 +25,37 @@ def started_cluster(): cluster.shutdown() +old_disk_config = """ + + + + + /var/lib/clickhouse/disk0/ + + + + + + + disk0 + + + + + + + + + + localhost + 9000 + + + + + +""" + new_disk_config = """ @@ -38,49 +66,120 @@ new_disk_config = """ /var/lib/clickhouse/disk1/ - - /var/lib/clickhouse/disk2/ - - - disk2 + disk1 + + disk0 - + + + + + + localhost + 9000 + + + + """ def set_config(node, config): - node.replace_config( - "/etc/clickhouse-server/config.d/storage_configuration.xml", config - ) + node.replace_config("/etc/clickhouse-server/config.d/config.xml", config) node.query("SYSTEM RELOAD CONFIG") + # to give ClickHouse time to refresh disks + time.sleep(1) def test_hot_reload_policy(started_cluster): - node0.query( - "CREATE TABLE t (d Int32, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/t', '0') PARTITION BY d ORDER BY tuple() SETTINGS storage_policy = 'default_policy'" + node.query( + "CREATE TABLE t (d Int32, s String) ENGINE = MergeTree() PARTITION BY d ORDER BY tuple() SETTINGS storage_policy = 'default_policy'" ) - node0.query("INSERT INTO TABLE t VALUES (1, 'foo') (1, 'bar')") + node.query("SYSTEM STOP MERGES t") + node.query("INSERT INTO TABLE t VALUES (1, 'foo')") - node1.query( - "CREATE TABLE t (d Int32, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/t_mirror', '1') PARTITION BY d ORDER BY tuple() SETTINGS storage_policy = 'default_policy'" + set_config(node, new_disk_config) + + # After reloading new policy with new disk, merge tree tables should reinitialize the new disk (create relative path, 'detached' folder...) + # and as default policy is `least_used`, at least one insertion should come to the new disk + node.query("INSERT INTO TABLE t VALUES (1, 'foo')") + node.query("INSERT INTO TABLE t VALUES (1, 'bar')") + + num_disks = int( + node.query( + "SELECT uniqExact(disk_name) FROM system.parts WHERE database = 'default' AND table = 't'" + ) ) - set_config(node1, new_disk_config) - time.sleep(1) - node1.query("ALTER TABLE t FETCH PARTITION 1 FROM '/clickhouse/tables/t'") - result = int(node1.query("SELECT count() FROM t")) + assert ( - result == 4, - "Node should have 2 x full data (4 rows) after reloading storage configuration and fetch new partition, but get {} rows".format( - result - ), + num_disks == 2 + ), "Node should write data to 2 disks after reloading disks, but got {}".format( + num_disks ) + + # If `detached` is not created this query will throw exception + node.query("ALTER TABLE t DETACH PARTITION 1") + + node.query("DROP TABLE t") + + +def test_hot_reload_policy_distributed_table(started_cluster): + # Same test for distributed table, it should reinitialize the storage policy and data volume + # We check it by trying an insert and the distribution queue must be on new disk + + # Restart node first + set_config(node, old_disk_config) + node.restart_clickhouse() + + node.query( + "CREATE TABLE t (d Int32, s String) ENGINE = MergeTree PARTITION BY d ORDER BY tuple()" + ) + node.query( + "CREATE TABLE t_d (d Int32, s String) ENGINE = Distributed('default', 'default', 't', d%20, 'default_policy')" + ) + + node.query("SYSTEM STOP DISTRIBUTED SENDS t_d") + node.query( + "INSERT INTO TABLE t_d SETTINGS prefer_localhost_replica = 0 VALUES (2, 'bar') (12, 'bar')" + ) + # t_d should create queue on disk0 + queue_path = node.query("SELECT data_path FROM system.distribution_queue") + + assert ( + "disk0" in queue_path + ), "Distributed table should create distributed queue on disk0 (disk1), but the queue path is {}".format( + queue_path + ) + + node.query("SYSTEM START DISTRIBUTED SENDS t_d") + + node.query("SYSTEM FLUSH DISTRIBUTED t_d") + + set_config(node, new_disk_config) + + node.query("SYSTEM STOP DISTRIBUTED SENDS t_d") + node.query( + "INSERT INTO TABLE t_d SETTINGS prefer_localhost_replica = 0 VALUES (2, 'bar') (12, 'bar')" + ) + + # t_d should create queue on disk1 + queue_path = node.query("SELECT data_path FROM system.distribution_queue") + + assert ( + "disk1" in queue_path + ), "Distributed table should be using new disk (disk1), but the queue paths are {}".format( + queue_path + ) + + node.query("DROP TABLE t") + node.query("DROP TABLE t_d") diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index f6ac42a2db2..b2efbf4bb65 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -88,6 +88,11 @@ def test_dynamic_query_handler(): "application/whatever; charset=cp1337" == res_custom_ct.headers["content-type"] ) + assert "it works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Works"] + assert ( + "also works" + == res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"] + ) def test_predefined_query_handler(): @@ -146,6 +151,10 @@ def test_predefined_query_handler(): ) assert b"max_final_threads\t1\nmax_threads\t1\n" == res2.content assert "application/generic+one" == res2.headers["content-type"] + assert "it works" == res2.headers["X-Test-Http-Response-Headers-Works"] + assert ( + "also works" == res2.headers["X-Test-Http-Response-Headers-Even-Multiple"] + ) cluster.instance.query( "CREATE TABLE test_table (id UInt32, data String) Engine=TinyLog" @@ -212,6 +221,18 @@ def test_fixed_static_handler(): "test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"} ).content ) + assert ( + "it works" + == cluster.instance.http_request( + "test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"} + ).headers["X-Test-Http-Response-Headers-Works"] + ) + assert ( + "also works" + == cluster.instance.http_request( + "test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"} + ).headers["X-Test-Http-Response-Headers-Even-Multiple"] + ) def test_config_static_handler(): diff --git a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml index c9b61c21507..58fedbd9078 100644 --- a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml @@ -18,6 +18,10 @@ dynamic_query_handler get_dynamic_handler_query application/whatever; charset=cp1337 + + it works + also works + diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml index 1b8ddfab323..a7804721f12 100644 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -19,6 +19,10 @@ predefined_query_handler SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String} application/generic+one + + it works + also works + diff --git a/tests/integration/test_http_handlers_config/test_static_handler/config.xml b/tests/integration/test_http_handlers_config/test_static_handler/config.xml index ff24e6dec96..76c1f588853 100644 --- a/tests/integration/test_http_handlers_config/test_static_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_static_handler/config.xml @@ -12,6 +12,10 @@ 402 text/html; charset=UTF-8 Test get static handler and fix content + + it works + also works + diff --git a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/proxy-resolver/resolver.py b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/proxy-resolver/resolver.py index 8c7611303b8..eaea4c1dab2 100644 --- a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/proxy-resolver/resolver.py +++ b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/proxy-resolver/resolver.py @@ -5,7 +5,10 @@ import bottle @bottle.route("/hostname") def index(): - return "proxy1" + if random.randrange(2) == 0: + return "proxy1" + else: + return "proxy2" bottle.run(host="0.0.0.0", port=8080) diff --git a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py index ae872a33cd4..3c8a5de8691 100644 --- a/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py +++ b/tests/integration/test_https_s3_table_function_with_http_proxy_no_tunneling/test.py @@ -56,7 +56,7 @@ def test_s3_with_https_proxy_list(cluster): def test_s3_with_https_remote_proxy(cluster): - proxy_util.simple_test(cluster, ["proxy1"], "https", "remote_proxy_node") + proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "remote_proxy_node") def test_s3_with_https_env_proxy(cluster): diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index 69ab83283ff..8635f5e612a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -90,7 +90,7 @@ def wait_until_fully_merged(node, table): except: return - raise Exception(f"There are still merges on-going after {retry} assignments") + raise Exception(f"There are still merges on-going after {i} assignments") def test_jbod_balanced_merge(start_cluster): diff --git a/tests/integration/test_jdbc_bridge/test.py b/tests/integration/test_jdbc_bridge/test.py index c4a0a525df3..1efd868e4a7 100644 --- a/tests/integration/test_jdbc_bridge/test.py +++ b/tests/integration/test_jdbc_bridge/test.py @@ -91,7 +91,7 @@ def test_jdbc_insert(started_cluster): """ CREATE TABLE test.test_insert ENGINE = Memory AS SELECT * FROM test.ClickHouseTable; - SELECT * + SELECT * FROM jdbc('{0}?mutation', 'INSERT INTO test.test_insert VALUES({1}, ''{1}'', ''{1}'')'); """.format( datasource, records @@ -115,7 +115,7 @@ def test_jdbc_update(started_cluster): """ CREATE TABLE test.test_update ENGINE = Memory AS SELECT * FROM test.ClickHouseTable; - SELECT * + SELECT * FROM jdbc( '{}?mutation', 'SET mutations_sync = 1; ALTER TABLE test.test_update UPDATE Str=''{}'' WHERE Num = {} - 1;' @@ -145,7 +145,7 @@ def test_jdbc_delete(started_cluster): """ CREATE TABLE test.test_delete ENGINE = Memory AS SELECT * FROM test.ClickHouseTable; - SELECT * + SELECT * FROM jdbc( '{}?mutation', 'SET mutations_sync = 1; ALTER TABLE test.test_delete DELETE WHERE Num < {} - 1;' @@ -158,7 +158,7 @@ def test_jdbc_delete(started_cluster): expected = records - 1 actual = instance.query( "SELECT Str FROM jdbc('{}', 'SELECT * FROM test.test_delete')".format( - datasource, records + datasource ) ) assert int(actual) == expected, "expecting {} but got {}".format(expected, actual) diff --git a/tests/queries/0_stateless/02864_statistic_exception.reference b/tests/integration/test_keeper_profiler/__init__.py similarity index 100% rename from tests/queries/0_stateless/02864_statistic_exception.reference rename to tests/integration/test_keeper_profiler/__init__.py diff --git a/tests/integration/test_keeper_profiler/configs/keeper_config1.xml b/tests/integration/test_keeper_profiler/configs/keeper_config1.xml new file mode 100644 index 00000000000..b541090bd98 --- /dev/null +++ b/tests/integration/test_keeper_profiler/configs/keeper_config1.xml @@ -0,0 +1,44 @@ + + + + http://minio1:9001/snapshots/ + minio + minio123 + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 50 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_keeper_profiler/configs/keeper_config2.xml b/tests/integration/test_keeper_profiler/configs/keeper_config2.xml new file mode 100644 index 00000000000..9ad0a772860 --- /dev/null +++ b/tests/integration/test_keeper_profiler/configs/keeper_config2.xml @@ -0,0 +1,44 @@ + + + + http://minio1:9001/snapshots/ + minio + minio123 + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 75 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_keeper_profiler/configs/keeper_config3.xml b/tests/integration/test_keeper_profiler/configs/keeper_config3.xml new file mode 100644 index 00000000000..ff31f7cdbf4 --- /dev/null +++ b/tests/integration/test_keeper_profiler/configs/keeper_config3.xml @@ -0,0 +1,44 @@ + + + + http://minio1:9001/snapshots/ + minio + minio123 + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 75 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py new file mode 100644 index 00000000000..848929df086 --- /dev/null +++ b/tests/integration/test_keeper_profiler/test.py @@ -0,0 +1,96 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.keeper_utils import KeeperClient, KeeperException + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node1", + main_configs=["configs/keeper_config1.xml"], + stay_alive=True, +) +node2 = cluster.add_instance( + "node2", + main_configs=["configs/keeper_config2.xml"], + stay_alive=True, + with_minio=True, +) +node3 = cluster.add_instance( + "node3", + main_configs=["configs/keeper_config3.xml"], + stay_alive=True, + with_minio=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_profiler(started_cluster): + node = cluster.instances["node1"] + if node.is_built_with_sanitizer(): + return + + node.query( + "CREATE TABLE t (key UInt32, value String) Engine = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') ORDER BY key" + ) + + for _ in range(100): + node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + + node.query("system flush logs") + assert int(node.query("exists system.trace_log")) + + result = node.query( + """ +set allow_introspection_functions=1; +system flush logs; +select cnt from ( + select count() as cnt, formatReadableSize(sum(size)), + arrayStringConcat( + arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') as trace +from system.trace_log where trace_type = ‘Real’ and (trace ilike '%KeeperTCPHandler%' or trace ilike '%KeeperDispatcher%') group by trace order by cnt desc) limit 1; + """ + ) + + if len(result) == 0: + assert 0 < int( + node.query( + """ + set allow_introspection_functions=1; + system flush logs; + select sum(cnt) from ( + select count() as cnt, formatReadableSize(sum(size)), + arrayStringConcat( + arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') as trace + from system.trace_log where trace_type = ‘Real’ group by trace); + """ + ) + ) + result = node.query( + """ + set allow_introspection_functions=1; + system flush logs; + select * from ( + select count() as cnt, formatReadableSize(sum(size)), + arrayStringConcat( + arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') as trace + from system.trace_log where trace_type = ‘Real’ group by trace); + """ + ) + print(result) + assert False + + assert 1 < int(result) diff --git a/tests/integration/test_keeper_snapshot_small_distance/test.py b/tests/integration/test_keeper_snapshot_small_distance/test.py index be8bf1bd245..612c5b3c65d 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/test.py +++ b/tests/integration/test_keeper_snapshot_small_distance/test.py @@ -1,5 +1,5 @@ #!/usr/bin/env python3 -##!/usr/bin/env python3 + import pytest from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils diff --git a/tests/integration/test_keeper_snapshots/test.py b/tests/integration/test_keeper_snapshots/test.py index f6f746c892e..951970dba23 100644 --- a/tests/integration/test_keeper_snapshots/test.py +++ b/tests/integration/test_keeper_snapshots/test.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -#!/usr/bin/env python3 import pytest from helpers.cluster import ClickHouseCluster import helpers.keeper_utils as keeper_utils @@ -17,7 +16,6 @@ node = cluster.add_instance( "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, - with_zookeeper=True, ) @@ -211,3 +209,46 @@ def test_invalid_snapshot(started_cluster): node_zk.close() except: pass + + +def test_snapshot_size(started_cluster): + keeper_utils.wait_until_connected(started_cluster, node) + node_zk = None + try: + node_zk = get_connection_zk("node") + + node_zk.create("/test_state_size", b"somevalue") + strs = [] + for i in range(100): + strs.append(random_string(123).encode()) + node_zk.create("/test_state_size/node" + str(i), strs[i]) + + node_zk.stop() + node_zk.close() + + keeper_utils.send_4lw_cmd(started_cluster, node, "csnp") + node.wait_for_log_line("Created persistent snapshot") + + def get_snapshot_size(): + return int( + next( + filter( + lambda line: "zk_latest_snapshot_size" in line, + keeper_utils.send_4lw_cmd(started_cluster, node, "mntr").split( + "\n" + ), + ) + ).split("\t")[1] + ) + + assert get_snapshot_size() != 0 + restart_clickhouse() + assert get_snapshot_size() != 0 + finally: + try: + if node_zk is not None: + node_zk.stop() + node_zk.close() + + except: + pass diff --git a/tests/integration/test_keeper_three_nodes_start/test.py b/tests/integration/test_keeper_three_nodes_start/test.py index bc93a6089cb..6576d386fcb 100644 --- a/tests/integration/test_keeper_three_nodes_start/test.py +++ b/tests/integration/test_keeper_three_nodes_start/test.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -#!/usr/bin/env python3 import pytest from helpers.cluster import ClickHouseCluster import random diff --git a/tests/integration/test_manipulate_statistics/__init__.py b/tests/integration/test_manipulate_statistics/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_manipulate_statistic/config/config.xml b/tests/integration/test_manipulate_statistics/config/config.xml similarity index 55% rename from tests/integration/test_manipulate_statistic/config/config.xml rename to tests/integration/test_manipulate_statistics/config/config.xml index b47f8123499..24225173eeb 100644 --- a/tests/integration/test_manipulate_statistic/config/config.xml +++ b/tests/integration/test_manipulate_statistics/config/config.xml @@ -1,7 +1,7 @@ - 1 + 1 diff --git a/tests/integration/test_manipulate_statistic/test.py b/tests/integration/test_manipulate_statistics/test.py similarity index 51% rename from tests/integration/test_manipulate_statistic/test.py rename to tests/integration/test_manipulate_statistics/test.py index 7b96b392da8..a602cce63df 100644 --- a/tests/integration/test_manipulate_statistic/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,7 +6,11 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=False + "node1", user_configs=["config/config.xml"], with_zookeeper=True +) + +node2 = cluster.add_instance( + "node2", user_configs=["config/config.xml"], with_zookeeper=True ) @@ -34,14 +38,14 @@ def check_stat_file_on_disk(node, table, part_name, column_name, exist): [ "bash", "-c", - "find {p} -type f -name statistic_{col}.stat".format( + "find {p} -type f -name statistics_{col}.stats".format( p=part_path, col=column_name ), ], privileged=True, ) logging.debug( - f"Checking stat file in {part_path} for column {column_name}, got {output}" + f"Checking stats file in {part_path} for column {column_name}, got {output}" ) if exist: assert len(output) != 0 @@ -56,26 +60,26 @@ def run_test_single_node(started_cluster): check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0", "c", True) - node1.query("ALTER TABLE test_stat DROP STATISTIC a type tdigest") + node1.query("ALTER TABLE test_stat DROP STATISTICS a") check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "a", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_2", "c", True) - node1.query("ALTER TABLE test_stat CLEAR STATISTIC b, c type tdigest") + node1.query("ALTER TABLE test_stat CLEAR STATISTICS b, c") check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "a", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "b", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_3", "c", False) - node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC b, c type tdigest") + node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS b, c") check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "a", False) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "b", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_4", "c", True) - node1.query("ALTER TABLE test_stat ADD STATISTIC a type tdigest") - node1.query("ALTER TABLE test_stat MATERIALIZE STATISTIC a type tdigest") + node1.query("ALTER TABLE test_stat ADD STATISTICS a type tdigest") + node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS a") check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "a", True) check_stat_file_on_disk(node1, "test_stat", "all_1_1_0_5", "b", True) @@ -104,7 +108,7 @@ def test_single_node_wide(started_cluster): node1.query( """ - CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest), b Int64 STATISTICS(tdigest), c Int64 STATISTICS(tdigest)) ENGINE = MergeTree() ORDER BY a SETTINGS min_bytes_for_wide_part = 0; """ @@ -117,8 +121,63 @@ def test_single_node_normal(started_cluster): node1.query( """ - CREATE TABLE test_stat(a Int64 STATISTIC(tdigest), b Int64 STATISTIC(tdigest), c Int64 STATISTIC(tdigest)) + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest), b Int64 STATISTICS(tdigest), c Int64 STATISTICS(tdigest)) ENGINE = MergeTree() ORDER BY a; """ ) run_test_single_node(started_cluster) + + +def test_replicated_table_ddl(started_cluster): + node1.query("DROP TABLE IF EXISTS test_stat") + node2.query("DROP TABLE IF EXISTS test_stat") + + node1.query( + """ + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest)) + ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '1') ORDER BY a; + """ + ) + node2.query( + """ + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest)) + ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '2') ORDER BY a; + """ + ) + + node1.query( + "ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", + settings={"alter_sync": "2"}, + ) + node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync": "2"}) + + assert ( + node2.query("SHOW CREATE TABLE test_stat") + == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" + ) + + node2.query("insert into test_stat values(1,2,3), (2,3,4)") + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) + node1.query( + "ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"} + ) + assert node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n" + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True) + node1.query( + "ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"} + ) + node1.query( + "ALTER TABLE test_stat ADD STATISTICS b type tdigest", + settings={"alter_sync": "2"}, + ) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False) + node1.query( + "ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"} + ) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 251da7e4e09..38cbf8c1aed 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -195,10 +195,10 @@ def test_create_table(): f"DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", "DNS_ERROR", ), - f"S3Queue('http://minio1:9001/root/data/', 'CSV')", - f"S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip')", - f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV')", - f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV', 'gzip')", + f"S3Queue('http://minio1:9001/root/data/', 'CSV') settings mode = 'ordered'", + f"S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip') settings mode = 'ordered'", + f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV') settings mode = 'ordered'", + f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV', 'gzip') settings mode = 'ordered'", ] def make_test_case(i): @@ -258,10 +258,11 @@ def test_create_table(): "CREATE TABLE table14 (x int) ENGINE = S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')", "CREATE TABLE table15 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", "CREATE TABLE table16 (`x` int) ENGINE = DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE table17 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV')", - "CREATE TABLE table18 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip')", - "CREATE TABLE table19 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV')", - "CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip')", + "CREATE TABLE table17 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV') settings mode = 'ordered'", + "CREATE TABLE table18 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip') settings mode = 'ordered'", + # due to sensitive data substituion the query will be normalized, so not "settings" but "SETTINGS" + "CREATE TABLE table19 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV') SETTINGS mode = 'ordered'", + "CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip') SETTINGS mode = 'ordered'", ], must_not_contain=[password], ) diff --git a/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml b/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml new file mode 100644 index 00000000000..1100874ad48 --- /dev/null +++ b/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 080a850a8c6..5efef3624db 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -24,7 +24,7 @@ mysql8_node = None node_db = cluster.add_instance( "node1", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users.xml"], with_mysql57=True, with_mysql8=True, @@ -32,7 +32,7 @@ node_db = cluster.add_instance( ) node_disable_bytes_settings = cluster.add_instance( "node2", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql57=False, with_mysql8=False, @@ -40,7 +40,7 @@ node_disable_bytes_settings = cluster.add_instance( ) node_disable_rows_settings = cluster.add_instance( "node3", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users_disable_rows_settings.xml"], with_mysql57=False, with_mysql8=False, diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 7f77627e793..45ae88f427e 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -537,10 +537,7 @@ def test_freeze_unfreeze(cluster): def test_apply_new_settings(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - config_path = os.path.join( - SCRIPT_DIR, - "./_gen/disk_storage_conf.xml".format(cluster.instances_dir_name), - ) + config_path = os.path.join(SCRIPT_DIR, "./_gen/disk_storage_conf.xml") azure_query( node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}" diff --git a/tests/integration/test_modify_engine_on_restart/test_zk_path_exists.py b/tests/integration/test_modify_engine_on_restart/test_zk_path_exists.py new file mode 100644 index 00000000000..3bf492cf69d --- /dev/null +++ b/tests/integration/test_modify_engine_on_restart/test_zk_path_exists.py @@ -0,0 +1,69 @@ +import pytest +from test_modify_engine_on_restart.common import ( + get_table_path, + set_convert_flags, +) +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +ch1 = cluster.add_instance( + "ch1", + main_configs=[ + "configs/config.d/clusters.xml", + "configs/config.d/distributed_ddl.xml", + ], + with_zookeeper=True, + macros={"replica": "node1"}, + stay_alive=True, +) + +database_name = "modify_engine_zk_path" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def q(node, query): + return node.query(database=database_name, sql=query) + + +def test_modify_engine_fails_if_zk_path_exists(started_cluster): + ch1.query("CREATE DATABASE " + database_name) + + q( + ch1, + "CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", + ) + uuid = q( + ch1, + f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'", + ).strip("'[]\n") + + q( + ch1, + f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{uuid}/{{shard}}', 'node2') PARTITION BY toYYYYMM(D) ORDER BY A;", + ) + + set_convert_flags(ch1, database_name, ["already_exists_1"]) + + table_data_path = get_table_path(ch1, "already_exists_1", database_name) + + ch1.stop_clickhouse() + ch1.start_clickhouse(start_wait_sec=120, expected_to_fail=True) + + # Check if we can cancel convertation + ch1.exec_in_container( + [ + "bash", + "-c", + f"rm {table_data_path}convert_to_replicated", + ] + ) + ch1.start_clickhouse() diff --git a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml new file mode 100644 index 00000000000..2d7946d1587 --- /dev/null +++ b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml @@ -0,0 +1,12 @@ + + + zookeeper + /named_collections_path/ + 5000 + + + + value1 + + + diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index cbb8c94c701..dbc502236c0 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -9,6 +9,8 @@ NAMED_COLLECTIONS_CONFIG = os.path.join( SCRIPT_DIR, "./configs/config.d/named_collections.xml" ) +ZK_PATH = "/named_collections_path" + @pytest.fixture(scope="module") def cluster(): @@ -24,6 +26,28 @@ def cluster(): ], stay_alive=True, ) + cluster.add_instance( + "node_with_keeper", + main_configs=[ + "configs/config.d/named_collections_with_zookeeper.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + with_zookeeper=True, + ) + cluster.add_instance( + "node_with_keeper_2", + main_configs=[ + "configs/config.d/named_collections_with_zookeeper.xml", + ], + user_configs=[ + "configs/users.d/users.xml", + ], + stay_alive=True, + with_zookeeper=True, + ) cluster.add_instance( "node_only_named_collection_control", main_configs=[ @@ -447,8 +471,16 @@ def test_config_reload(cluster): ) -def test_sql_commands(cluster): - node = cluster.instances["node"] +@pytest.mark.parametrize("with_keeper", [False, True]) +def test_sql_commands(cluster, with_keeper): + zk = None + node = None + if with_keeper: + node = cluster.instances["node_with_keeper"] + zk = cluster.get_kazoo_client("zoo1") + else: + node = cluster.instances["node"] + assert "1" == node.query("select count() from system.named_collections").strip() node.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'") @@ -479,6 +511,14 @@ def test_sql_commands(cluster): "select collection['key2'] from system.named_collections where name = 'collection2'" ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 1, key2 = 'value2'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) check_created() node.restart_clickhouse() @@ -508,6 +548,15 @@ def test_sql_commands(cluster): ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 4, key2 = 'value2', key3 = 'value3'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + check_altered() node.restart_clickhouse() check_altered() @@ -522,6 +571,15 @@ def test_sql_commands(cluster): ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 4, key3 = 'value3'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + check_deleted() node.restart_clickhouse() check_deleted() @@ -529,6 +587,7 @@ def test_sql_commands(cluster): node.query( "ALTER NAMED COLLECTION collection2 SET key3=3, key4='value4' DELETE key1" ) + time.sleep(2) def check_altered_and_deleted(): assert ( @@ -552,6 +611,15 @@ def test_sql_commands(cluster): ).strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key3 = 3, key4 = 'value4'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + check_altered_and_deleted() node.restart_clickhouse() check_altered_and_deleted() @@ -564,7 +632,132 @@ def test_sql_commands(cluster): "collection1" == node.query("select name from system.named_collections").strip() ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 0 == len(children) check_dropped() node.restart_clickhouse() check_dropped() + + +def test_keeper_storage(cluster): + node1 = cluster.instances["node_with_keeper"] + node2 = cluster.instances["node_with_keeper_2"] + zk = cluster.get_kazoo_client("zoo1") + + assert "1" == node1.query("select count() from system.named_collections").strip() + assert "1" == node2.query("select count() from system.named_collections").strip() + + node1.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'") + + def check_created(node): + assert ( + "collection1\ncollection2" + == node.query("select name from system.named_collections").strip() + ) + + assert ( + "['key1','key2']" + == node.query( + "select mapKeys(collection) from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "1" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "value2" + == node.query( + "select collection['key2'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 1, key2 = 'value2'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + + check_created(node1) + check_created(node2) + + node1.restart_clickhouse() + node2.restart_clickhouse() + + check_created(node1) + check_created(node2) + + node2.query("ALTER NAMED COLLECTION collection2 SET key1=4, key3='value3'") + + time.sleep(5) + + def check_altered(node): + assert ( + "['key1','key2','key3']" + == node.query( + "select mapKeys(collection) from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "4" + == node.query( + "select collection['key1'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + assert ( + "value3" + == node.query( + "select collection['key3'] from system.named_collections where name = 'collection2'" + ).strip() + ) + + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 1 == len(children) + assert "collection2.sql" in children + assert ( + b"CREATE NAMED COLLECTION collection2 AS key1 = 4, key2 = 'value2', key3 = 'value3'" + in zk.get(ZK_PATH + "/collection2.sql")[0] + ) + + check_altered(node2) + check_altered(node1) + + node1.restart_clickhouse() + node2.restart_clickhouse() + + check_altered(node1) + check_altered(node2) + + node1.query("DROP NAMED COLLECTION collection2") + + time.sleep(5) + + def check_dropped(node): + assert "1" == node.query("select count() from system.named_collections").strip() + assert ( + "collection1" + == node.query("select name from system.named_collections").strip() + ) + if zk is not None: + children = zk.get_children(ZK_PATH) + assert 0 == len(children) + + check_dropped(node1) + check_dropped(node2) + + node1.restart_clickhouse() + node2.restart_clickhouse() + + check_dropped(node1) + check_dropped(node2) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index c646a678512..07a9e2badff 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -26,7 +26,7 @@ def create_tables(cluster): n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}") n1.query( - f"CREATE TABLE test_table ON CLUSTER {cluster} (key Int32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" + f"CREATE TABLE test_table ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" ) n1.query( f""" diff --git a/tests/integration/test_parallel_replicas_custom_key_failover/test.py b/tests/integration/test_parallel_replicas_custom_key_failover/test.py index bbb8fd5abb0..3ba3ce092c3 100644 --- a/tests/integration/test_parallel_replicas_custom_key_failover/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_failover/test.py @@ -28,10 +28,10 @@ def create_tables(cluster, table_name): node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data diff --git a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py index d5e17103296..ddfd07a0864 100644 --- a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py @@ -33,7 +33,7 @@ def create_tables(table_name): for i in range(0, 4): nodes[i].query(f"DROP TABLE IF EXISTS {table_name} SYNC") nodes[i].query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)" ) # populate data diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index f04425d83d4..0e87cb0e690 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -179,9 +179,7 @@ def test_different_data_types(started_cluster): for i in range(10): col = random.choice(["a", "b", "c"]) cursor.execute("UPDATE test_data_types SET {} = {};".format(col, i)) - cursor.execute( - """UPDATE test_data_types SET i = '2020-12-12';""".format(col, i) - ) + cursor.execute("UPDATE test_data_types SET i = '2020-12-12';") check_tables_are_synchronized(instance, "test_data_types", "id") @@ -452,7 +450,7 @@ def test_many_concurrent_queries(started_cluster): # also change primary key value print("try update primary key {}".format(thread_id)) cursor.execute( - "UPDATE {table}_{} SET key=key%100000+100000*{} WHERE key%{}=0".format( + "UPDATE {} SET key=key%100000+100000*{} WHERE key%{}=0".format( table_name, i + 1, i + 1 ) ) diff --git a/tests/integration/test_prometheus_endpoint/test.py b/tests/integration/test_prometheus_endpoint/test.py index f140ebdfbe7..c1f04497b55 100644 --- a/tests/integration/test_prometheus_endpoint/test.py +++ b/tests/integration/test_prometheus_endpoint/test.py @@ -28,7 +28,7 @@ def parse_response_line(line): if line.startswith("#"): return {} - match = re.match("^([a-zA-Z_:][a-zA-Z0-9_:]+)(\{.*\})? -?(\d)", line) + match = re.match(r"^([a-zA-Z_:][a-zA-Z0-9_:]+)(\{.*\})? -?(\d)", line) assert match, line name, _, val = match.groups() return {name: int(val)} diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 5150e9096de..706628cf93b 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -6,4 +6,5 @@ 50 42 + false diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml new file mode 100644 index 00000000000..8192c191952 --- /dev/null +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -0,0 +1,11 @@ + + 10 + 1 + + 10 + + 50 + 42 + group + false + diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index fd1bfc75227..f23384b5c04 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -61,7 +61,7 @@ all_nodes = [ bad_settings_node = cluster.add_instance( "bad_settings_node", - main_configs=["configs/config.xml"], + main_configs=["configs/config2.xml"], user_configs=["configs/inconsistent_settings.xml"], with_zookeeper=True, macros={"shard": 1, "replica": 4}, @@ -404,6 +404,8 @@ def test_alter_detach_part(started_cluster, engine): main_node.query(f"INSERT INTO {database}.alter_detach VALUES (123)") if engine == "MergeTree": dummy_node.query(f"INSERT INTO {database}.alter_detach VALUES (456)") + else: + main_node.query(f"SYSTEM SYNC REPLICA {database}.alter_detach PULL") main_node.query(f"ALTER TABLE {database}.alter_detach DETACH PART '{part_name}'") detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_detach'" assert main_node.query(detached_parts_query) == f"{part_name}\n" @@ -1522,3 +1524,24 @@ def test_auto_recovery(started_cluster): assert "42\n" == bad_settings_node.query("SELECT * FROM auto_recovery.t2") assert "137\n" == bad_settings_node.query("SELECT * FROM auto_recovery.t1") + + +def test_all_groups_cluster(started_cluster): + dummy_node.query("DROP DATABASE IF EXISTS db_cluster") + bad_settings_node.query("DROP DATABASE IF EXISTS db_cluster") + dummy_node.query( + "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica1');" + ) + bad_settings_node.query( + "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica2');" + ) + + assert "dummy_node\n" == dummy_node.query( + "select host_name from system.clusters where name='db_cluster' order by host_name" + ) + assert "bad_settings_node\n" == bad_settings_node.query( + "select host_name from system.clusters where name='db_cluster' order by host_name" + ) + assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( + "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" + ) diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml new file mode 100644 index 00000000000..c654074740a --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml @@ -0,0 +1,11 @@ + + + + 1 + 1 + default + 100 + 0 + + + diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml new file mode 100644 index 00000000000..86e392f09ca --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml @@ -0,0 +1,6 @@ + + + node1 + default + + diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py new file mode 100644 index 00000000000..1c6d15d9c7b --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python3 + +import logging +import random +import string +import time + +import pytest +from multiprocessing.dummy import Pool +from helpers.cluster import ClickHouseCluster +import minio + + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.add_instance( + "node1", + main_configs=["configs/node1_macro.xml"], + user_configs=[ + "configs/enable_parallel_replicas.xml", + ], + with_minio=True, + with_zookeeper=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_replicated_db_and_ttl(started_cluster): + node1 = cluster.instances["node1"] + node1.query("DROP DATABASE default") + node1.query("CREATE DATABASE default ENGINE Replicated('/replicated')") + + node1.query( + "CREATE TABLE 02908_main (a UInt32) ENGINE = ReplicatedMergeTree ORDER BY a" + ) + node1.query( + "CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = ReplicatedMergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM 02908_main)" + ) + + node1.query("INSERT INTO 02908_main VALUES (1)") + node1.query("INSERT INTO 02908_dependent VALUES (1, now())") + + node1.query("SELECT * FROM 02908_dependent") diff --git a/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/__init__.py b/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/configs/settings.xml b/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/configs/settings.xml new file mode 100644 index 00000000000..0a390937413 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/configs/settings.xml @@ -0,0 +1,6 @@ + + + 300 + 1 + 128 + diff --git a/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/test.py b/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/test.py new file mode 100644 index 00000000000..515d9530424 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_thread_schedule_timeouts/test.py @@ -0,0 +1,68 @@ +import concurrent.futures + +import pytest +from helpers.cluster import ClickHouseCluster + + +MAX_THREADS = 60 + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + macros={"cluster": "test-cluster", "replica": "node1"}, + main_configs=["configs/settings.xml"], + with_zookeeper=True, +) + + +def prepare_cluster(): + node1.query("DROP TABLE IF EXISTS test_threads_busy SYNC") + node1.query( + """ + CREATE TABLE test_threads_busy(d Date, i Int64, s String) ENGINE=MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY d + """ + ) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def do_slow_select(): + # Do a bunch of slow queries that use a large number of threads to saturate max_thread_pool_size + # explicitly set max_threads as otherwise it's relative to the number of CPU cores + query = ( + "SELECT d, i, s, sleepEachRow(3) from test_threads_busy SETTINGS max_threads=40" + ) + node1.query(query) + + +def test_query_exception_on_thread_pool_full(started_cluster): + prepare_cluster() + # Generate some sample data so sleepEachRow in do_slow_select works + node1.query( + f"INSERT INTO test_threads_busy VALUES ('2024-01-01', 1, 'thread-test')" + ) + + futures = [] + errors = [] + with concurrent.futures.ThreadPoolExecutor(max_workers=MAX_THREADS) as executor: + for _ in range(MAX_THREADS): + futures.append(executor.submit(do_slow_select)) + + for f in futures: + try: + f.result() + except Exception as err: + errors.append(str(err)) + assert len(errors) > 0, "Should be 'Cannot schedule a task' exceptions" + assert all( + "Cannot schedule a task" in err for err in errors + ), "Query threads are stuck, or returned an unexpected error" diff --git a/tests/integration/test_s3_plain_rewritable/configs/storage_conf.xml b/tests/integration/test_s3_plain_rewritable/configs/storage_conf.xml index 560e6b6eca4..23368394494 100644 --- a/tests/integration/test_s3_plain_rewritable/configs/storage_conf.xml +++ b/tests/integration/test_s3_plain_rewritable/configs/storage_conf.xml @@ -8,6 +8,13 @@ minio minio123 + + cache + disk_s3_plain_rewritable + /var/lib/clickhouse/disks/s3_plain_rewritable_cache/ + 1000000000 + 1 + @@ -17,6 +24,13 @@ + + +
+ disk_cache_s3_plain_rewritable +
+
+
diff --git a/tests/integration/test_s3_plain_rewritable/test.py b/tests/integration/test_s3_plain_rewritable/test.py index 06967958631..4b1aaafc814 100644 --- a/tests/integration/test_s3_plain_rewritable/test.py +++ b/tests/integration/test_s3_plain_rewritable/test.py @@ -8,11 +8,8 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) NUM_WORKERS = 5 - MAX_ROWS = 1000 -dirs_created = [] - def gen_insert_values(size): return ",".join( @@ -46,8 +43,14 @@ def start_cluster(): cluster.shutdown() -@pytest.mark.order(0) -def test_insert(): +@pytest.mark.parametrize( + "storage_policy", + [ + pytest.param("s3_plain_rewritable"), + pytest.param("cache_s3_plain_rewritable"), + ], +) +def test(storage_policy): def create_insert(node, insert_values): node.query( """ @@ -56,8 +59,10 @@ def test_insert(): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS storage_policy='s3_plain_rewritable' - """ + SETTINGS storage_policy='{}' + """.format( + storage_policy + ) ) node.query("INSERT INTO test VALUES {}".format(insert_values)) @@ -107,25 +112,6 @@ def test_insert(): != -1 ) - created = int( - node.query( - "SELECT value FROM system.events WHERE event = 'DiskPlainRewritableS3DirectoryCreated'" - ) - ) - assert created > 0 - dirs_created.append(created) - assert ( - int( - node.query( - "SELECT value FROM system.metrics WHERE metric = 'DiskPlainRewritableS3DirectoryMapSize'" - ) - ) - == created - ) - - -@pytest.mark.order(1) -def test_restart(): insert_values_arr = [] for i in range(NUM_WORKERS): node = cluster.instances[f"node{i + 1}"] @@ -138,6 +124,7 @@ def test_restart(): threads = [] for i in range(NUM_WORKERS): + node = cluster.instances[f"node{i + 1}"] t = threading.Thread(target=restart, args=(node,)) threads.append(t) t.start() @@ -152,21 +139,10 @@ def test_restart(): == insert_values_arr[i] ) - -@pytest.mark.order(2) -def test_drop(): for i in range(NUM_WORKERS): node = cluster.instances[f"node{i + 1}"] node.query("DROP TABLE IF EXISTS test SYNC") - removed = int( - node.query( - "SELECT value FROM system.events WHERE event = 'DiskPlainRewritableS3DirectoryRemoved'" - ) - ) - - assert dirs_created[i] == removed - it = cluster.minio_client.list_objects( cluster.minio_bucket, "data/", recursive=True ) diff --git a/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml b/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml index 24c1eb29fbc..84e91495304 100644 --- a/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml +++ b/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml @@ -2,7 +2,6 @@ http://proxy1 - http://proxy2 diff --git a/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/storage_conf.xml b/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/storage_conf.xml index 94ac83b32ac..1d31272a395 100644 --- a/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/storage_conf.xml @@ -3,7 +3,7 @@ s3 - http://minio1:9001/root/data/ + http://minio1:9001/root/data/s3 minio minio123 diff --git a/tests/integration/test_s3_storage_conf_new_proxy/test.py b/tests/integration/test_s3_storage_conf_new_proxy/test.py index c98eb05a217..3b3b07aaa09 100644 --- a/tests/integration/test_s3_storage_conf_new_proxy/test.py +++ b/tests/integration/test_s3_storage_conf_new_proxy/test.py @@ -3,6 +3,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster +import helpers.s3_url_proxy_tests_util as proxy_util @pytest.fixture(scope="module") @@ -26,41 +27,8 @@ def cluster(): cluster.shutdown() -def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET"}): - for i in range(10): - logs = cluster.get_container_logs(proxy_instance) - # Check with retry that all possible interactions with Minio are present - for http_method in http_methods: - if logs.find(http_method + " http://minio1") >= 0: - return - time.sleep(1) - else: - assert False, f"{http_methods} method not found in logs of {proxy_instance}" - - @pytest.mark.parametrize("policy", ["s3"]) def test_s3_with_proxy_list(cluster, policy): - node = cluster.instances["node"] - - node.query( - """ - CREATE TABLE s3_test ( - id Int64, - data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS storage_policy='{}' - """.format( - policy - ) + proxy_util.simple_storage_test( + cluster, cluster.instances["node"], ["proxy1"], policy ) - node.query("INSERT INTO s3_test VALUES (0,'data'),(1,'data')") - assert ( - node.query("SELECT * FROM s3_test order by id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - node.query("DROP TABLE IF EXISTS s3_test SYNC") - - for proxy in ["proxy1", "proxy2"]: - check_proxy_logs(cluster, proxy, ["PUT", "GET"]) diff --git a/tests/integration/test_s3_storage_conf_proxy/configs/config.d/storage_conf.xml b/tests/integration/test_s3_storage_conf_proxy/configs/config.d/storage_conf.xml index 132eac4a2a6..73e7e8175c5 100644 --- a/tests/integration/test_s3_storage_conf_proxy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_s3_storage_conf_proxy/configs/config.d/storage_conf.xml @@ -3,7 +3,7 @@ s3 - http://minio1:9001/root/data/ + http://minio1:9001/root/data/s3 minio minio123 @@ -13,9 +13,10 @@ s3 - http://minio1:9001/root/data/ + http://minio1:9001/root/data/s3_with_resolver minio minio123 + true + + + http://resolver:8080/hostname + http + 80 + 10 + + + + diff --git a/tests/integration/test_s3_table_function_with_http_proxy/proxy-resolver/resolver.py b/tests/integration/test_s3_table_function_with_http_proxy/proxy-resolver/resolver.py index 8c7611303b8..eaea4c1dab2 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/proxy-resolver/resolver.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/proxy-resolver/resolver.py @@ -5,7 +5,10 @@ import bottle @bottle.route("/hostname") def index(): - return "proxy1" + if random.randrange(2) == 0: + return "proxy1" + else: + return "proxy2" bottle.run(host="0.0.0.0", port=8080) diff --git a/tests/integration/test_s3_table_function_with_http_proxy/test.py b/tests/integration/test_s3_table_function_with_http_proxy/test.py index 1619b413099..2ec73ecbef6 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/test.py @@ -19,6 +19,14 @@ def cluster(): with_minio=True, ) + cluster.add_instance( + "remote_proxy_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_remote_no_proxy.xml", + ], + with_minio=True, + ) + cluster.add_instance( "proxy_list_node", main_configs=[ @@ -27,6 +35,14 @@ def cluster(): with_minio=True, ) + cluster.add_instance( + "proxy_list_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_list_no_proxy.xml", + ], + with_minio=True, + ) + cluster.add_instance( "env_node", with_minio=True, @@ -36,6 +52,16 @@ def cluster(): instance_env_variables=True, ) + cluster.add_instance( + "env_node_no_proxy", + with_minio=True, + env_variables={ + "http_proxy": "http://proxy1", + "no_proxy": "not_important_host,, minio1 ,", + }, + instance_env_variables=True, + ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -48,12 +74,30 @@ def cluster(): cluster.shutdown() +def test_s3_with_http_proxy_list_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1", "proxy2"], "http", "proxy_list_node_no_proxy" + ) + + +def test_s3_with_http_remote_proxy_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "http", "remote_proxy_node_no_proxy" + ) + + +def test_s3_with_http_env_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "http", "env_node_no_proxy" + ) + + def test_s3_with_http_proxy_list(cluster): proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "http", "proxy_list_node") def test_s3_with_http_remote_proxy(cluster): - proxy_util.simple_test(cluster, ["proxy1"], "http", "remote_proxy_node") + proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "http", "remote_proxy_node") def test_s3_with_http_env_proxy(cluster): diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list.xml index 4dad8a2a682..e2fca64766a 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list.xml +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list.xml @@ -1,9 +1,5 @@ - - http://proxy1 - http://proxy2 - https://proxy1 https://proxy2 diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml new file mode 100644 index 00000000000..0a03986f839 --- /dev/null +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_list_no_proxy.xml @@ -0,0 +1,13 @@ + + + not_important_host,, minio1 , + + http://proxy1 + http://proxy2 + + + https://proxy1 + https://proxy2 + + + diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml new file mode 100644 index 00000000000..943f2b36a34 --- /dev/null +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote_no_proxy.xml @@ -0,0 +1,18 @@ + + + not_important_host,, minio1 , + + + + http://resolver:8080/hostname + https + 443 + 10 + + + + diff --git a/tests/integration/test_s3_table_function_with_https_proxy/proxy-resolver/resolver.py b/tests/integration/test_s3_table_function_with_https_proxy/proxy-resolver/resolver.py index 8c7611303b8..eaea4c1dab2 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/proxy-resolver/resolver.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/proxy-resolver/resolver.py @@ -5,7 +5,10 @@ import bottle @bottle.route("/hostname") def index(): - return "proxy1" + if random.randrange(2) == 0: + return "proxy1" + else: + return "proxy2" bottle.run(host="0.0.0.0", port=8080) diff --git a/tests/integration/test_s3_table_function_with_https_proxy/test.py b/tests/integration/test_s3_table_function_with_https_proxy/test.py index 83af407093c..54452dda401 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/test.py @@ -23,6 +23,15 @@ def cluster(): minio_certs_dir="minio_certs", ) + cluster.add_instance( + "remote_proxy_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_remote_no_proxy.xml", + "configs/config.d/ssl.xml", + ], + with_minio=True, + ) + cluster.add_instance( "proxy_list_node", main_configs=[ @@ -32,6 +41,15 @@ def cluster(): with_minio=True, ) + cluster.add_instance( + "proxy_list_node_no_proxy", + main_configs=[ + "configs/config.d/proxy_list_no_proxy.xml", + "configs/config.d/ssl.xml", + ], + with_minio=True, + ) + cluster.add_instance( "env_node", main_configs=[ @@ -44,6 +62,19 @@ def cluster(): instance_env_variables=True, ) + cluster.add_instance( + "env_node_no_proxy", + main_configs=[ + "configs/config.d/ssl.xml", + ], + with_minio=True, + env_variables={ + "https_proxy": "https://proxy1", + "no_proxy": "not_important_host,, minio1 ,", + }, + instance_env_variables=True, + ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -56,12 +87,30 @@ def cluster(): cluster.shutdown() +def test_s3_with_https_proxy_list_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1", "proxy2"], "https", "proxy_list_node_no_proxy" + ) + + +def test_s3_with_https_env_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "https", "env_node_no_proxy" + ) + + +def test_s3_with_https_remote_no_proxy(cluster): + proxy_util.simple_test_assert_no_proxy( + cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy" + ) + + def test_s3_with_https_proxy_list(cluster): proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node") def test_s3_with_https_remote_proxy(cluster): - proxy_util.simple_test(cluster, ["proxy1"], "https", "remote_proxy_node") + proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "remote_proxy_node") def test_s3_with_https_env_proxy(cluster): diff --git a/tests/integration/test_scheduler/configs/resources.xml b/tests/integration/test_scheduler/configs/resources.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_scheduler/configs/resources.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_scheduler/configs/resources.xml.default b/tests/integration/test_scheduler/configs/resources.xml.default new file mode 100644 index 00000000000..3b003a17557 --- /dev/null +++ b/tests/integration/test_scheduler/configs/resources.xml.default @@ -0,0 +1,76 @@ + + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + fair90 + fifo + fifo + fifo9 + fifo9 + fifo9 + fifo9 + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + fair90 + fifo + fifo + fifo9 + fifo9 + fifo9 + fifo9 + + + + + /prio/admin + /prio/admin + + + /prio/fair/prod + /prio/fair/prod + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/sys/merges + /prio/fair/sys/merges + + + /prio/fair/sys/mutations + /prio/fair/sys/mutations + + + /prio/fair/prod_merges + /prio/fair/prod_merges + + + /prio/fair/prod_mutations + /prio/fair/prod_mutations + + + /prio/fair/dev_merges + /prio/fair/dev_merges + + + /prio/fair/dev_mutations + /prio/fair/dev_mutations + + + diff --git a/tests/integration/test_scheduler/configs/scheduler.xml b/tests/integration/test_scheduler/configs/scheduler.xml deleted file mode 100644 index 523ba1a5a98..00000000000 --- a/tests/integration/test_scheduler/configs/scheduler.xml +++ /dev/null @@ -1,62 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - 33554432 - 10 - 10 - network_read - network_write - - - - - -
- s3 -
-
-
-
-
- - - inflight_limit1000000 - priority - fifo0 - fair1 - fifo9 - fifo1 - - - inflight_limit1000000 - priority - fifo0 - fair1 - fifo9 - fifo1 - - - - - /prio/admin - /prio/admin - - - /prio/fair/prod - /prio/fair/prod - - - /prio/fair/dev - /prio/fair/dev - - - /prio/fair/dev - /prio/fair/dev - - -
diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml new file mode 100644 index 00000000000..823a00a05de --- /dev/null +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -0,0 +1,26 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + network_read + network_write + + + + + +
+ s3 +
+
+
+
+
+
diff --git a/tests/integration/test_scheduler/configs/workloads.xml b/tests/integration/test_scheduler/configs/workloads.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_scheduler/configs/workloads.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_scheduler/configs/workloads.xml.default b/tests/integration/test_scheduler/configs/workloads.xml.default new file mode 100644 index 00000000000..f010993335d --- /dev/null +++ b/tests/integration/test_scheduler/configs/workloads.xml.default @@ -0,0 +1,4 @@ + + sys_merges + sys_mutations + diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e6def99c076..cde75c244e8 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -6,6 +6,7 @@ import time import threading import pytest +from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -13,7 +14,13 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", stay_alive=True, - main_configs=["configs/scheduler.xml"], + main_configs=[ + "configs/storage_configuration.xml", + "configs/resources.xml", + "configs/resources.xml.default", + "configs/workloads.xml", + "configs/workloads.xml.default", + ], with_minio=True, ) @@ -27,6 +34,41 @@ def start_cluster(): cluster.shutdown() +@pytest.fixture(scope="function", autouse=True) +def set_default_configs(): + node.exec_in_container( + [ + "bash", + "-c", + "cp /etc/clickhouse-server/config.d/resources.xml.default /etc/clickhouse-server/config.d/resources.xml", + ] + ) + node.exec_in_container( + [ + "bash", + "-c", + "cp /etc/clickhouse-server/config.d/workloads.xml.default /etc/clickhouse-server/config.d/workloads.xml", + ] + ) + node.query("system reload config") + yield + + +def update_workloads_config(**settings): + xml = "" + for name in settings: + xml += f"<{name}>{settings[name]}" + print(xml) + node.exec_in_container( + [ + "bash", + "-c", + f"echo '{xml}' > /etc/clickhouse-server/config.d/workloads.xml", + ] + ) + node.query("system reload config") + + def test_s3_disk(): node.query( f""" @@ -110,3 +152,302 @@ def test_s3_disk(): ) == "1\n" ) + + +def test_merge_workload(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'" + ).strip() + ) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"insert into data select * from numbers(2e4)") + node.query(f"insert into data select * from numbers(3e4)") + node.query(f"optimize table data final") + + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'" + ).strip() + ) + + assert reads_before < reads_after + assert writes_before < writes_after + + +def test_merge_workload_override(): + node.query( + f""" + drop table if exists prod_data; + drop table if exists dev_data; + create table prod_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', merge_workload='prod_merges'; + create table dev_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', merge_workload='dev_merges'; + """ + ) + + prod_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'" + ).strip() + ) + prod_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'" + ).strip() + ) + dev_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'" + ).strip() + ) + dev_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'" + ).strip() + ) + + node.query(f"insert into prod_data select * from numbers(1e4)") + node.query(f"insert into prod_data select * from numbers(2e4)") + node.query(f"insert into prod_data select * from numbers(3e4)") + node.query(f"insert into dev_data select * from numbers(1e4)") + node.query(f"insert into dev_data select * from numbers(2e4)") + node.query(f"insert into dev_data select * from numbers(3e4)") + node.query(f"optimize table prod_data final") + node.query(f"optimize table dev_data final") + + prod_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'" + ).strip() + ) + prod_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'" + ).strip() + ) + dev_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'" + ).strip() + ) + dev_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'" + ).strip() + ) + + assert prod_reads_before < prod_reads_after + assert prod_writes_before < prod_writes_after + assert dev_reads_before < dev_reads_after + assert dev_writes_before < dev_writes_after + + +def test_mutate_workload(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"optimize table data final") + + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'" + ).strip() + ) + + node.query(f"alter table data update key = 1 where key = 42") + node.query(f"optimize table data final") + + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'" + ).strip() + ) + + assert reads_before < reads_after + assert writes_before < writes_after + + +def test_mutation_workload_override(): + node.query( + f""" + drop table if exists prod_data; + drop table if exists dev_data; + create table prod_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', mutation_workload='prod_mutations'; + create table dev_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', mutation_workload='dev_mutations'; + """ + ) + + node.query(f"insert into prod_data select * from numbers(1e4)") + node.query(f"optimize table prod_data final") + node.query(f"insert into dev_data select * from numbers(1e4)") + node.query(f"optimize table dev_data final") + + prod_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'" + ).strip() + ) + prod_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'" + ).strip() + ) + dev_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'" + ).strip() + ) + dev_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'" + ).strip() + ) + + node.query(f"alter table prod_data update key = 1 where key = 42") + node.query(f"optimize table prod_data final") + node.query(f"alter table dev_data update key = 1 where key = 42") + node.query(f"optimize table dev_data final") + + prod_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'" + ).strip() + ) + prod_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'" + ).strip() + ) + dev_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'" + ).strip() + ) + dev_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'" + ).strip() + ) + + assert prod_reads_before < prod_reads_after + assert prod_writes_before < prod_writes_after + assert dev_reads_before < dev_reads_after + assert dev_writes_before < dev_writes_after + + +def test_merge_workload_change(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + for env in ["prod", "dev"]: + update_workloads_config(merge_workload=f"{env}_merges") + + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'" + ).strip() + ) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"insert into data select * from numbers(2e4)") + node.query(f"insert into data select * from numbers(3e4)") + node.query(f"optimize table data final") + + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'" + ).strip() + ) + + assert reads_before < reads_after + assert writes_before < writes_after + + +def test_mutation_workload_change(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + for env in ["prod", "dev"]: + update_workloads_config(mutation_workload=f"{env}_mutations") + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"optimize table data final") + + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + + node.query(f"alter table data update key = 1 where key = 42") + node.query(f"optimize table data final") + + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + + assert reads_before < reads_after + assert writes_before < writes_after diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f836c58ce30..d986c1f9746 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -790,6 +790,25 @@ def test_read_subcolumns(cluster): assert res == "42\tcont/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" +def test_read_subcolumn_time(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a UInt32') select (42)", + ) + + res = node.query( + f"select a, dateDiff('minute', _time, now()) < 59 from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a UInt32')" + ) + + assert res == "42\t1\n" + + def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] query = ( diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 44c0223e677..47d8f44c0b7 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -999,6 +999,20 @@ def test_read_subcolumns(started_cluster): assert res == "42\ttest_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" +def test_read_subcolumn_time(started_cluster): + node = started_cluster.instances["node1"] + + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a UInt32') select (42)" + ) + + res = node.query( + f"select a, dateDiff('minute', _time, now()) < 59 from hdfs('hdfs://hdfs1:9000/test_subcolumn_time.tsv', auto, 'a UInt32')" + ) + + assert res == "42\t1\n" + + def test_union_schema_inference_mode(started_cluster): node = started_cluster.instances["node1"] diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 6fe7a193129..0c3fbfb3cda 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -4,7 +4,7 @@ import os import json import helpers.client -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index d9dee0541b0..7762d17b96f 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -1,5 +1,5 @@ import helpers.client -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV import pyspark diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 23a95d5dd71..3240039ee81 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -702,7 +702,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages: {}".format(result1) assert int(result2) == 10 @@ -1516,7 +1516,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages: {}".format(result1) assert int(result2) == 4 * num_tables @@ -1966,7 +1966,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result) + ), "ClickHouse lost some messages: {}".format(result1) # 4 tables, 2 consumers for each table => 8 consumer tags assert int(result2) == 8 @@ -2427,9 +2427,7 @@ def test_rabbitmq_drop_table_properly(rabbitmq_cluster): time.sleep(30) try: - exists = channel.queue_declare( - callback, queue="rabbit_queue_drop", passive=True - ) + exists = channel.queue_declare(queue="rabbit_queue_drop", passive=True) except Exception as e: exists = False @@ -3364,7 +3362,7 @@ def test_rabbitmq_flush_by_block_size(rabbitmq_cluster): routing_key="", body=json.dumps({"key": 0, "value": 0}), ) - except e: + except Exception as e: logging.debug(f"Got error: {str(e)}") produce_thread = threading.Thread(target=produce) @@ -3442,7 +3440,7 @@ def test_rabbitmq_flush_by_time(rabbitmq_cluster): ) logging.debug("Produced a message") time.sleep(0.8) - except e: + except Exception as e: logging.debug(f"Got error: {str(e)}") produce_thread = threading.Thread(target=produce) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 09b27fff1e8..b2ebd12ce00 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1133,6 +1133,7 @@ def test_seekable_formats(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", + timeout=100, ) result = instance.query(f"SELECT count() FROM {table_function}") @@ -1142,6 +1143,7 @@ def test_seekable_formats(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", + timeout=100, ) result = instance.query( @@ -1169,6 +1171,7 @@ def test_seekable_formats_url(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", + timeout=100, ) result = instance.query(f"SELECT count() FROM {table_function}") @@ -1178,6 +1181,7 @@ def test_seekable_formats_url(started_cluster): exec_query_with_retry( instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", + timeout=100, ) table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" @@ -2149,6 +2153,21 @@ def test_read_subcolumns(started_cluster): ) +def test_read_subcolumn_time(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a UInt32') select (42)" + ) + + res = instance.query( + f"select a, dateDiff('minute', _time, now()) < 59 from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumn_time.tsv', auto, 'a UInt32')" + ) + + assert res == "42\t1\n" + + def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] diff --git a/tests/integration/test_storage_s3_queue/configs/zookeeper.xml b/tests/integration/test_storage_s3_queue/configs/zookeeper.xml index 27334dca590..1115d335f4f 100644 --- a/tests/integration/test_storage_s3_queue/configs/zookeeper.xml +++ b/tests/integration/test_storage_s3_queue/configs/zookeeper.xml @@ -13,4 +13,19 @@ 2181 + + + + + + instance + 9000 + + + instance2 + 9000 + + + +
diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index e7925d55d00..66631c51b03 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -711,9 +711,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): table_name, mode, files_path, - additional_settings={ - "keeper_path": keeper_path, - }, + additional_settings={"keeper_path": keeper_path, "s3queue_buckets": 2}, ) for instance in [node, node_2]: @@ -1031,6 +1029,29 @@ def test_s3_client_reused(started_cluster): assert s3_clients_before == s3_clients_after +def get_processed_files(node, table_name): + return ( + node.query( + f""" +select splitByChar('/', file_name)[-1] as file +from system.s3queue where zookeeper_path ilike '%{table_name}%' and status = 'Processed' order by file + """ + ) + .strip() + .split("\n") + ) + + +def get_unprocessed_files(node, table_name): + return node.query( + f""" + select concat('test_', toString(number), '.csv') as file from numbers(300) + where file not + in (select splitByChar('/', file_name)[-1] from system.s3queue where zookeeper_path ilike '%{table_name}%' and status = 'Processed') + """ + ) + + @pytest.mark.parametrize("mode", ["unordered", "ordered"]) def test_processing_threads(started_cluster, mode): node = started_cluster.instances["instance"] @@ -1061,12 +1082,18 @@ def test_processing_threads(started_cluster, mode): def get_count(table_name): return int(run_query(node, f"SELECT count() FROM {table_name}")) - for _ in range(100): + for _ in range(50): if (get_count(f"{dst_table_name}")) == files_to_generate: break time.sleep(1) - assert get_count(dst_table_name) == files_to_generate + if get_count(dst_table_name) != files_to_generate: + processed_files = get_processed_files(node, table_name) + unprocessed_files = get_unprocessed_files(node, table_name) + logging.debug( + f"Processed files: {len(processed_files)}/{files_to_generate}, unprocessed files: {unprocessed_files}, count: {get_count(dst_table_name)}" + ) + assert False res = [ list(map(int, l.split())) @@ -1078,7 +1105,9 @@ def test_processing_threads(started_cluster, mode): if mode == "ordered": zk = started_cluster.get_kazoo_client("zoo1") - processed_nodes = zk.get_children(f"{keeper_path}/processed/") + nodes = zk.get_children(f"{keeper_path}") + print(f"Metadata nodes: {nodes}") + processed_nodes = zk.get_children(f"{keeper_path}/buckets/") assert len(processed_nodes) == processing_threads @@ -1112,7 +1141,7 @@ def test_shards(started_cluster, mode, processing_threads): additional_settings={ "keeper_path": keeper_path, "s3queue_processing_threads_num": processing_threads, - "s3queue_total_shards_num": shards_num, + "s3queue_buckets": shards_num, }, ) create_mv(node, table, dst_table) @@ -1124,13 +1153,15 @@ def test_shards(started_cluster, mode, processing_threads): def get_count(table_name): return int(run_query(node, f"SELECT count() FROM {table_name}")) - for _ in range(100): - if ( + for _ in range(30): + count = ( get_count(f"{dst_table_name}_1") + get_count(f"{dst_table_name}_2") + get_count(f"{dst_table_name}_3") - ) == files_to_generate: + ) + if count == files_to_generate: break + print(f"Current {count}/{files_to_generate}") time.sleep(1) if ( @@ -1138,10 +1169,36 @@ def test_shards(started_cluster, mode, processing_threads): + get_count(f"{dst_table_name}_2") + get_count(f"{dst_table_name}_3") ) != files_to_generate: - info = node.query( - f"SELECT * FROM system.s3queue WHERE zookeeper_path like '%{table_name}' ORDER BY file_name FORMAT Vertical" + processed_files = ( + node.query( + f""" +select splitByChar('/', file_name)[-1] as file from system.s3queue +where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0 order by file + """ + ) + .strip() + .split("\n") ) - logging.debug(info) + logging.debug( + f"Processed files: {len(processed_files)}/{files_to_generate}: {processed_files}" + ) + + count = ( + get_count(f"{dst_table_name}_1") + + get_count(f"{dst_table_name}_2") + + get_count(f"{dst_table_name}_3") + ) + logging.debug(f"Processed rows: {count}/{files_to_generate}") + + info = node.query( + f""" + select concat('test_', toString(number), '.csv') as file from numbers(300) + where file not in (select splitByChar('/', file_name)[-1] from system.s3queue + where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0) + """ + ) + logging.debug(f"Unprocessed files: {info}") + assert False res1 = [ @@ -1176,10 +1233,8 @@ def test_shards(started_cluster, mode, processing_threads): if mode == "ordered": zk = started_cluster.get_kazoo_client("zoo1") - processed_nodes = zk.get_children(f"{keeper_path}/processed/") - assert len(processed_nodes) == shards_num * processing_threads - shard_nodes = zk.get_children(f"{keeper_path}/shards/") - assert len(shard_nodes) == shards_num + processed_nodes = zk.get_children(f"{keeper_path}/buckets/") + assert len(processed_nodes) == shards_num @pytest.mark.parametrize( @@ -1214,7 +1269,7 @@ def test_shards_distributed(started_cluster, mode, processing_threads): additional_settings={ "keeper_path": keeper_path, "s3queue_processing_threads_num": processing_threads, - "s3queue_total_shards_num": shards_num, + "s3queue_buckets": shards_num, }, ) i += 1 @@ -1229,7 +1284,7 @@ def test_shards_distributed(started_cluster, mode, processing_threads): def get_count(node, table_name): return int(run_query(node, f"SELECT count() FROM {table_name}")) - for _ in range(150): + for _ in range(10): if ( get_count(node, dst_table_name) + get_count(node_2, dst_table_name) ) == total_rows: @@ -1239,10 +1294,69 @@ def test_shards_distributed(started_cluster, mode, processing_threads): if ( get_count(node, dst_table_name) + get_count(node_2, dst_table_name) ) != total_rows: - info = node.query( - f"SELECT * FROM system.s3queue WHERE zookeeper_path like '%{table_name}' ORDER BY file_name FORMAT Vertical" + processed_files = ( + node.query( + f""" +select splitByChar('/', file_name)[-1] as file from system.s3queue where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0 order by file + """ + ) + .strip() + .split("\n") ) - logging.debug(info) + logging.debug( + f"Processed files by node 1: {len(processed_files)}/{files_to_generate}" + ) + processed_files = ( + node_2.query( + f""" +select splitByChar('/', file_name)[-1] as file from system.s3queue where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0 order by file + """ + ) + .strip() + .split("\n") + ) + logging.debug( + f"Processed files by node 2: {len(processed_files)}/{files_to_generate}" + ) + + count = get_count(node, dst_table_name) + get_count(node_2, dst_table_name) + logging.debug(f"Processed rows: {count}/{files_to_generate}") + + info = node.query( + f""" + select concat('test_', toString(number), '.csv') as file from numbers(300) + where file not in (select splitByChar('/', file_name)[-1] from clusterAllReplicas(default, system.s3queue) + where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0) + """ + ) + logging.debug(f"Unprocessed files: {info}") + + files1 = ( + node.query( + f""" + select splitByChar('/', file_name)[-1] from system.s3queue + where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0 + """ + ) + .strip() + .split("\n") + ) + files2 = ( + node_2.query( + f""" + select splitByChar('/', file_name)[-1] from system.s3queue + where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_processed > 0 + """ + ) + .strip() + .split("\n") + ) + + def intersection(list_a, list_b): + return [e for e in list_a if e in list_b] + + logging.debug(f"Intersecting files: {intersection(files1, files2)}") + assert False get_query = f"SELECT column1, column2, column3 FROM {dst_table_name}" @@ -1267,10 +1381,8 @@ def test_shards_distributed(started_cluster, mode, processing_threads): if mode == "ordered": zk = started_cluster.get_kazoo_client("zoo1") - processed_nodes = zk.get_children(f"{keeper_path}/processed/") - assert len(processed_nodes) == shards_num * processing_threads - shard_nodes = zk.get_children(f"{keeper_path}/shards/") - assert len(shard_nodes) == shards_num + processed_nodes = zk.get_children(f"{keeper_path}/buckets/") + assert len(processed_nodes) == shards_num node.restart_clickhouse() time.sleep(10) @@ -1297,12 +1409,12 @@ def test_settings_check(started_cluster): additional_settings={ "keeper_path": keeper_path, "s3queue_processing_threads_num": 5, - "s3queue_total_shards_num": 2, + "s3queue_buckets": 2, }, ) assert ( - "Existing table metadata in ZooKeeper differs in s3queue_total_shards_num setting. Stored in ZooKeeper: 2, local: 3" + "Existing table metadata in ZooKeeper differs in s3queue_buckets setting. Stored in ZooKeeper: 2, local: 3" in create_table( started_cluster, node_2, @@ -1312,39 +1424,12 @@ def test_settings_check(started_cluster): additional_settings={ "keeper_path": keeper_path, "s3queue_processing_threads_num": 5, - "s3queue_total_shards_num": 3, + "s3queue_buckets": 3, }, expect_error=True, ) ) - assert ( - "Existing table metadata in ZooKeeper differs in s3queue_processing_threads_num setting. Stored in ZooKeeper: 5, local: 2" - in create_table( - started_cluster, - node_2, - table_name, - mode, - files_path, - additional_settings={ - "keeper_path": keeper_path, - "s3queue_processing_threads_num": 2, - "s3queue_total_shards_num": 2, - }, - expect_error=True, - ) - ) - - assert "s3queue_current_shard_num = 0" in node.query( - f"SHOW CREATE TABLE {table_name}" - ) - - node.restart_clickhouse() - - assert "s3queue_current_shard_num = 0" in node.query( - f"SHOW CREATE TABLE {table_name}" - ) - node.query(f"DROP TABLE {table_name} SYNC") @@ -1353,7 +1438,7 @@ def test_processed_file_setting(started_cluster, processing_threads): node = started_cluster.instances["instance"] table_name = f"test_processed_file_setting_{processing_threads}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + keeper_path = f"/clickhouse/test_{table_name}_{processing_threads}" files_path = f"{table_name}_data" files_to_generate = 10 @@ -1419,7 +1504,7 @@ def test_processed_file_setting_distributed(started_cluster, processing_threads) "keeper_path": keeper_path, "s3queue_processing_threads_num": processing_threads, "s3queue_last_processed_path": f"{files_path}/test_5.csv", - "s3queue_total_shards_num": 2, + "s3queue_buckets": 2, }, ) diff --git a/tests/integration/test_table_db_num_limit/__init__.py b/tests/integration/test_table_db_num_limit/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_table_db_num_limit/config/config.xml b/tests/integration/test_table_db_num_limit/config/config.xml new file mode 100644 index 00000000000..9a573b158fe --- /dev/null +++ b/tests/integration/test_table_db_num_limit/config/config.xml @@ -0,0 +1,5 @@ + + 10 + 10 + + diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py new file mode 100644 index 00000000000..aa8030b077c --- /dev/null +++ b/tests/integration/test_table_db_num_limit/test.py @@ -0,0 +1,43 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["config/config.xml"], with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_table_db_limit(started_cluster): + for i in range(10): + node1.query("create database db{}".format(i)) + + with pytest.raises(QueryRuntimeException) as exp_info: + node1.query("create database db_exp".format(i)) + + assert "TOO_MANY_DATABASES" in str(exp_info) + + for i in range(10): + node1.query("create table t{} (a Int32) Engine = Log".format(i)) + + node1.query("system flush logs") + for i in range(10): + node1.query("drop table t{}".format(i)) + for i in range(10): + node1.query("create table t{} (a Int32) Engine = Log".format(i)) + + with pytest.raises(QueryRuntimeException) as exp_info: + node1.query("create table default.tx (a Int32) Engine = Log") + assert "TOO_MANY_TABLES" in str(exp_info) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 94432b89ab6..3b79ea7916d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1850,7 +1850,7 @@ class TestCancelBackgroundMoving: config = inspect.cleandoc( f""" - { 256 * 1024 } + {256 * 1024} """ ) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index f944adbea41..538322473ee 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -325,7 +325,7 @@ def optimize_with_retry(node, table_name, retry=20): settings={"optimize_throw_if_noop": "1"}, ) break - except e: + except: time.sleep(0.5) diff --git a/tests/integration/test_unknown_column_dist_table_with_alias/__init__.py b/tests/integration/test_unknown_column_dist_table_with_alias/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_unknown_column_dist_table_with_alias/configs/clusters.xml b/tests/integration/test_unknown_column_dist_table_with_alias/configs/clusters.xml new file mode 100644 index 00000000000..754d765f23f --- /dev/null +++ b/tests/integration/test_unknown_column_dist_table_with_alias/configs/clusters.xml @@ -0,0 +1,12 @@ + + + + + + localhost + 9000 + + + + + diff --git a/tests/integration/test_unknown_column_dist_table_with_alias/test.py b/tests/integration/test_unknown_column_dist_table_with_alias/test.py new file mode 100644 index 00000000000..0d3890f3e09 --- /dev/null +++ b/tests/integration/test_unknown_column_dist_table_with_alias/test.py @@ -0,0 +1,35 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import logging + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/clusters.xml"]) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +@pytest.mark.parametrize("prefer_localhost_replica", [0, 1]) +def test_distributed_table_with_alias(start_cluster, prefer_localhost_replica): + node.query( + """ + DROP TABLE IF EXISTS local; + DROP TABLE IF EXISTS dist; + CREATE TABLE local(`dummy` UInt8) ENGINE = MergeTree ORDER BY tuple(); + CREATE TABLE dist AS local ENGINE = Distributed(localhost_cluster, currentDatabase(), local); + """ + ) + + node.query( + "WITH 'Hello' AS `alias` SELECT `alias` FROM dist GROUP BY `alias`;", + settings={"prefer_localhost_replica": prefer_localhost_replica}, + ) diff --git a/tests/performance/least_greatest.xml b/tests/performance/least_greatest.xml new file mode 100644 index 00000000000..522bcd9004d --- /dev/null +++ b/tests/performance/least_greatest.xml @@ -0,0 +1,10 @@ + + CREATE TABLE test (id Int32, x1 Nullable(Int32), x2 Nullable(Float32)) ENGINE = MergeTree() ORDER BY id + + INSERT INTO test SELECT number, number+1, number + 2 FROM numbers(1000000) + + SELECT COUNT(1) FROM test WHERE least(x1, x2) > 1 + SELECT COUNT(1) FROM test WHERE GREATEST(x1, x2) > 1 + + DROP TABLE IF EXISTS test + \ No newline at end of file diff --git a/tests/queries/0_stateless/00098_k_union_all.sql b/tests/queries/0_stateless/00098_k_union_all.sql index 059d27075d7..280ad5ca2e6 100644 --- a/tests/queries/0_stateless/00098_k_union_all.sql +++ b/tests/queries/0_stateless/00098_k_union_all.sql @@ -1,4 +1,5 @@ SET output_format_pretty_color=1; +SET output_format_pretty_display_footer_column_names=0; SELECT 1 FORMAT PrettySpace; SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace; SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace; diff --git a/tests/queries/0_stateless/00137_in_constants.reference b/tests/queries/0_stateless/00137_in_constants.reference index 379885fb1ab..94607ffa924 100644 --- a/tests/queries/0_stateless/00137_in_constants.reference +++ b/tests/queries/0_stateless/00137_in_constants.reference @@ -13,6 +13,7 @@ 1 1 1 +1 0 0 0 diff --git a/tests/queries/0_stateless/00137_in_constants.sql b/tests/queries/0_stateless/00137_in_constants.sql index 297acc4ef26..bc365523be1 100644 --- a/tests/queries/0_stateless/00137_in_constants.sql +++ b/tests/queries/0_stateless/00137_in_constants.sql @@ -13,6 +13,7 @@ SELECT 'Hello' IN (SELECT 'Hello'); SELECT materialize('Hello') IN (SELECT 'Hello'); SELECT 'Hello' IN (SELECT materialize('Hello')); SELECT materialize('Hello') IN (SELECT materialize('Hello')); +SELECT toDate('2020-01-01') IN (toDateTime('2020-01-01', 'UTC')); SELECT 2 IN (SELECT 1); SELECT materialize(2) IN (SELECT 1); diff --git a/tests/queries/0_stateless/00300_csv.reference b/tests/queries/0_stateless/00300_csv.reference index e7966a9e8d9..42cd22078c4 100644 --- a/tests/queries/0_stateless/00300_csv.reference +++ b/tests/queries/0_stateless/00300_csv.reference @@ -1,11 +1,11 @@ -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" "x","y","z","a","b" -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" "x","y","z","a","b" "String","UInt8","Array(UInt8)","Tuple(UInt16, Array(String))","String" -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" 0,"0","[]","2000-01-01","2000-01-01 00:00:00" 1,"1","[0]","2000-01-02","2000-01-01 00:00:01" diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index 5c0ae4d2c3b..e637ee0363a 100644 Binary files a/tests/queries/0_stateless/00309_formats.reference and b/tests/queries/0_stateless/00309_formats.reference differ diff --git a/tests/queries/0_stateless/00405_output_format_pretty_color.sql b/tests/queries/0_stateless/00405_output_format_pretty_color.sql index bc2d0c3adbf..de83567dd5d 100644 --- a/tests/queries/0_stateless/00405_output_format_pretty_color.sql +++ b/tests/queries/0_stateless/00405_output_format_pretty_color.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_color = 0; SHOW SETTING output_format_pretty_color; diff --git a/tests/queries/0_stateless/00405_pretty_formats.sql b/tests/queries/0_stateless/00405_pretty_formats.sql index 00bb09a1c30..4715f4fc110 100644 --- a/tests/queries/0_stateless/00405_pretty_formats.sql +++ b/tests/queries/0_stateless/00405_pretty_formats.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_color = 1; SELECT number AS hello, toString(number) AS world, (hello, world) AS tuple, nullIf(hello % 3, 0) AS sometimes_nulls FROM system.numbers LIMIT 10 SETTINGS max_block_size = 5 FORMAT Pretty; diff --git a/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql b/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql index 4bace207fb5..59809841726 100644 --- a/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql +++ b/tests/queries/0_stateless/00476_pretty_formats_and_widths.sql @@ -1,4 +1,4 @@ -SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0; +SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0, output_format_pretty_display_footer_column_names=0; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact; SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace; diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.sql b/tests/queries/0_stateless/00515_enhanced_time_zones.sql index 837b0b4be20..e39b618b670 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.sql +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.sql @@ -1,3 +1,5 @@ +SET uniform_snowflake_conversion_functions = 0; + SELECT addMonths(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 1, 'Asia/Kolkata'); SELECT addMonths(toDateTime('2017-11-05 10:37:47', 'Asia/Kolkata'), 1); SELECT addMonths(toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata'), 1); diff --git a/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql b/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql index 511addb4e4d..ca423c1922a 100644 --- a/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql +++ b/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT s, parseDateTimeBestEffortOrNull(s, 'UTC') AS a, diff --git a/tests/queries/0_stateless/00576_nested_and_prewhere.sql b/tests/queries/0_stateless/00576_nested_and_prewhere.sql index 5916e679f1e..f5d9f0d6240 100644 --- a/tests/queries/0_stateless/00576_nested_and_prewhere.sql +++ b/tests/queries/0_stateless/00576_nested_and_prewhere.sql @@ -4,10 +4,10 @@ CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeT INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000; ALTER TABLE nested ADD COLUMN n.b Array(UInt64); -SELECT DISTINCT n.b FROM nested PREWHERE filter; +SELECT DISTINCT n.b FROM nested PREWHERE filter ORDER BY ALL; ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a); -SELECT DISTINCT n.c FROM nested PREWHERE filter; -SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter; +SELECT DISTINCT n.c FROM nested PREWHERE filter ORDER BY ALL; +SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter ORDER BY ALL; DROP TABLE nested; diff --git a/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh b/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh index 7ec4d99f028..f2b4cae8bb0 100755 --- a/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh +++ b/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh @@ -11,9 +11,11 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE single_col_partition_key(x UInt32) EN ${CLICKHOUSE_CLIENT} --query="INSERT INTO single_col_partition_key VALUES (1), (2), (3), (4), (11), (12), (20)" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x < 3 FORMAT XML" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x >= 11 FORMAT XML" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x = 20 FORMAT XML" | grep -F rows_read | sed 's/^[ \t]*//g' +DISABLE_COUNT_OPTIMIZATION="SETTINGS optimize_trivial_count_query = 0, optimize_use_implicit_projections = 0" + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x < 3 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x >= 11 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM single_col_partition_key WHERE x = 20 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' ${CLICKHOUSE_CLIENT} --query="DROP TABLE single_col_partition_key" @@ -31,14 +33,14 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO composite_partition_key VALUES \ ${CLICKHOUSE_CLIENT} --query="INSERT INTO composite_partition_key VALUES \ (301, 20, 3), (302, 21, 3), (303, 22, 3)" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a > 400 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b = 11 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE c = 4 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a > 400 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b = 11 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE c = 4 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a < 200 AND c = 2 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b < 20 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b >= 12 AND c = 2 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a < 200 AND c = 2 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b < 20 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE b >= 12 AND c = 2 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b = 21 AND c = 3 FORMAT XML SETTINGS optimize_trivial_count_query = 0" | grep -F rows_read | sed 's/^[ \t]*//g' +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM composite_partition_key WHERE a = 301 AND b = 21 AND c = 3 FORMAT XML $DISABLE_COUNT_OPTIMIZATION" | grep -F rows_read | sed 's/^[ \t]*//g' ${CLICKHOUSE_CLIENT} --query="DROP TABLE composite_partition_key" diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index e10b2f86145..7766d7720e1 100755 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -45,6 +45,7 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO enum_test_table VALUES ('hello'), ('wo ${CLICKHOUSE_CLIENT} --query="INSERT INTO date_test_table VALUES (1), (2), (2), (256), (257), (257);" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') +CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --optimize_use_implicit_projections 0" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" diff --git a/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql b/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql index 9a421361b71..24dd9336a88 100644 --- a/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql +++ b/tests/queries/0_stateless/00695_pretty_max_column_pad_width.sql @@ -1,2 +1,3 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_max_column_pad_width = 250; SELECT range(number) FROM system.numbers LIMIT 100 FORMAT PrettyCompactNoEscapes; diff --git a/tests/queries/0_stateless/00700_decimal_math.reference b/tests/queries/0_stateless/00700_decimal_math.reference index 389b428e27b..613fbc1ecd5 100644 --- a/tests/queries/0_stateless/00700_decimal_math.reference +++ b/tests/queries/0_stateless/00700_decimal_math.reference @@ -28,3 +28,6 @@ 0 0 1 0 3.14159265358979 0 -1 -0 1 1.5707963267948966 0 0.7853981633974483 +4.2 17.64 2.04939015319192 +4.2 17.64 2.04939015319192 +4.2 17.64 2.04939015319192 diff --git a/tests/queries/0_stateless/00700_decimal_math.sql b/tests/queries/0_stateless/00700_decimal_math.sql index 7f695a5b017..5dc8f800334 100644 --- a/tests/queries/0_stateless/00700_decimal_math.sql +++ b/tests/queries/0_stateless/00700_decimal_math.sql @@ -40,6 +40,6 @@ SELECT toDecimal128(pi(), 14) AS x, round(sin(x), 8), round(cos(x), 8), round(ta SELECT toDecimal128('1.0', 2) AS x, asin(x), acos(x), atan(x); -SELECT toDecimal32('4.2', 1) AS x, pow(x, 2), pow(x, 0.5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT toDecimal64('4.2', 1) AS x, pow(x, 2), pow(x, 0.5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT toDecimal128('4.2', 1) AS x, pow(x, 2), pow(x, 0.5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toDecimal32('4.2', 1) AS x, pow(x, 2), pow(x, 0.5); +SELECT toDecimal64('4.2', 1) AS x, pow(x, 2), pow(x, 0.5); +SELECT toDecimal128('4.2', 1) AS x, pow(x, 2), pow(x, 0.5); diff --git a/tests/queries/0_stateless/00732_base64_functions.sql b/tests/queries/0_stateless/00732_base64_functions.sql index 3c60bf939fe..b4be8db4ede 100644 --- a/tests/queries/0_stateless/00732_base64_functions.sql +++ b/tests/queries/0_stateless/00732_base64_functions.sql @@ -1,6 +1,5 @@ -- Tags: no-fasttest - -SET send_logs_level = 'fatal'; +-- no-fasttest because aklomp-base64 library is required SELECT base64Encode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT base64Decode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql b/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql index 1e3b24e60c0..7c2b75cdd70 100644 --- a/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql +++ b/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT s, parseDateTimeBestEffortOrNull(s, 'UTC') AS a, diff --git a/tests/queries/0_stateless/00825_protobuf_format_map.sh b/tests/queries/0_stateless/00825_protobuf_format_map.sh index 81d1cf2e305..82ebcc6959a 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_map.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_map.sh @@ -10,8 +10,6 @@ set -eo pipefail # Run the client. $CLICKHOUSE_CLIENT --multiquery < one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4)) +ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) +ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -63,12 +63,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2 +FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2 +FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -82,12 +82,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) +FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) +FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -100,12 +100,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) +ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) +ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 864dd69412a..4bd0eb7d908 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -49,14 +49,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -69,14 +69,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -89,14 +89,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -109,14 +109,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" echo "> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql index 54e46a2b718..db23ae1919c 100644 --- a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql @@ -22,7 +22,8 @@ DETACH DATABASE test_01676; ATTACH DATABASE test_01676; SELECT 'status_after_detach_and_attach:'; -SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict'; +-- It can be not loaded, or not even finish attaching in case of asynchronous tables loading. +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::Nullable(String), 'NOT_LOADED'); INSERT INTO test_01676.table (x) VALUES (toInt64(4)); SELECT * FROM test_01676.table ORDER BY x; diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.reference b/tests/queries/0_stateless/01683_text_log_deadlock.reference index 4cf61460252..3805f2a95e9 100644 --- a/tests/queries/0_stateless/01683_text_log_deadlock.reference +++ b/tests/queries/0_stateless/01683_text_log_deadlock.reference @@ -1 +1 @@ -queries: 25000 +queries: 5000 diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh index 1aced61cb42..6b3bcc58868 100755 --- a/tests/queries/0_stateless/01683_text_log_deadlock.sh +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -1,8 +1,8 @@ #!/usr/bin/env bash -# Tags: deadlock, no-tsan, no-asan +# Tags: deadlock CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_BENCHMARK --secure -i 25000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries: 25000' +$CLICKHOUSE_BENCHMARK --secure -i 5000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries: 5000' diff --git a/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh b/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh index 9284348dd62..d2dcd501428 100755 --- a/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh +++ b/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()"; -python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||: +python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 --max_parsing_threads=2 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||: ${CLICKHOUSE_CLIENT} -q "drop table insert_big_json" diff --git a/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference b/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference new file mode 100644 index 00000000000..43316772467 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference @@ -0,0 +1,3 @@ + ReadFromMergeTree (default.x) + ReadFromPreparedSource (Optimized trivial count) +5 diff --git a/tests/queries/0_stateless/01710_projection_pk_trivial_count.sql b/tests/queries/0_stateless/01710_projection_pk_trivial_count.sql new file mode 100644 index 00000000000..ce9eadf06b2 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_pk_trivial_count.sql @@ -0,0 +1,8 @@ +drop table if exists x; + +create table x (i int) engine MergeTree order by i settings index_granularity = 3; +insert into x select * from numbers(10); +select * from (explain select count() from x where (i >= 3 and i <= 6) or i = 7) where explain like '%ReadFromPreparedSource%' or explain like '%ReadFromMergeTree%'; +select count() from x where (i >= 3 and i <= 6) or i = 7; + +drop table x; diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql index 64edcae9edd..6bfcea84424 100644 --- a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql @@ -45,8 +45,6 @@ DROP TABLE test_tuple_nested_in_array_nested_in_tuple; SELECT 'Tuple nested in Map'; -SET allow_experimental_map_type = 1; - DROP TABLE IF EXISTS test_tuple_nested_in_map; CREATE TABLE test_tuple_nested_in_map (value Map(String, Tuple(UInt8, UInt8))) ENGINE=TinyLog; diff --git a/tests/queries/0_stateless/01720_type_map_and_casts.sql b/tests/queries/0_stateless/01720_type_map_and_casts.sql index 72d00da01f2..7bbe18512cb 100644 --- a/tests/queries/0_stateless/01720_type_map_and_casts.sql +++ b/tests/queries/0_stateless/01720_type_map_and_casts.sql @@ -1,5 +1,3 @@ -SET allow_experimental_map_type = 1; - DROP TABLE IF EXISTS table_map_with_key_integer; CREATE TABLE table_map_with_key_integer (d DATE, m Map(Int8, Int8)) diff --git a/tests/queries/0_stateless/01763_support_map_lowcardinality_type.sql b/tests/queries/0_stateless/01763_support_map_lowcardinality_type.sql index ccade153ca1..28361bbd76c 100644 --- a/tests/queries/0_stateless/01763_support_map_lowcardinality_type.sql +++ b/tests/queries/0_stateless/01763_support_map_lowcardinality_type.sql @@ -1,5 +1,4 @@ DROP TABLE IF EXISTS map_lc; -SET allow_experimental_map_type = 1; CREATE TABLE map_lc ( `kv` Map(LowCardinality(String), LowCardinality(String)) diff --git a/tests/queries/0_stateless/01803_const_nullable_map.sql b/tests/queries/0_stateless/01803_const_nullable_map.sql index 4ac9f925e24..6496dfb390c 100644 --- a/tests/queries/0_stateless/01803_const_nullable_map.sql +++ b/tests/queries/0_stateless/01803_const_nullable_map.sql @@ -1,7 +1,5 @@ DROP TABLE IF EXISTS t_map_null; -SET allow_experimental_map_type = 1; - CREATE TABLE t_map_null (a Map(String, String), b String) engine = MergeTree() ORDER BY a; INSERT INTO t_map_null VALUES (map('a', 'b', 'c', 'd'), 'foo'); SELECT count() FROM t_map_null WHERE a = map('name', NULL, '', NULL); diff --git a/tests/queries/0_stateless/01822_union_and_constans_error.reference b/tests/queries/0_stateless/01822_union_and_constans_error.reference index d00491fd7e5..e69de29bb2d 100644 --- a/tests/queries/0_stateless/01822_union_and_constans_error.reference +++ b/tests/queries/0_stateless/01822_union_and_constans_error.reference @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/01822_union_and_constans_error.sql b/tests/queries/0_stateless/01822_union_and_constans_error.sql index 38b7df700cd..9017e8769eb 100644 --- a/tests/queries/0_stateless/01822_union_and_constans_error.sql +++ b/tests/queries/0_stateless/01822_union_and_constans_error.sql @@ -15,6 +15,6 @@ SELECT isNull(t0.c0) OR COUNT('\n?pVa') FROM t0 GROUP BY t0.c0 HAVING isNull(isNull(t0.c0)) -SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0; +SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0 format Null; drop table if exists t0; diff --git a/tests/queries/0_stateless/01852_map_combinator.sql b/tests/queries/0_stateless/01852_map_combinator.sql index 546b040db53..09eadc7e7dc 100644 --- a/tests/queries/0_stateless/01852_map_combinator.sql +++ b/tests/queries/0_stateless/01852_map_combinator.sql @@ -1,5 +1,4 @@ SET send_logs_level = 'fatal'; -SET allow_experimental_map_type = 1; DROP TABLE IF EXISTS map_comb; CREATE TABLE map_comb(a int, statusMap Map(UInt16, UInt32)) ENGINE = Log; diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.sql b/tests/queries/0_stateless/01872_functions_to_subcolumns.sql index eb0165f4e13..45f83bf20e5 100644 --- a/tests/queries/0_stateless/01872_functions_to_subcolumns.sql +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.sql @@ -1,6 +1,5 @@ DROP TABLE IF EXISTS t_func_to_subcolumns; -SET allow_experimental_map_type = 1; SET optimize_functions_to_subcolumns = 1; CREATE TABLE t_func_to_subcolumns (id UInt64, arr Array(UInt64), n Nullable(String), m Map(String, UInt64)) diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.sh b/tests/queries/0_stateless/01889_sqlite_read_write.sh index fd0a1df20ac..30496af46f6 100755 --- a/tests/queries/0_stateless/01889_sqlite_read_write.sh +++ b/tests/queries/0_stateless/01889_sqlite_read_write.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel +# no-parallel: dealing with an SQLite database makes concurrent SHOW TABLES queries fail sporadically with the "database is locked" error. CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01905_to_json_string.sql b/tests/queries/0_stateless/01905_to_json_string.sql index 38c02ef13fb..42437de1b5f 100644 --- a/tests/queries/0_stateless/01905_to_json_string.sql +++ b/tests/queries/0_stateless/01905_to_json_string.sql @@ -15,5 +15,4 @@ $$, 10, 5, 3) limit 2; select * apply toJSONString from t; -set allow_experimental_map_type = 1; select toJSONString(map('1234', '5678')); diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 54c7ae59894..6406534a647 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -15,6 +15,6 @@ log = None with client(name="client1>", log=log) as client1: client1.expect(prompt) client1.send("SELECT number FROM numbers(1000) FORMAT Null") - client1.expect("Progress: 1\.00 thousand rows, 8\.00 KB .*" + end_of_block) - client1.expect("0 rows in set. Elapsed: [\\w]{1}\.[\\w]{3} sec.") + client1.expect("Progress: 1\\.00 thousand rows, 8\\.00 KB .*" + end_of_block) + client1.expect("0 rows in set. Elapsed: [\\w]{1}\\.[\\w]{3} sec.") client1.expect("Peak memory usage: .*B" + end_of_block) diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference index dd5738331c9..49eaf5520ed 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -1,6 +1,5 @@ -- { echo } drop table if exists map_test; -set allow_experimental_map_type = 1; create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); select mapPopulateSeries(m) from map_test; {1:1} diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql index f2b0dd60286..22c56e7e414 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql @@ -1,6 +1,5 @@ -- { echo } drop table if exists map_test; -set allow_experimental_map_type = 1; create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); select mapPopulateSeries(m) from map_test; diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 1090179bb67..0386717c933 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -1,3 +1,4 @@ +SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI) SET session_timezone = 'Africa/Juba'; -- Error cases @@ -10,6 +11,9 @@ SELECT dateTime64ToSnowflake('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT dateTimeToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT dateTime64ToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTimeToSnowflake(now()) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } +SELECT dateTime64ToSnowflake(now64()) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } + SELECT '-- const / non-const inputs'; WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference new file mode 100644 index 00000000000..5dcd0c9dfcd --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference @@ -0,0 +1,32 @@ +-- Negative tests +-- Return type +UInt64 +UInt64 +-- Standard and twitter epoch +Row 1: +────── +dt: 2021-08-15 18:57:56 +dt64: 2021-08-15 18:57:56.492 +dateTimeToSnowflakeID(dt): 6832747188322304000 +dateTime64ToSnowflakeID(dt64): 6832747190385901568 +dateTimeToSnowflakeID(dt, twitter_epoch): 1426981498778550272 +dateTime64ToSnowflakeID(dt64, twitter_epoch): 1426981500842147840 +-- Different DateTime64 scales +Row 1: +────── +dateTime64ToSnowflakeID(dt64_0): 6832747188322304000 +dateTime64ToSnowflakeID(dt64_1): 6832747190000025600 +dateTime64ToSnowflakeID(dt64_2): 6832747190377512960 +dateTime64ToSnowflakeID(dt64_3): 6832747190385901568 +dateTime64ToSnowflakeID(dt64_4): 6832747190385901568 +-- Idempotency +Row 1: +────── +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0), 0, 'UTC'), dt64_0): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1), 0, 'UTC'), dt64_1): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2), 0, 'UTC'), dt64_2): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC'), dt64_3): 1 +Row 1: +────── +dt64_4: 2023-11-11 11:11:11.1231 +snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)): 2023-11-11 11:11:11.123 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql new file mode 100644 index 00000000000..33bac8aaa35 --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -0,0 +1,78 @@ +SET session_timezone = 'UTC'; -- disable timezone randomization +SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces +SET uniform_snowflake_conversion_functions = 1; -- Force-enable uniform snowflake conversion functions (in case this is randomized in CI) + +SELECT '-- Negative tests'; +SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTime64ToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTimeToSnowflakeID('invalid_dt'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTime64ToSnowflakeID('invalid_dt'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTimeToSnowflakeID(now(), 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTime64ToSnowflakeID(now64(), 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTimeToSnowflakeID(now(), 42, 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTime64ToSnowflakeID(now64(), 42, 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT dateTimeToSnowflakeID(now()) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } +SELECT dateTime64ToSnowflakeID(now64()) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } + +SELECT '-- Return type'; +SELECT toTypeName(dateTimeToSnowflakeID(now())); +SELECT toTypeName(dateTime64ToSnowflakeID(now64())); + +SELECT '-- Standard and twitter epoch'; + +WITH + toDateTime('2021-08-15 18:57:56') AS dt, + toDateTime64('2021-08-15 18:57:56.492', 3) AS dt64, + 1288834974657 AS twitter_epoch +SELECT + dt, + dt64, + dateTimeToSnowflakeID(dt), + dateTime64ToSnowflakeID(dt64), + dateTimeToSnowflakeID(dt, twitter_epoch), + dateTime64ToSnowflakeID(dt64, twitter_epoch) +FORMAT + Vertical; + +SELECT '-- Different DateTime64 scales'; + +WITH + toDateTime64('2021-08-15 18:57:56.492', 0, 'UTC') AS dt64_0, + toDateTime64('2021-08-15 18:57:56.492', 1, 'UTC') AS dt64_1, + toDateTime64('2021-08-15 18:57:56.492', 2, 'UTC') AS dt64_2, + toDateTime64('2021-08-15 18:57:56.492', 3, 'UTC') AS dt64_3, + toDateTime64('2021-08-15 18:57:56.492', 4, 'UTC') AS dt64_4 +SELECT + dateTime64ToSnowflakeID(dt64_0), + dateTime64ToSnowflakeID(dt64_1), + dateTime64ToSnowflakeID(dt64_2), + dateTime64ToSnowflakeID(dt64_3), + dateTime64ToSnowflakeID(dt64_4) +Format + Vertical; + +SELECT '-- Idempotency'; + + -- DateTime64-to-SnowflakeID-to-DateTime64 is idempotent if the scale is <=3 (millisecond precision) +WITH + now64(0) AS dt64_0, + now64(1) AS dt64_1, + now64(2) AS dt64_2, + now64(3) AS dt64_3 +SELECT + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0), 0, 'UTC') == dt64_0, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1), 0, 'UTC') == dt64_1, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2), 0, 'UTC') == dt64_2, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC') == dt64_3 +FORMAT + Vertical; + +-- not idempotent +WITH + toDateTime64('2023-11-11 11:11:11.1231', 4, 'UTC') AS dt64_4 +SELECT + dt64_4, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)) +FORMAT + Vertical; diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference new file mode 100644 index 00000000000..9ed8c1dd3e5 --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference @@ -0,0 +1,27 @@ +-- Negative tests +-- Return type +DateTime +DateTime64(3) +-- Non-const path +Row 1: +────── +sf: 7204436857747984384 +dt: 2024-06-06 10:59:58 +dt64: 2024-06-06 10:59:58.851 +Row 1: +────── +sf: 1426981498778550272 +dt: 2021-08-15 18:57:56 +dt64: 2021-08-15 18:57:56.000 +Row 1: +────── +sf: 7204436857747984384 +dt: 2024-06-06 18:59:58 +dt64: 2024-06-06 18:59:58.851 +-- Const path +Row 1: +────── +sf: 7204436857747984384 +dt: 2024-06-06 10:59:58 +dt64: 2024-06-06 10:59:58.851 +-- Can be combined with generateSnowflakeID diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql new file mode 100644 index 00000000000..b0e244ef814 --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql @@ -0,0 +1,86 @@ +SET session_timezone = 'UTC'; -- disable timezone randomization +SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces +SET uniform_snowflake_conversion_functions = 1; -- Force-enable uniform snowflake conversion functions (in case this is randomized in CI) + +SELECT '-- Negative tests'; +SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeIDToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeIDToDateTime('invalid_snowflake'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime64('invalid_snowflake'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime(123::UInt64, 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime64(123::UInt64, 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime(123::UInt64, materialize(42)); -- {serverError ILLEGAL_COLUMN} +SELECT snowflakeIDToDateTime64(123::UInt64, materialize(42)); -- {serverError ILLEGAL_COLUMN} +SELECT snowflakeIDToDateTime(123::UInt64, 42, 42); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime64(123::UInt64, 42, 42); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime(123::UInt64, 42, 'UTC', 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeIDToDateTime64(123::UInt64, 42, 'UTC', 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT snowflakeIDToDateTime(123::UInt64) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } +SELECT snowflakeIDToDateTime64(123::UInt64) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } + +SELECT '-- Return type'; +SELECT toTypeName(snowflakeIDToDateTime(123::UInt64)); +SELECT toTypeName(snowflakeIDToDateTime64(123::UInt64)); + +SELECT '-- Non-const path'; +-- Two const arguments are mapped to two non-const arguments ('getDefaultImplementationForConstants'), the non-const path is taken + +WITH + 7204436857747984384 AS sf +SELECT + sf, + snowflakeIDToDateTime(sf) as dt, + snowflakeIDToDateTime64(sf) as dt64 +FORMAT + Vertical; + +-- With Twitter Snowflake ID and Twitter epoch +WITH + 1426981498778550272 AS sf, + 1288834974657 AS epoch +SELECT + sf, + snowflakeIDToDateTime(sf, epoch) as dt, + snowflakeIDToDateTime64(sf, epoch) as dt64 +FORMAT + Vertical; + +-- non-default timezone +WITH + 7204436857747984384 AS sf, + 0 AS epoch, -- default epoch + 'Asia/Shanghai' AS tz +SELECT + sf, + snowflakeIDToDateTime(sf, epoch, tz) as dt, + snowflakeIDToDateTime64(sf, epoch, tz) as dt64 +FORMAT + Vertical; + +SELECT '-- Const path'; + +-- The const path can only be tested by const snowflake + const epoch + non-const time-zone. The latter requires a special setting. +WITH + 7204436857747984384 AS sf, + 0 AS epoch, -- default epoch + materialize('Asia/Shanghai') AS tz +SELECT + sf, + snowflakeIDToDateTime(sf, epoch, tz) as dt, + snowflakeIDToDateTime64(sf, epoch, tz) as dt64 +FORMAT + Vertical +SETTINGS + allow_nonconst_timezone_arguments = 1; + + +SELECT '-- Can be combined with generateSnowflakeID'; + +WITH + generateSnowflakeID() AS snowflake +SELECT + snowflakeIDToDateTime(snowflake), + snowflakeIDToDateTime64(snowflake) +FORMAT + Null; diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index f1a50dd370d..1729a50ae44 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -1,3 +1,5 @@ +SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI) + -- -- Error cases SELECT snowflakeToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT snowflakeToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} @@ -8,6 +10,9 @@ SELECT snowflakeToDateTime64('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT snowflakeToDateTime('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT snowflakeToDateTime64('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeToDateTime(123::Int64) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } +SELECT snowflakeToDateTime64(123::Int64) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } + SELECT 'const column'; WITH CAST(1426860704886947840 AS Int64) AS i64, diff --git a/tests/queries/0_stateless/02002_parse_map_int_key.sql b/tests/queries/0_stateless/02002_parse_map_int_key.sql index ecd2a090975..8d009037dd7 100644 --- a/tests/queries/0_stateless/02002_parse_map_int_key.sql +++ b/tests/queries/0_stateless/02002_parse_map_int_key.sql @@ -1,5 +1,3 @@ -SET allow_experimental_map_type = 1; - DROP TABLE IF EXISTS t_map_int_key; CREATE TABLE t_map_int_key (m1 Map(UInt32, UInt32), m2 Map(Date, UInt32)) ENGINE = Memory; diff --git a/tests/queries/0_stateless/02011_tuple_vector_functions.sql b/tests/queries/0_stateless/02011_tuple_vector_functions.sql index d0cd89dc464..9575001bd49 100644 --- a/tests/queries/0_stateless/02011_tuple_vector_functions.sql +++ b/tests/queries/0_stateless/02011_tuple_vector_functions.sql @@ -78,11 +78,12 @@ SELECT max2(NULL, 1) - min2(NULL, 1); SELECT L1Norm(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT (1, 1) / toString(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT -(1, toString(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT LpNorm((1, 2), toDecimal32(2, 4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT LpNorm((1, 2), toDecimal32(2, 4)); -- { serverError ILLEGAL_COLUMN } SELECT (1, 2) * toDecimal32(3.1, 8); SELECT cosineDistance((1, 2), (2, 3, 4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT tuple() + tuple(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- TODO: what's the expected value of () + ()? Currently it returns 0. +-- SELECT tuple() + tuple(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT LpNorm((1, 2, 3)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT max2(1, 2, -1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference index 1f8f2bf0d0c..3cedda81a08 100644 --- a/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference +++ b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.reference @@ -9,6 +9,7 @@ 1 1 1 +1 0 {'aa':NULL} {} diff --git a/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql index b04c5945a08..9ebf06c1e46 100644 --- a/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql +++ b/tests/queries/0_stateless/02030_function_mapContainsKeyLike.sql @@ -12,6 +12,7 @@ SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '3-%' DROP TABLE map_containsKeyLike_test; SELECT mapContainsKeyLike(map('aa', 1, 'bb', 2), 'a%'); +SELECT mapContainsKeyLike(map(toLowCardinality('aa'), 1, toLowCardinality('b'), 2), 'a%'); SELECT mapContainsKeyLike(map('aa', 1, 'bb', 2), materialize('a%')); SELECT mapContainsKeyLike(materialize(map('aa', 1, 'bb', 2)), 'a%'); SELECT mapContainsKeyLike(materialize(map('aa', 1, 'bb', 2)), materialize('a%')); 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 d625feb94d3..d15c1545135 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -4,7 +4,7 @@ CREATE TABLE system.aggregate_function_combinators `is_internal` UInt8 ) ENGINE = SystemAggregateFunctionCombinators -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available aggregate function combinators, which could be applied to aggregate functions and change the way they work.' CREATE TABLE system.asynchronous_inserts ( `query` String, @@ -17,7 +17,7 @@ CREATE TABLE system.asynchronous_inserts `entries.bytes` Array(UInt64) ) ENGINE = SystemAsynchronousInserts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about pending asynchronous inserts in queue in server\'s memory.' CREATE TABLE system.asynchronous_metrics ( `metric` String, @@ -25,14 +25,14 @@ CREATE TABLE system.asynchronous_metrics `description` String ) ENGINE = SystemAsynchronousMetrics -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.' CREATE TABLE system.build_options ( `name` String, `value` String ) ENGINE = SystemBuildOptions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all build flags, compiler options and commit hash for used build.' CREATE TABLE system.clusters ( `cluster` String, @@ -55,14 +55,14 @@ CREATE TABLE system.clusters `name` String ALIAS cluster ) ENGINE = SystemClusters -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about clusters defined in the configuration file or generated by a Replicated database.' CREATE TABLE system.collations ( `name` String, `language` Nullable(String) ) ENGINE = SystemTableCollations -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available collations for alphabetical comparison of strings.' CREATE TABLE system.columns ( `database` String, @@ -88,13 +88,13 @@ CREATE TABLE system.columns `datetime_precision` Nullable(UInt64) ) ENGINE = SystemColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all columns from all tables of the current server.' CREATE TABLE system.contributors ( `name` String ) ENGINE = SystemContributors -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all ClickHouse contributors <3' CREATE TABLE system.current_roles ( `role_name` String, @@ -102,7 +102,7 @@ CREATE TABLE system.current_roles `is_default` UInt8 ) ENGINE = SystemCurrentRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains active roles of a current user. SET ROLE changes the contents of this table.' CREATE TABLE system.data_skipping_indices ( `database` String, @@ -117,7 +117,7 @@ CREATE TABLE system.data_skipping_indices `marks` UInt64 ) ENGINE = SystemDataSkippingIndices -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains all the information about all the data skipping indices in tables, similar to system.columns.' CREATE TABLE system.data_type_families ( `name` String, @@ -125,7 +125,7 @@ CREATE TABLE system.data_type_families `alias_to` String ) ENGINE = SystemTableDataTypeFamilies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available native data types along with all the aliases used for compatibility with other DBMS.' CREATE TABLE system.databases ( `name` String, @@ -138,7 +138,7 @@ CREATE TABLE system.databases `database` String ALIAS name ) ENGINE = SystemDatabases -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all databases of the current server.' CREATE TABLE system.detached_parts ( `database` String, @@ -155,7 +155,7 @@ CREATE TABLE system.detached_parts `level` Nullable(UInt32) ) ENGINE = SystemDetachedParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all parts which are being found in /detached directory along with a reason why it was detached. ClickHouse server doesn\'t use such parts anyhow.' CREATE TABLE system.dictionaries ( `database` String, @@ -185,7 +185,7 @@ CREATE TABLE system.dictionaries `comment` String ) ENGINE = SystemDictionaries -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about dictionaries.' CREATE TABLE system.disks ( `name` String, @@ -205,7 +205,7 @@ CREATE TABLE system.disks `cache_path` String ) ENGINE = SystemDisks -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about disks defined in the server configuration.' CREATE TABLE system.distributed_ddl_queue ( `entry` String, @@ -225,7 +225,7 @@ CREATE TABLE system.distributed_ddl_queue `query_duration_ms` Nullable(UInt64) ) ENGINE = SystemDDLWorkerQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about distributed DDL queries (ON CLUSTER clause) that were executed on a cluster.' CREATE TABLE system.distribution_queue ( `database` String, @@ -241,7 +241,7 @@ CREATE TABLE system.distribution_queue `last_exception_time` DateTime ) ENGINE = SystemDistributionQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about local files that are in the queue to be sent to the shards. These local files contain new parts that are created by inserting new data into the Distributed table in asynchronous mode.' CREATE TABLE system.enabled_roles ( `role_name` String, @@ -250,7 +250,7 @@ CREATE TABLE system.enabled_roles `is_default` UInt8 ) ENGINE = SystemEnabledRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains all active roles at the moment, including current role of the current user and granted roles for current role.' CREATE TABLE system.errors ( `name` String, @@ -262,7 +262,7 @@ CREATE TABLE system.errors `remote` UInt8 ) ENGINE = SystemErrors -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace.' CREATE TABLE system.events ( `event` String, @@ -271,7 +271,7 @@ CREATE TABLE system.events `name` String ALIAS event ) ENGINE = SystemEvents -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains profiling events and their current value.' CREATE TABLE system.formats ( `name` String, @@ -281,7 +281,7 @@ CREATE TABLE system.formats `supports_parallel_formatting` UInt8 ) ENGINE = SystemFormats -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all the formats along with flags whether a format is suitable for input/output or whether it supports parallelization.' CREATE TABLE system.functions ( `name` String, @@ -299,7 +299,7 @@ CREATE TABLE system.functions `categories` String ) ENGINE = SystemFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available ordinary and aggregate functions with their descriptions.' CREATE TABLE system.graphite_retentions ( `config_name` String, @@ -314,7 +314,7 @@ CREATE TABLE system.graphite_retentions `Tables.table` Array(String) ) ENGINE = SystemGraphite -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about parameters graphite_rollup which are used in tables with *GraphiteMergeTree engines.' CREATE TABLE system.licenses ( `library_name` String, @@ -323,14 +323,14 @@ CREATE TABLE system.licenses `license_text` String ) ENGINE = SystemLicenses -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources.' CREATE TABLE system.macros ( `macro` String, `substitution` String ) ENGINE = SystemMacros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all macros defined in server configuration.' CREATE TABLE system.merge_tree_settings ( `name` String, @@ -344,7 +344,7 @@ CREATE TABLE system.merge_tree_settings `is_obsolete` UInt8 ) ENGINE = SystemMergeTreeSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all MergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query.' CREATE TABLE system.merges ( `database` String, @@ -373,7 +373,7 @@ CREATE TABLE system.merges `merge_algorithm` String ) ENGINE = SystemMerges -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row.' CREATE TABLE system.metrics ( `metric` String, @@ -382,7 +382,7 @@ CREATE TABLE system.metrics `name` String ALIAS metric ) ENGINE = SystemMetrics -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date.' CREATE TABLE system.moves ( `database` String, @@ -395,7 +395,7 @@ CREATE TABLE system.moves `thread_id` UInt64 ) ENGINE = SystemMoves -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row.' CREATE TABLE system.mutations ( `database` String, @@ -414,25 +414,25 @@ CREATE TABLE system.mutations `latest_fail_reason` String ) ENGINE = SystemMutations -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of mutations and their progress. Each mutation command is represented by a single row.' CREATE TABLE system.numbers ( `number` UInt64 ) ENGINE = SystemNumbers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.' CREATE TABLE system.numbers_mt ( `number` UInt64 ) ENGINE = SystemNumbers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Multithreaded version of `system.numbers`. Numbers order is not guaranteed.' CREATE TABLE system.one ( `dummy` UInt8 ) ENGINE = SystemOne -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.' CREATE TABLE system.part_moves_between_shards ( `database` String, @@ -451,7 +451,7 @@ CREATE TABLE system.part_moves_between_shards `last_exception` String ) ENGINE = SystemShardMoves -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about parts which are currently in a process of moving between shards and their progress.' CREATE TABLE system.parts ( `partition` String, @@ -522,7 +522,7 @@ CREATE TABLE system.parts `part_name` String ALIAS name ) ENGINE = SystemParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently existing (both active and inactive) parts of all *-MergeTree tables. Each part is represented by a single row.' CREATE TABLE system.parts_columns ( `partition` String, @@ -582,7 +582,7 @@ CREATE TABLE system.parts_columns `part_name` String ALIAS name ) ENGINE = SystemPartsColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of columns of all currently existing parts of all MergeTree tables. Each column is represented by a single row.' CREATE TABLE system.processes ( `is_initial_query` UInt8, @@ -630,7 +630,7 @@ CREATE TABLE system.processes `Settings.Values` Array(String) ALIAS mapValues(Settings) ) ENGINE = SystemProcesses -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently executing processes (queries) with their progress.' CREATE TABLE system.projection_parts ( `partition` String, @@ -698,7 +698,7 @@ CREATE TABLE system.projection_parts `part_name` String ALIAS name ) ENGINE = SystemProjectionParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently existing projection parts (a copy of some part containing aggregated data or just sorted in different order) created for all the projections for all tables within a cluster.' CREATE TABLE system.projection_parts_columns ( `partition` String, @@ -754,7 +754,7 @@ CREATE TABLE system.projection_parts_columns `part_name` String ALIAS name ) ENGINE = SystemProjectionPartsColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of columns of all currently existing projection parts of all MergeTree tables. Each column is represented by a single row.' CREATE TABLE system.quota_limits ( `quota_name` String, @@ -773,7 +773,7 @@ CREATE TABLE system.quota_limits `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotaLimits -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about maximums for all intervals of all quotas. Any number of rows or zero can correspond to specific quota.' CREATE TABLE system.quota_usage ( `quota_name` String, @@ -805,7 +805,7 @@ CREATE TABLE system.quota_usage `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotaUsage -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains quota usage by the current user: how much is used and how much is left.' CREATE TABLE system.quotas ( `name` String, @@ -818,7 +818,7 @@ CREATE TABLE system.quotas `apply_to_except` Array(String) ) ENGINE = SystemQuotas -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about quotas.' CREATE TABLE system.quotas_usage ( `quota_name` String, @@ -851,7 +851,7 @@ CREATE TABLE system.quotas_usage `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotasUsage -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains quota usage by all users.' CREATE TABLE system.replicas ( `database` String, @@ -891,7 +891,7 @@ CREATE TABLE system.replicas `replica_is_active` Map(String, UInt8) ) ENGINE = SystemReplicas -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information and status of all table replicas on current server. Each replica is represented by a single row.' CREATE TABLE system.replicated_fetches ( `database` String, @@ -912,7 +912,7 @@ CREATE TABLE system.replicated_fetches `thread_id` UInt64 ) ENGINE = SystemReplicatedFetches -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about currently running background fetches.' CREATE TABLE system.replicated_merge_tree_settings ( `name` String, @@ -926,7 +926,7 @@ CREATE TABLE system.replicated_merge_tree_settings `is_obsolete` UInt8 ) ENGINE = SystemReplicatedMergeTreeSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all ReplicatedMergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query. ' CREATE TABLE system.replication_queue ( `database` String, @@ -952,7 +952,7 @@ CREATE TABLE system.replication_queue `merge_type` String ) ENGINE = SystemReplicationQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica.' CREATE TABLE system.role_grants ( `user_name` Nullable(String), @@ -963,7 +963,7 @@ CREATE TABLE system.role_grants `with_admin_option` UInt8 ) ENGINE = SystemRoleGrants -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains the role grants for users and roles. To add entries to this table, use `GRANT role TO user`. Using this table you may find out which roles are assigned to which users or which roles a user has.' CREATE TABLE system.roles ( `name` String, @@ -971,7 +971,7 @@ CREATE TABLE system.roles `storage` String ) ENGINE = SystemRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all roles created at the server.' CREATE TABLE system.row_policies ( `name` String, @@ -987,7 +987,7 @@ CREATE TABLE system.row_policies `apply_to_except` Array(String) ) ENGINE = SystemRowPolicies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains filters for one particular table, as well as a list of roles and/or users which should use this row policy.' CREATE TABLE system.settings ( `name` String, @@ -1003,7 +1003,7 @@ CREATE TABLE system.settings `is_obsolete` UInt8 ) ENGINE = SystemSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions.' CREATE TABLE system.settings_profile_elements ( `profile_name` Nullable(String), @@ -1018,7 +1018,7 @@ CREATE TABLE system.settings_profile_elements `inherit_profile` Nullable(String) ) ENGINE = SystemSettingsProfileElements -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Describes the content of each settings profile configured on the server. Including settings constraints, roles and users for which the settings are applied, and parent settings profiles.' CREATE TABLE system.settings_profiles ( `name` String, @@ -1030,7 +1030,7 @@ CREATE TABLE system.settings_profiles `apply_to_except` Array(String) ) ENGINE = SystemSettingsProfiles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains properties of configured setting profiles.' CREATE TABLE system.stack_trace ( `thread_name` String, @@ -1039,7 +1039,7 @@ CREATE TABLE system.stack_trace `trace` Array(UInt64) ) ENGINE = SystemStackTrace -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Allows to obtain an unsymbolized stacktrace from all the threads of the server process.' CREATE TABLE system.storage_policies ( `policy_name` String, @@ -1054,7 +1054,7 @@ CREATE TABLE system.storage_policies `load_balancing` Enum8('ROUND_ROBIN' = 0, 'LEAST_USED' = 1) ) ENGINE = SystemStoragePolicies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about storage policies and volumes defined in the server configuration.' CREATE TABLE system.table_engines ( `name` String, @@ -1068,7 +1068,7 @@ CREATE TABLE system.table_engines `supports_parallel_insert` UInt8 ) ENGINE = SystemTableEngines -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available table engines along with information whether a particular table engine supports some specific features (e.g. settings, skipping indices, projections, replication, TTL, deduplication, parallel insert, etc.)' CREATE TABLE system.table_functions ( `name` String, @@ -1076,7 +1076,7 @@ CREATE TABLE system.table_functions `allow_readonly` UInt8 ) ENGINE = SystemTableFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available table functions with their descriptions.' CREATE TABLE system.tables ( `database` String, @@ -1115,13 +1115,13 @@ CREATE TABLE system.tables `table` String ALIAS name ) ENGINE = SystemTables -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all tables of the current server.' CREATE TABLE system.time_zones ( `time_zone` String ) ENGINE = SystemTimeZones -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse.' CREATE TABLE system.user_directories ( `name` String, @@ -1130,7 +1130,7 @@ CREATE TABLE system.user_directories `precedence` UInt64 ) ENGINE = SystemUserDirectories -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains the information about configured user directories - directories on the file system from which ClickHouse server is allowed to read user provided data.' CREATE TABLE system.users ( `name` String, @@ -1151,22 +1151,22 @@ CREATE TABLE system.users `default_database` String ) ENGINE = SystemUsers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all users profiles either configured at the server through a configuration file or created via SQL.' CREATE TABLE system.warnings ( `message` String ) ENGINE = SystemWarnings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server.' CREATE TABLE system.zeros ( `zero` UInt8 ) ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Produces unlimited number of non-materialized zeros.' CREATE TABLE system.zeros_mt ( `zero` UInt8 ) ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Multithreaded version of system.zeros.' diff --git a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference index 848abb332bb..e1e1ab3f441 100644 --- a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference +++ b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference @@ -1 +1 @@ -CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'Contains a list of metrics exposed from embedded RocksDB.\' diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 86a36a9392c..8a18c609ede 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,6 +1,3 @@ - Prewhere info - Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) @@ -8,8 +5,15 @@ Prewhere filter Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) 2 - Filter column: and(equals(k, 3), notEmpty(v)) (removed) + Filter column: and(equals(k, 3), notEmpty(v)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) 2 - Filter column: and(equals(k, 3), notEmpty(v)) (removed) - Filter column: and(equals(k, 3), notEmpty(v)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) 2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference new file mode 100644 index 00000000000..20c58c33770 --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.reference @@ -0,0 +1 @@ +59900 1000 1396 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql new file mode 100644 index 00000000000..fc18c97cb6e --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_not_ready_set_bug.sql @@ -0,0 +1,7 @@ +create table merge_kek_1 (x UInt32, y UInt32) engine = MergeTree order by x; +create table merge_kek_2 (x UInt32, y UInt32) engine = MergeTree order by x; + +insert into merge_kek_1 select number, number from numbers(100); +insert into merge_kek_2 select number + 500, number + 500 from numbers(1e6); + +select sum(x), min(x + x), max(x + x) from merge(currentDatabase(), '^merge_kek_.$') where x > 200 and y in (select 500 + number * 2 from numbers(100)) settings max_threads=2; diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 6100de6abaf..869ffd2ff29 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -36,6 +36,11 @@ 1 1 1 +0 +0 +0 +0 +0 {0.1:4,0.2:5} {0.1:4,0.2:5} {[1,2]:4,[3,4]:5} @@ -76,6 +81,20 @@ 0 0 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 {'key2':101,'key3':100,'key4':102} {'key2':102,'key3':101,'key4':103} {'key2':103,'key3':102,'key4':104} @@ -102,6 +121,8 @@ {1:2,2:3} {'x':'y','x':'y'} {'x':'y','x':'y'} +{1:2,2:3} +{1:2,2:3} {'k1':11,'k2':22} {'k1':11,'k2':22} {'k1':11,'k2':22} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 854ce214924..42de17f9b85 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -21,6 +21,12 @@ SELECT mapExists((k, v) -> k = array(1,2), map(array(1,2), 4, array(3,4), 5)); SELECT mapExists((k, v) -> k = map(1,2), map(map(1,2), 4, map(3,4), 5)); SELECT mapExists((k, v) -> k = tuple(1,2), map(tuple(1,2), 4, tuple(3,4), 5)); +SELECT mapAll((k, v) -> k = 0.1::Float32, map(0.1::Float32, 4, 0.2::Float32, 5)); +SELECT mapAll((k, v) -> k = 0.1::Float64, map(0.1::Float64, 4, 0.2::Float64, 5)); +SELECT mapAll((k, v) -> k = array(1,2), map(array(1,2), 4, array(3,4), 5)); +SELECT mapAll((k, v) -> k = map(1,2), map(map(1,2), 4, map(3,4), 5)); +SELECT mapAll((k, v) -> k = tuple(1,2), map(tuple(1,2), 4, tuple(3,4), 5)); + SELECT mapSort((k, v) -> k, map(0.1::Float32, 4, 0.2::Float32, 5)); SELECT mapSort((k, v) -> k, map(0.1::Float64, 4, 0.2::Float64, 5)); SELECT mapSort((k, v) -> k, map(array(1,2), 4, array(3,4), 5)); @@ -41,6 +47,9 @@ SELECT mapConcat(map(tuple(1,2), 4), map(tuple(3,4), 5)); SELECT mapExists((k, v) -> k LIKE '%3', col) FROM table_map ORDER BY id; SELECT mapExists((k, v) -> k LIKE '%2' AND v < 1000, col) FROM table_map ORDER BY id; +SELECT mapAll((k, v) -> k LIKE '%3', col) FROM table_map ORDER BY id; +SELECT mapAll((k, v) -> k LIKE '%2' AND v < 1000, col) FROM table_map ORDER BY id; + SELECT mapSort(col) FROM table_map ORDER BY id; SELECT mapSort((k, v) -> v, col) FROM table_map ORDER BY id; SELECT mapPartialSort((k, v) -> k, 2, col) FROM table_map ORDER BY id; @@ -50,6 +59,8 @@ SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1, 0, 2, 0))); SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0))); +SELECT mapApply((x, y) -> (x, x + 1), map(1.0, 0, 2.0, 0)); +SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1.0, 0, 2.0, 0))); SELECT mapUpdate(map('k1', 1, 'k2', 2), map('k1', 11, 'k2', 22)); SELECT mapUpdate(materialize(map('k1', 1, 'k2', 2)), map('k1', 11, 'k2', 22)); diff --git a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh index 39ba17fc7eb..37fdde95ea7 100755 --- a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh +++ b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel +# no-parallel: dealing with an SQLite database makes concurrent SHOW TABLES queries fail sporadically with the "database is locked" error. CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 26a201ec89f..ca1ee2738c7 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,4 +1,5 @@ --- Tags: long, no-parallel +-- Tags: long, no-parallel, no-s3-storage +-- no-s3-storage: Avoid flakiness due to cache / buffer usage SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 100 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference index 8ad0a566c62..1c60e40942c 100644 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference @@ -97,8 +97,8 @@ c1 Array(Nullable(Bool)) [] [NULL] [false] -c1 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) -(1,2,3) +c1 Nullable(String) +(1, 2, 3) c1 Nullable(String) 123.123 c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) diff --git a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql index b2a04788bbb..f8faa3e653b 100644 --- a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql +++ b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql @@ -68,4 +68,10 @@ with last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc ) select d.repo_name, columns('count') from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name; +set allow_suspicious_low_cardinality_types=1; + +CREATE TABLE github_events__fuzz_0 (`file_time` Int64, `event_type` Enum8('CommitCommentEvent' = 1, 'CreateEvent' = 2, 'DeleteEvent' = 3, 'ForkEvent' = 4, 'GollumEvent' = 5, 'IssueCommentEvent' = 6, 'IssuesEvent' = 7, 'MemberEvent' = 8, 'PublicEvent' = 9, 'PullRequestEvent' = 10, 'PullRequestReviewCommentEvent' = 11, 'PushEvent' = 12, 'ReleaseEvent' = 13, 'SponsorshipEvent' = 14, 'WatchEvent' = 15, 'GistEvent' = 16, 'FollowEvent' = 17, 'DownloadEvent' = 18, 'PullRequestReviewEvent' = 19, 'ForkApplyEvent' = 20, 'Event' = 21, 'TeamAddEvent' = 22), `actor_login` LowCardinality(String), `repo_name` LowCardinality(Nullable(String)), `created_at` DateTime, `updated_at` DateTime, `action` Array(Enum8('none' = 0, 'created' = 1, 'added' = 2, 'edited' = 3, 'deleted' = 4, 'opened' = 5, 'closed' = 6, 'reopened' = 7, 'assigned' = 8, 'unassigned' = 9, 'labeled' = 10, 'unlabeled' = 11, 'review_requested' = 12, 'review_request_removed' = 13, 'synchronize' = 14, 'started' = 15, 'published' = 16, 'update' = 17, 'create' = 18, 'fork' = 19, 'merged' = 20)), `comment_id` UInt64, `body` String, `path` LowCardinality(String), `position` Int32, `line` Int32, `ref` String, `ref_type` Enum8('none' = 0, 'branch' = 1, 'tag' = 2, 'repository' = 3, 'unknown' = 4), `creator_user_login` Int16, `number` UInt32, `title` String, `labels` Array(Array(LowCardinality(String))), `state` Enum8('none' = 0, 'open' = 1, 'closed' = 2), `locked` UInt8, `assignee` Array(LowCardinality(String)), `assignees` Array(LowCardinality(String)), `comments` UInt32, `author_association` Array(Enum8('NONE' = 0, 'CONTRIBUTOR' = 1, 'OWNER' = 2, 'COLLABORATOR' = 3, 'MEMBER' = 4, 'MANNEQUIN' = 5)), `closed_at` UUID, `merged_at` DateTime, `merge_commit_sha` Nullable(String), `requested_reviewers` Array(LowCardinality(Int64)), `requested_teams` Array(String), `head_ref` String, `head_sha` String, `base_ref` String, `base_sha` String, `merged` Nullable(UInt8), `mergeable` Nullable(UInt8), `rebaseable` LowCardinality(UInt8), `mergeable_state` Array(Enum8('unknown' = 0, 'dirty' = 1, 'clean' = 2, 'unstable' = 3, 'draft' = 4)), `merged_by` LowCardinality(String), `review_comments` UInt32, `maintainer_can_modify` Nullable(UInt8), `commits` UInt32, `additions` Nullable(UInt32), `deletions` UInt32, `changed_files` UInt32, `diff_hunk` Nullable(String), `original_position` UInt32, `commit_id` String, `original_commit_id` String, `push_size` UInt32, `push_distinct_size` UInt32, `member_login` LowCardinality(String), `release_tag_name` LowCardinality(String), `release_name` String, `review_state` Int16) ENGINE = MergeTree ORDER BY (event_type, repo_name, created_at) settings allow_nullable_key=1; + +EXPLAIN PIPELINE header = true, compact = true WITH top_repos AS (SELECT repo_name FROM github_events__fuzz_0 WHERE (event_type = 'WatchEvent') AND (toDate(created_at) = (today() - 1)) GROUP BY repo_name ORDER BY count() DESC LIMIT 100 UNION DISTINCT SELECT repo_name FROM github_events__fuzz_0 WHERE (event_type = 'WatchEvent') AND (toMonday(created_at) = toMonday(today() - toIntervalWeek(1))) GROUP BY repo_name ORDER BY count() DESC LIMIT 100 UNION DISTINCT SELECT repo_name FROM github_events__fuzz_0 PREWHERE (event_type = 'WatchEvent') AND (toStartOfMonth(created_at) = (toStartOfMonth(today()) - toIntervalMonth(1))) GROUP BY repo_name ORDER BY count() DESC LIMIT 100 UNION DISTINCT SELECT repo_name FROM github_events WHERE (event_type = 'WatchEvent') AND (toYear(created_at) = (toYear(today()) - 1)) GROUP BY repo_name ORDER BY count() DESC LIMIT 100), last_day AS (SELECT repo_name, count() AS count_last_day, rowNumberInAllBlocks() + 1 AS position_last_day FROM github_events WHERE (repo_name IN (SELECT repo_name FROM top_repos)) AND (toDate(created_at) = (today() - 1)) GROUP BY repo_name ORDER BY count_last_day DESC), last_week AS (SELECT repo_name, count() AS count_last_week, rowNumberInAllBlocks() + 1 AS position_last_week FROM github_events WHERE (repo_name IN (SELECT repo_name FROM top_repos)) AND (toMonday(created_at) = (toMonday(today()) - toIntervalWeek(2))) GROUP BY repo_name ORDER BY count_last_week DESC), last_month AS (SELECT repo_name, count() AS count_last_month, rowNumberInAllBlocks() + 1 AS position_last_month FROM github_events__fuzz_0 WHERE ('deleted' = 4) AND in(repo_name) AND (toStartOfMonth(created_at) = (toStartOfMonth(today()) - toIntervalMonth(1))) GROUP BY repo_name ORDER BY count_last_month DESC) SELECT d.repo_name, COLUMNS(count) FROM last_day AS d INNER JOIN last_week AS w ON d.repo_name = w.repo_name INNER JOIN last_month AS m ON d.repo_name = m.repo_name format Null; -- { serverError INVALID_SETTING_VALUE } + DROP TABLE github_events; diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 85e8a802bdc..9b9885478cd 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -87,7 +87,7 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC -Sorting (Stream): __table1.a ASC, b ASC +Sorting (Stream): a ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): __table1.a ASC Sorting (Stream): __table1.a ASC diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index caa600298ce..b100f96befa 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -24,6 +24,7 @@ function check_refcnt_for_table() local log_file log_file=$(mktemp "$CUR_DIR/clickhouse-tests.XXXXXX.log") local args=( + --allow_repeated_settings --format Null --max_threads 1 --max_block_size 1 diff --git a/tests/queries/0_stateless/02346_fulltext_index_bug47393.sql b/tests/queries/0_stateless/02346_fulltext_index_bug47393.sql index c9743dce3aa..49711fcd56b 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_bug47393.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_bug47393.sql @@ -1,4 +1,4 @@ -SET allow_experimental_inverted_index = 1; +SET allow_experimental_full_text_index = 1; DROP TABLE IF EXISTS tab; CREATE TABLE tab diff --git a/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql b/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql index e9b8088dcc1..a643df65095 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_bug52019.sql @@ -1,6 +1,6 @@ -- Test for Bug 52019: Undefined behavior -SET allow_experimental_inverted_index=1; +SET allow_experimental_full_text_index=1; DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02346_fulltext_index_bug59039.sql b/tests/queries/0_stateless/02346_fulltext_index_bug59039.sql index e94d25d8d9e..0910939a366 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_bug59039.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_bug59039.sql @@ -1,7 +1,7 @@ -- This is supposed to test that DROP INDEX removes all index related files. Can't test this directly but at least run the statement and -- check that no bad things happen. -SET allow_experimental_inverted_index = 1; +SET allow_experimental_full_text_index = 1; DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02346_fulltext_index_detach_attach.sql b/tests/queries/0_stateless/02346_fulltext_index_detach_attach.sql index 20d7c5f5477..aa0f94e4de8 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_detach_attach.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_detach_attach.sql @@ -1,4 +1,4 @@ -SET allow_experimental_inverted_index = 1; +SET allow_experimental_full_text_index = 1; CREATE TABLE t ( diff --git a/tests/queries/0_stateless/02346_fulltext_index_match_predicate.reference b/tests/queries/0_stateless/02346_fulltext_index_match_predicate.reference index 84fc422379c..e890eac1794 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_match_predicate.reference +++ b/tests/queries/0_stateless/02346_fulltext_index_match_predicate.reference @@ -1,19 +1,19 @@ -1 Hello ClickHouse -2 Hello World +1 Well, Hello ClickHouse ! +2 Well, Hello World ! Granules: 6/6 Granules: 2/6 Granules: 6/6 Granules: 2/6 --- -1 Hello ClickHouse -2 Hello World -6 World Champion +1 Well, Hello ClickHouse ! +2 Well, Hello World ! +6 True World Champion Granules: 6/6 Granules: 3/6 Granules: 6/6 Granules: 3/6 --- -5 OLAP Database +5 Its An OLAP Database Granules: 6/6 Granules: 1/6 Granules: 6/6 diff --git a/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql b/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql index 2233c8a1f31..7f36c423a41 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_match_predicate.sql @@ -1,6 +1,6 @@ -- Tests that match() utilizes the inverted index -SET allow_experimental_inverted_index = true; +SET allow_experimental_full_text_index = true; DROP TABLE IF EXISTS tab; @@ -14,19 +14,19 @@ ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 1; -INSERT INTO tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion'); +INSERT INTO tab VALUES (1, 'Well, Hello ClickHouse !'), (2, 'Well, Hello World !'), (3, 'Good Weather !'), (4, 'Say Hello !'), (5, 'Its An OLAP Database'), (6, 'True World Champion'); -SELECT * FROM tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id; +SELECT * FROM tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id; -- Read 2/6 granules --- Required string: 'Hello ' --- Alternatives: 'Hello ClickHouse', 'Hello World' +-- Required string: ' Hello ' +-- Alternatives: ' Hello ClickHouse ', ' Hello World ' SELECT * FROM ( EXPLAIN PLAN indexes=1 - SELECT * FROM tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id + SELECT * FROM tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -37,7 +37,7 @@ SELECT * FROM ( EXPLAIN PLAN indexes=1 - SELECT * FROM tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id + SELECT * FROM tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -46,17 +46,17 @@ SETTINGS SELECT '---'; -SELECT * FROM tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id; +SELECT * FROM tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id; -- Read 3/6 granules -- Required string: - --- Alternatives: 'ClickHouse', 'World' +-- Alternatives: ' ClickHouse ', ' World ' SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id + SELECT * FROM tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -67,7 +67,7 @@ SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id + SELECT * FROM tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -76,17 +76,17 @@ SETTINGS SELECT '---'; -SELECT * FROM tab WHERE match(str, 'OLAP.*') ORDER BY id; +SELECT * FROM tab WHERE match(str, ' OLAP .*') ORDER BY id; -- Read 1/6 granules --- Required string: 'OLAP' +-- Required string: ' OLAP ' -- Alternatives: - SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id + SELECT * FROM tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -97,7 +97,7 @@ SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id + SELECT * FROM tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id ) WHERE explain LIKE '%Granules: %' diff --git a/tests/queries/0_stateless/02346_fulltext_index_old_name.reference b/tests/queries/0_stateless/02346_fulltext_index_old_name.reference new file mode 100644 index 00000000000..ea5611c0c34 --- /dev/null +++ b/tests/queries/0_stateless/02346_fulltext_index_old_name.reference @@ -0,0 +1 @@ +2 bc diff --git a/tests/queries/0_stateless/02346_fulltext_index_old_name.sql b/tests/queries/0_stateless/02346_fulltext_index_old_name.sql new file mode 100644 index 00000000000..4e52e689211 --- /dev/null +++ b/tests/queries/0_stateless/02346_fulltext_index_old_name.sql @@ -0,0 +1,31 @@ +-- Index type 'inverted' was renamed to 'full_text' in April 2024. +-- Such indexes are experimental. Test what happens when ClickHouse encounters tables with the old index type. + +DROP TABLE IF EXISTS tab; + +-- It must be possible to load old tables with 'inverted'-type indexes +-- In stateless tests, we cannot use old persistences. Emulate "loading an old index" by creating it (internally, similar code executes). + +-- Creation only works with the (old) setting enabled. +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab(k UInt64, s String, INDEX idx(s) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k; -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab(k UInt64, s String, INDEX idx(s) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k; +INSERT INTO tab VALUES (1, 'ab') (2, 'bc'); + +-- Detach and attach should work. +DETACH TABLE tab; +ATTACH TABLE tab; + +-- To encourage users to migrate to the new index type, we now throw an exception when the index is used by queries. +SELECT * from tab WHERE s = 'bc'; -- { serverError ILLEGAL_INDEX } + +-- The exception recommends to drop the index and create a 'full_text' index instead. Let's try. +ALTER TABLE tab DROP INDEX idx; +SET allow_experimental_full_text_index = 1; -- the new setting +ALTER TABLE tab ADD INDEX idx(s) TYPE full_text(2); + +SELECT * from tab WHERE s = 'bc'; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02346_fulltext_index_search.reference b/tests/queries/0_stateless/02346_fulltext_index_search.reference index d742bbc77ec..d7c89d434e7 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_search.reference +++ b/tests/queries/0_stateless/02346_fulltext_index_search.reference @@ -13,19 +13,19 @@ af full_text 1 Test full_text() af full_text -101 Alick a01 -106 Alick a06 -111 Alick b01 -116 Alick b06 -101 Alick a01 -106 Alick a06 +101 x Alick a01 y +106 x Alick a06 y +111 x Alick b01 y +116 x Alick b06 y +101 x Alick a01 y +106 x Alick a06 y 1 -101 Alick a01 -111 Alick b01 +101 x Alick a01 y +111 x Alick b01 y 1 Test on array columns af full_text -3 ['Click a03','Click b03'] +3 ['x Click a03 y','x Click b03 y'] 1 Test on map columns af full_text diff --git a/tests/queries/0_stateless/02346_fulltext_index_search.sql b/tests/queries/0_stateless/02346_fulltext_index_search.sql index 62cd6073842..80f49790201 100644 --- a/tests/queries/0_stateless/02346_fulltext_index_search.sql +++ b/tests/queries/0_stateless/02346_fulltext_index_search.sql @@ -1,4 +1,4 @@ -SET allow_experimental_inverted_index = 1; +SET allow_experimental_full_text_index = 1; SET log_queries = 1; SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0; @@ -67,7 +67,7 @@ CREATE TABLE tab_x(k UInt64, s String, INDEX af(s) TYPE full_text()) ENGINE = MergeTree() ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; -INSERT INTO tab_x VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick a10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10'); +INSERT INTO tab_x VALUES (101, 'x Alick a01 y'), (102, 'x Blick a02 y'), (103, 'x Click a03 y'), (104, 'x Dlick a04 y'), (105, 'x Elick a05 y'), (106, 'x Alick a06 y'), (107, 'x Blick a07 y'), (108, 'x Click a08 y'), (109, 'x Dlick a09 y'), (110, 'x Elick a10 y'), (111, 'x Alick b01 y'), (112, 'x Blick b02 y'), (113, 'x Click b03 y'), (114, 'x Dlick b04 y'), (115, 'x Elick b05 y'), (116, 'x Alick b06 y'), (117, 'x Blick b07 y'), (118, 'x Click b08 y'), (119, 'x Dlick b09 y'), (120, 'x Elick b10 y'); -- check full_text index was created SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab_x' AND database = currentDatabase() LIMIT 1; @@ -86,27 +86,27 @@ SELECT read_rows==8 from system.query_log LIMIT 1; -- search full_text index with IN operator -SELECT * FROM tab_x WHERE s IN ('Alick a01', 'Alick a06') ORDER BY k; +SELECT * FROM tab_x WHERE s IN ('x Alick a01 y', 'x Alick a06 y') ORDER BY k; -- check the query only read 2 granules (4 rows total; each granule has 2 rows) SYSTEM FLUSH LOGS; SELECT read_rows==4 from system.query_log WHERE query_kind ='Select' AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE s IN (\'Alick a01\', \'Alick a06\') ORDER BY k;') + AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE s IN (\'x Alick a01 y\', \'x Alick a06 y\') ORDER BY k;') AND type='QueryFinish' AND result_rows==2 LIMIT 1; -- search full_text index with multiSearch -SELECT * FROM tab_x WHERE multiSearchAny(s, ['a01', 'b01']) ORDER BY k; +SELECT * FROM tab_x WHERE multiSearchAny(s, [' a01 ', ' b01 ']) ORDER BY k; -- check the query only read 2 granules (4 rows total; each granule has 2 rows) SYSTEM FLUSH LOGS; SELECT read_rows==4 from system.query_log WHERE query_kind ='Select' AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE multiSearchAny(s, [\'a01\', \'b01\']) ORDER BY k;') + AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE multiSearchAny(s, [\' a01 \', \' b01 \']) ORDER BY k;') AND type='QueryFinish' AND result_rows==2 LIMIT 1; @@ -126,14 +126,14 @@ INSERT INTO tab SELECT rowNumberInBlock(), groupArray(s) FROM tab_x GROUP BY k%1 SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; -- search full_text index with has -SELECT * FROM tab WHERE has(s, 'Click a03') ORDER BY k; +SELECT * FROM tab WHERE has(s, 'x Click a03 y') ORDER BY k; -- check the query must read all 10 granules (20 rows total; each granule has 2 rows) SYSTEM FLUSH LOGS; SELECT read_rows==2 from system.query_log WHERE query_kind ='Select' AND current_database = currentDatabase() - AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE has(s, \'Click a03\') ORDER BY k;') + AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE has(s, \'x Click a03 y\') ORDER BY k;') AND type='QueryFinish' AND result_rows==1 LIMIT 1; diff --git a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql index edc562a5c57..f2d294ff9e4 100644 --- a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql +++ b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql @@ -1,6 +1,6 @@ --- Tests that the inverted index can only be supported when allow_experimental_inverted_index = 1. +-- Tests that the inverted index can only be supported when allow_experimental_full_text_index = 1. -SET allow_experimental_inverted_index = 0; +SET allow_experimental_full_text_index = 0; DROP TABLE IF EXISTS tab; CREATE TABLE tab diff --git a/tests/queries/0_stateless/02366_kql_create_table.sql b/tests/queries/0_stateless/02366_kql_create_table.sql index b266679b06a..75a81c5dbd3 100644 --- a/tests/queries/0_stateless/02366_kql_create_table.sql +++ b/tests/queries/0_stateless/02366_kql_create_table.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS Customers; CREATE TABLE Customers -( +( FirstName Nullable(String), - LastName String, + LastName String, Occupation String, Education String, Age Nullable(UInt8) @@ -10,20 +10,20 @@ CREATE TABLE Customers INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); Select '-- test create table --' ; -Select * from kql(Customers|project FirstName) limit 1;; +Select * from kql($$Customers|project FirstName$$) limit 1;; DROP TABLE IF EXISTS kql_table1; -CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName | filter LastName=='Diaz'); +CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql($$Customers | project LastName | filter LastName=='Diaz'$$); select LastName from kql_table1 limit 1; DROP TABLE IF EXISTS kql_table2; CREATE TABLE kql_table2 -( +( FirstName Nullable(String), - LastName String, + LastName String, Age Nullable(UInt8) ) ENGINE = Memory; -INSERT INTO kql_table2 select * from kql(Customers|project FirstName,LastName,Age | filter FirstName=='Theodore'); +INSERT INTO kql_table2 select * from kql($$Customers|project FirstName,LastName,Age | filter FirstName=='Theodore'$$); select * from kql_table2 limit 1; --- select * from kql(Customers | where FirstName !in ("test", "test2")); +-- select * from kql($$Customers | where FirstName !in ("test", "test2")$$); DROP TABLE IF EXISTS Customers; DROP TABLE IF EXISTS kql_table1; -DROP TABLE IF EXISTS kql_table2; \ No newline at end of file +DROP TABLE IF EXISTS kql_table2; diff --git a/tests/queries/0_stateless/02375_pretty_formats.sql.j2 b/tests/queries/0_stateless/02375_pretty_formats.sql.j2 index 55462ea6b61..36a4479260c 100644 --- a/tests/queries/0_stateless/02375_pretty_formats.sql.j2 +++ b/tests/queries/0_stateless/02375_pretty_formats.sql.j2 @@ -3,6 +3,6 @@ 'PrettySpaceNoEscapesMonoBlock'] -%} select '{{ format }}'; -select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1 format {{ format }}; +select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1, output_format_pretty_display_footer_column_names=0 format {{ format }}; {% endfor -%} diff --git a/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql b/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql index 21dc7b1a990..510ed4b126f 100644 --- a/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql +++ b/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 'parseDateTime64BestEffortUS'; SELECT diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index cabcd230eb6..e9deb778075 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -3,7 +3,6 @@ SELECT name FROM system.functions WHERE NOT is_aggregate AND origin = 'System' AND alias_to = '' AND length(description) < 10 AND name NOT IN ( 'aes_decrypt_mysql', 'aes_encrypt_mysql', 'decrypt', 'encrypt', - 'base64Decode', 'base64Encode', 'tryBase64Decode', 'convertCharset', 'detectLanguage', 'detectLanguageMixed', 'geoToH3', diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index e635aec1163..61a057cc0ae 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -10,6 +10,9 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toInterv (1,2) (1,0) --- +() +() +--- 2022-10-12 2022-10-10 2022-10-12 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 9b2c3a475d2..69340199d3b 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -12,6 +12,10 @@ SELECT addTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH SELECT subtractTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); SELECT subtractInterval(tuple(INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT '---'; + +SELECT addInterval((), INTERVAL 1 MONTH); +SELECT subtractInterval(tuple(), INTERVAL 1 SECOND); SELECT '---'; @@ -68,4 +72,4 @@ WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; \ No newline at end of file +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py index 9941736107f..4165eeb6d31 100755 --- a/tests/queries/0_stateless/02473_infile_progress.py +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -32,12 +32,12 @@ with client( ) client1.expect(prompt) client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") - client1.expect("Progress: 5.00 rows, 10.00 B.*\)") + client1.expect("Progress: 5.00 rows, 10.00 B.*\\)") client1.expect(prompt) # send Ctrl-C client1.send("\x03", eol="") - match = client1.expect("(%s)|([#\$] )" % prompt) + match = client1.expect("(%s)|([#\\$] )" % prompt) if match.groups()[1]: client1.send(client1.command) client1.expect(prompt) diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index 649b037fafa..11c178ac0d0 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -10,11 +10,15 @@ QUERY id: 0 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 - CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: LowCardinality(UInt8) + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality @@ -28,11 +32,15 @@ QUERY id: 0 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 - CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: LowCardinality(UInt8) + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality @@ -46,11 +54,15 @@ QUERY id: 0 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 - CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + FUNCTION id: 6, function_name: notIn, function_type: ordinary, result_type: LowCardinality(UInt8) + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality @@ -64,11 +76,15 @@ QUERY id: 0 JOIN TREE TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 - CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + FUNCTION id: 6, function_name: notIn, function_type: ordinary, result_type: LowCardinality(UInt8) + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality diff --git a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh index b01f16e1cad..60cec5caea3 100755 --- a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh +++ b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is rm -f $path/count.txt $CLICKHOUSE_CLIENT -q "detach table rmt2 sync" -$CLICKHOUSE_CLIENT --send_logs_level='fatal' -q "attach table rmt2" +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level='fatal' -q "attach table rmt2" $CLICKHOUSE_CLIENT -q "select reason, name from system.detached_parts where database='$CLICKHOUSE_DATABASE' and table='rmt2'" diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 6bc3d03ac66..24803ed7420 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -20,12 +20,12 @@ SETTINGS_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" diff --git a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql index 9988eef0ad3..7b49378d4da 100644 --- a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql +++ b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql @@ -9,6 +9,6 @@ ENGINE = Memory; INSERT INTO data_a_02187 SELECT * FROM system.one -SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', distributed_foreground_insert = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', allow_experimental_map_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; +SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', distributed_foreground_insert = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; DROP TABLE data_a_02187; diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 88c910e0313..4d5a7447a49 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -10,7 +10,7 @@ SELECT key, value FROM dict_flat ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -27,7 +27,7 @@ SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -44,7 +44,7 @@ SELECT key, value FROM dict_hashed ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -61,7 +61,7 @@ SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -78,7 +78,7 @@ SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -95,7 +95,7 @@ SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 35881bdf896..9ced78a1196 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -60,7 +60,7 @@ for layout in "${layouts[@]}"; do INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); - SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; -- { echoOff } diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 9850406eb3a..dccb680be42 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -30,7 +30,7 @@ run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" -$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x)" +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x)" $CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number % 3 FROM numbers(1000)" function run_count_with_custom_key { @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index ff8b9c71e92..741d0177971 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -1,4 +1,4 @@ --- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan, no-parallel +-- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-sanitize-coverage -- no-parallel because the sets use a lot of memory, which may interfere with other tests diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index e6c4d5768af..6e8325a2ff3 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -24,10 +24,10 @@ Positions: 3 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table) Header: id UInt64 value String diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql index 5b9976714ea..b0c08134816 100644 --- a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_color=1; SET read_in_order_two_level_merge_threshold=1000000; diff --git a/tests/queries/0_stateless/02864_statistic_exception.sql b/tests/queries/0_stateless/02864_statistic_exception.sql deleted file mode 100644 index 092fa9bda85..00000000000 --- a/tests/queries/0_stateless/02864_statistic_exception.sql +++ /dev/null @@ -1,53 +0,0 @@ -DROP TABLE IF EXISTS t1; - -CREATE TABLE t1 -( - a Float64 STATISTIC(tdigest), - b Int64 STATISTIC(tdigest), - pk String, -) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } - -SET allow_experimental_statistic = 1; - -CREATE TABLE t1 -( - a Float64 STATISTIC(tdigest), - b Int64, - pk String STATISTIC(tdigest), -) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTIC } - -CREATE TABLE t1 -( - a Float64 STATISTIC(tdigest, tdigest(10)), - b Int64, -) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } - -CREATE TABLE t1 -( - a Float64 STATISTIC(xyz), - b Int64, -) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } - -CREATE TABLE t1 -( - a Float64, - b Int64, - pk String, -) Engine = MergeTree() ORDER BY pk; - -ALTER TABLE t1 ADD STATISTIC a TYPE xyz; -- { serverError INCORRECT_QUERY } -ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } -ALTER TABLE t1 ADD STATISTIC pk TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } -ALTER TABLE t1 DROP STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } -ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; -ALTER TABLE t1 DROP STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } -ALTER TABLE t1 CLEAR STATISTIC a TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } -ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_STATISTIC } - -ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; -ALTER TABLE t1 ADD STATISTIC b TYPE tdigest; -ALTER TABLE t1 MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; -ALTER TABLE t1 MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - -DROP TABLE t1; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql new file mode 100644 index 00000000000..c531d39cd69 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_exception.sql @@ -0,0 +1,57 @@ +DROP TABLE IF EXISTS t1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + pk String, +) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } + +SET allow_experimental_statistics = 1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64, + pk String STATISTICS(tdigest), +) Engine = MergeTree() ORDER BY pk; -- { serverError ILLEGAL_STATISTICS } + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest, tdigest(10)), + b Int64, +) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } + +CREATE TABLE t1 +( + a Float64 STATISTICS(xyz), + b Int64, +) Engine = MergeTree() ORDER BY pk; -- { serverError INCORRECT_QUERY } + +CREATE TABLE t1 +( + a Float64, + b Int64, + pk String, +) Engine = MergeTree() ORDER BY pk; + +ALTER TABLE t1 ADD STATISTICS a TYPE xyz; -- { serverError INCORRECT_QUERY } +ALTER TABLE t1 ADD STATISTICS a TYPE tdigest; +ALTER TABLE t1 ADD STATISTICS IF NOT EXISTS a TYPE tdigest; +ALTER TABLE t1 ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +-- Statistics can be created only on integer columns +ALTER TABLE t1 MODIFY STATISTICS a TYPE tdigest; +ALTER TABLE t1 ADD STATISTICS pk TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE t1 DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE t1 DROP STATISTICS a; +ALTER TABLE t1 DROP STATISTICS IF EXISTS a; +ALTER TABLE t1 CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE t1 CLEAR STATISTICS IF EXISTS a; +ALTER TABLE t1 MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } + +ALTER TABLE t1 ADD STATISTICS a TYPE tdigest; +ALTER TABLE t1 ADD STATISTICS b TYPE tdigest; +ALTER TABLE t1 MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; +ALTER TABLE t1 MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +DROP TABLE t1; diff --git a/tests/queries/0_stateless/02864_statistic_operate.reference b/tests/queries/0_stateless/02864_statistics_operate.reference similarity index 58% rename from tests/queries/0_stateless/02864_statistic_operate.reference rename to tests/queries/0_stateless/02864_statistics_operate.reference index 3e291485031..6398a9bd000 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.reference +++ b/tests/queries/0_stateless/02864_statistics_operate.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After insert Prewhere info Prewhere filter @@ -12,7 +12,7 @@ After drop statistic 10 CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After add statistic -CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After materialize statistic Prewhere info Prewhere filter @@ -23,7 +23,7 @@ After merge Prewhere filter Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) 20 -CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `c` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 After rename Prewhere info Prewhere filter diff --git a/tests/queries/0_stateless/02864_statistic_operate.sql b/tests/queries/0_stateless/02864_statistics_operate.sql similarity index 87% rename from tests/queries/0_stateless/02864_statistic_operate.sql rename to tests/queries/0_stateless/02864_statistics_operate.sql index 5f1c30f8eec..bf69c11bc91 100644 --- a/tests/queries/0_stateless/02864_statistic_operate.sql +++ b/tests/queries/0_stateless/02864_statistics_operate.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS t1; -SET allow_experimental_statistic = 1; -SET allow_statistic_optimize = 1; +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; CREATE TABLE t1 ( - a Float64 STATISTIC(tdigest), - b Int64 STATISTIC(tdigest), + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; @@ -20,7 +20,7 @@ SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions= SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SELECT count(*) FROM t1 WHERE b < NULL and a < '10'; -ALTER TABLE t1 DROP STATISTIC a, b TYPE tdigest; +ALTER TABLE t1 DROP STATISTICS a, b; SELECT 'After drop statistic'; SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; @@ -28,13 +28,13 @@ SELECT count(*) FROM t1 WHERE b < 10 and a < 10; SHOW CREATE TABLE t1; -ALTER TABLE t1 ADD STATISTIC a, b TYPE tdigest; +ALTER TABLE t1 ADD STATISTICS a, b TYPE tdigest; SELECT 'After add statistic'; SHOW CREATE TABLE t1; -ALTER TABLE t1 MATERIALIZE STATISTIC a, b TYPE tdigest; +ALTER TABLE t1 MATERIALIZE STATISTICS a, b; INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; SELECT 'After materialize statistic'; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference new file mode 100644 index 00000000000..77786dbdd8c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.reference @@ -0,0 +1,35 @@ +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After modify TDigest + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) +After drop + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql new file mode 100644 index 00000000000..c6b51d2a377 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.sql @@ -0,0 +1,71 @@ +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Int64 STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +OPTIMIZE TABLE t1 FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'After modify TDigest'; +ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; +ALTER TABLE t1 MATERIALIZE STATISTICS c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + + +ALTER TABLE t1 DROP STATISTICS c; + +SELECT 'After drop'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c LowCardinality(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Nullable(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t3; diff --git a/tests/queries/0_stateless/02884_string_distance_function.reference b/tests/queries/0_stateless/02884_string_distance_function.reference index e8cce2017d9..71b15bc8753 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.reference +++ b/tests/queries/0_stateless/02884_string_distance_function.reference @@ -13,53 +13,84 @@ clickhouse mouse 6 -- non-const arguments byteHammingDistance 0 byteHammingDistance abc 3 +byteHammingDistance Jerry 我是谁 9 byteHammingDistance abc 3 byteHammingDistance abc ab 1 byteHammingDistance abc abc 0 byteHammingDistance abc bc 3 byteHammingDistance clickhouse mouse 10 +byteHammingDistance 我是谁 Tom 9 +byteHammingDistance 我是谁 我是我 3 editDistance 0 editDistance abc 3 +editDistance Jerry 我是谁 9 editDistance abc 3 editDistance abc ab 1 editDistance abc abc 0 editDistance abc bc 1 editDistance clickhouse mouse 6 +editDistance 我是谁 Tom 9 +editDistance 我是谁 我是我 3 +editDistanceUTF8 0 +editDistanceUTF8 abc 3 +editDistanceUTF8 Jerry 我是谁 5 +editDistanceUTF8 abc 3 +editDistanceUTF8 abc ab 1 +editDistanceUTF8 abc abc 0 +editDistanceUTF8 abc bc 1 +editDistanceUTF8 clickhouse mouse 6 +editDistanceUTF8 我是谁 Tom 3 +editDistanceUTF8 我是谁 我是我 1 damerauLevenshteinDistance 0 damerauLevenshteinDistance abc 3 +damerauLevenshteinDistance Jerry 我是谁 9 damerauLevenshteinDistance abc 3 damerauLevenshteinDistance abc ab 1 damerauLevenshteinDistance abc abc 0 damerauLevenshteinDistance abc bc 1 damerauLevenshteinDistance clickhouse mouse 6 +damerauLevenshteinDistance 我是谁 Tom 9 +damerauLevenshteinDistance 我是谁 我是我 3 stringJaccardIndex 0 stringJaccardIndex abc 0 +stringJaccardIndex Jerry 我是谁 0 stringJaccardIndex abc 0 stringJaccardIndex abc ab 0.6666666666666666 stringJaccardIndex abc abc 1 stringJaccardIndex abc bc 0.6666666666666666 stringJaccardIndex clickhouse mouse 0.4 +stringJaccardIndex 我是谁 Tom 0 +stringJaccardIndex 我是谁 我是我 0.625 stringJaccardIndexUTF8 0 stringJaccardIndexUTF8 abc 0 +stringJaccardIndexUTF8 Jerry 我是谁 0 stringJaccardIndexUTF8 abc 0 stringJaccardIndexUTF8 abc ab 0.6666666666666666 stringJaccardIndexUTF8 abc abc 1 stringJaccardIndexUTF8 abc bc 0.6666666666666666 stringJaccardIndexUTF8 clickhouse mouse 0.4 +stringJaccardIndexUTF8 我是谁 Tom 0 +stringJaccardIndexUTF8 我是谁 我是我 0.6666666666666666 jaroSimilarity 0 jaroSimilarity abc 3 +jaroSimilarity Jerry 我是谁 0 jaroSimilarity abc 3 jaroSimilarity abc ab 0.8888888888888888 jaroSimilarity abc abc 1 jaroSimilarity abc bc 0 jaroSimilarity clickhouse mouse 0 +jaroSimilarity 我是谁 Tom 0 +jaroSimilarity 我是谁 我是我 0.7777777777777777 jaroWinklerSimilarity 0 jaroWinklerSimilarity abc 3 +jaroWinklerSimilarity Jerry 我是谁 0 jaroWinklerSimilarity abc 3 jaroWinklerSimilarity abc ab 0.9111111111111111 jaroWinklerSimilarity abc abc 1 jaroWinklerSimilarity abc bc 0 jaroWinklerSimilarity clickhouse mouse 0 +jaroWinklerSimilarity 我是谁 Tom 0 +jaroWinklerSimilarity 我是谁 我是我 0.8666666666666666 -- Special UTF-8 tests 0.4 0 diff --git a/tests/queries/0_stateless/02884_string_distance_function.sql b/tests/queries/0_stateless/02884_string_distance_function.sql index fddbf41f0e5..482996e1448 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.sql +++ b/tests/queries/0_stateless/02884_string_distance_function.sql @@ -26,11 +26,12 @@ CREATE TABLE t ) ENGINE = MergeTree ORDER BY s1; -- actual test cases -INSERT INTO t VALUES ('', '') ('abc', '') ('', 'abc') ('abc', 'abc') ('abc', 'ab') ('abc', 'bc') ('clickhouse', 'mouse'); +INSERT INTO t VALUES ('', '') ('abc', '') ('', 'abc') ('abc', 'abc') ('abc', 'ab') ('abc', 'bc') ('clickhouse', 'mouse') ('我是谁', 'Tom') ('Jerry', '我是谁') ('我是谁', '我是我'); SELECT '-- non-const arguments'; SELECT 'byteHammingDistance', s1, s2, byteHammingDistance(s1, s2) FROM t ORDER BY ALL; SELECT 'editDistance', s1, s2, editDistance(s1, s2) FROM t ORDER BY ALL; +SELECT 'editDistanceUTF8', s1, s2, editDistanceUTF8(s1, s2) FROM t ORDER BY ALL; SELECT 'damerauLevenshteinDistance', s1, s2, damerauLevenshteinDistance(s1, s2) FROM t ORDER BY ALL; SELECT 'stringJaccardIndex', s1, s2, stringJaccardIndex(s1, s2) FROM t ORDER BY ALL; SELECT 'stringJaccardIndexUTF8', s1, s2, stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY ALL; diff --git a/tests/queries/0_stateless/02891_empty_tuple.reference b/tests/queries/0_stateless/02891_empty_tuple.reference new file mode 100644 index 00000000000..3ead41c1167 --- /dev/null +++ b/tests/queries/0_stateless/02891_empty_tuple.reference @@ -0,0 +1,7 @@ +2 +() () +() () +() +2 +() [(),()] +() [] diff --git a/tests/queries/0_stateless/02891_empty_tuple.sql b/tests/queries/0_stateless/02891_empty_tuple.sql new file mode 100644 index 00000000000..c8669d4a020 --- /dev/null +++ b/tests/queries/0_stateless/02891_empty_tuple.sql @@ -0,0 +1,27 @@ +drop table if exists x; + +create table x engine MergeTree order by () as select () as a, () as b; + +insert into x values ((), ()); + +select count() from x; + +select * from x order by (); + +select (); + +drop table x; + +drop table if exists x; + +create table x (i Nullable(Tuple())) engine MergeTree order by (); -- { serverError 43 } +create table x (i LowCardinality(Tuple())) engine MergeTree order by (); -- { serverError 43 } +create table x (i Tuple(), j Array(Tuple())) engine MergeTree order by (); + +insert into x values ((), [(), ()]), ((), []); + +select count() from x; + +select * from x order by () settings max_threads = 1; + +drop table x; diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference index 9059b403a34..903d42bf492 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference @@ -1,8 +1,4 @@ number Nullable(Int64) -u8 Nullable(Int8) -u16 Nullable(Int16) -u32 Nullable(Int32) -u64 Nullable(Int64) i8 Nullable(Int8) i16 Nullable(Int16) i32 Nullable(Int32) @@ -22,34 +18,34 @@ d64 Nullable(Decimal(18, 10)) d128 Nullable(Decimal(38, 20)) -- Go over all types individually -- { echoOn } -select count(), sum(number) from file('02892.orc') where indexHint(u8 in (10, 15, 250)); -800 4229600 -select count(1), min(u8), max(u8) from file('02892.orc') where u8 in (10, 15, 250); -66 10 15 +select count(), sum(number) from file('02892.orc') where indexHint(i8 in (10, 15, -6)); +1100 5744450 +select count(1), min(i8), max(i8) from file('02892.orc') where i8 in (10, 15, -6); +99 -6 15 select count(), sum(number) from file('02892.orc') where indexHint(i8 between -3 and 2); 1000 4999500 select count(1), min(i8), max(i8) from file('02892.orc') where i8 between -3 and 2; 208 -3 2 -select count(), sum(number) from file('02892.orc') where indexHint(u16 between 4000 and 61000 or u16 == 42); -1800 6479100 -select count(1), min(u16), max(u16) from file('02892.orc') where u16 between 4000 and 61000 or u16 == 42; +select count(), sum(number) from file('02892.orc') where indexHint(i16 between 4000 and 61000 or i16 == 42); +1200 1099400 +select count(1), min(i16), max(i16) from file('02892.orc') where i16 between 4000 and 61000 or i16 == 42; 1002 42 5000 select count(), sum(number) from file('02892.orc') where indexHint(i16 between -150 and 250); 500 2474750 select count(1), min(i16), max(i16) from file('02892.orc') where i16 between -150 and 250; 401 -150 250 -select count(), sum(number) from file('02892.orc') where indexHint(u32 in (42, 4294966296)); -200 999900 -select count(1), min(u32), max(u32) from file('02892.orc') where u32 in (42, 4294966296); -1 42 42 +select count(), sum(number) from file('02892.orc') where indexHint(i32 in (42, -1000)); +200 1099900 +select count(1), min(i32), max(i32) from file('02892.orc') where i32 in (42, -1000); +2 -1000 42 select count(), sum(number) from file('02892.orc') where indexHint(i32 between -150 and 250); 500 2474750 select count(1), min(i32), max(i32) from file('02892.orc') where i32 between -150 and 250; 401 -150 250 -select count(), sum(number) from file('02892.orc') where indexHint(u64 in (42, 18446744073709550616)); -100 494950 -select count(1), min(u64), max(u64) from file('02892.orc') where u64 in (42, 18446744073709550616); -1 42 42 +select count(), sum(number) from file('02892.orc') where indexHint(i64 in (42, -1000)); +200 1099900 +select count(1), min(i64), max(i64) from file('02892.orc') where i64 in (42, -1000); +2 -1000 42 select count(), sum(number) from file('02892.orc') where indexHint(i64 between -150 and 250); 500 2474750 select count(1), min(i64), max(i64) from file('02892.orc') where i64 between -150 and 250; @@ -111,21 +107,21 @@ select count(), sum(number) from file('02892.orc') where indexHint(0); 0 \N select count(), min(number), max(number) from file('02892.orc') where indexHint(0); 0 \N \N -select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or u64 == 2000); +select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or i64 == 2000); 300 1204850 -select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or u64 == 2000); +select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or i64 == 2000); 12 2000 999 select count(), sum(number) from file('02892.orc') where indexHint(s like 'z%'); 0 \N select count(), min(s), max(s) from file('02892.orc') where (s like 'z%'); 0 \N \N -select count(), sum(number) from file('02892.orc') where indexHint(u8 == 10 or 1 == 1); +select count(), sum(number) from file('02892.orc') where indexHint(i8 == 10 or 1 == 1); 10000 49995000 -select count(), min(u8), max(u8) from file('02892.orc') where (u8 == 10 or 1 == 1); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 == 10 or 1 == 1); 10000 -128 127 -select count(), sum(number) from file('02892.orc') where indexHint(u8 < 0); +select count(), sum(number) from file('02892.orc') where indexHint(i8 < 0); 5300 26042350 -select count(), min(u8), max(u8) from file('02892.orc') where (u8 < 0); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 < 0); 5001 -128 -1 -- { echoOn } select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null is NULL); @@ -210,7 +206,7 @@ select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null < -500); 596 -1099 -501 select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null); -1000 499500 +0 0 select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null is null); 0 0 0 select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null in (0, -1, -10, -100, -1000)); diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql index f9aa7696ac6..e3736de6a17 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-cpu-aarch64 +-- Tags: no-fasttest, no-parallel set output_format_orc_string_as_string = 1; set output_format_orc_row_index_stride = 100; @@ -16,15 +16,9 @@ SET session_timezone = 'UTC'; -- Try all the types. insert into function file('02892.orc') - -- Use negative numbers to test sign extension for signed types and lack of sign extension for - -- unsigned types. with 5000 - number as n select number, - intDiv(n, 11)::UInt8 as u8, - n::UInt16 u16, - n::UInt32 as u32, - n::UInt64 as u64, intDiv(n, 11)::Int8 as i8, n::Int16 i16, n::Int32 as i32, @@ -50,26 +44,26 @@ desc file('02892.orc'); -- Go over all types individually -- { echoOn } -select count(), sum(number) from file('02892.orc') where indexHint(u8 in (10, 15, 250)); -select count(1), min(u8), max(u8) from file('02892.orc') where u8 in (10, 15, 250); +select count(), sum(number) from file('02892.orc') where indexHint(i8 in (10, 15, -6)); +select count(1), min(i8), max(i8) from file('02892.orc') where i8 in (10, 15, -6); select count(), sum(number) from file('02892.orc') where indexHint(i8 between -3 and 2); select count(1), min(i8), max(i8) from file('02892.orc') where i8 between -3 and 2; -select count(), sum(number) from file('02892.orc') where indexHint(u16 between 4000 and 61000 or u16 == 42); -select count(1), min(u16), max(u16) from file('02892.orc') where u16 between 4000 and 61000 or u16 == 42; +select count(), sum(number) from file('02892.orc') where indexHint(i16 between 4000 and 61000 or i16 == 42); +select count(1), min(i16), max(i16) from file('02892.orc') where i16 between 4000 and 61000 or i16 == 42; select count(), sum(number) from file('02892.orc') where indexHint(i16 between -150 and 250); select count(1), min(i16), max(i16) from file('02892.orc') where i16 between -150 and 250; -select count(), sum(number) from file('02892.orc') where indexHint(u32 in (42, 4294966296)); -select count(1), min(u32), max(u32) from file('02892.orc') where u32 in (42, 4294966296); +select count(), sum(number) from file('02892.orc') where indexHint(i32 in (42, -1000)); +select count(1), min(i32), max(i32) from file('02892.orc') where i32 in (42, -1000); select count(), sum(number) from file('02892.orc') where indexHint(i32 between -150 and 250); select count(1), min(i32), max(i32) from file('02892.orc') where i32 between -150 and 250; -select count(), sum(number) from file('02892.orc') where indexHint(u64 in (42, 18446744073709550616)); -select count(1), min(u64), max(u64) from file('02892.orc') where u64 in (42, 18446744073709550616); +select count(), sum(number) from file('02892.orc') where indexHint(i64 in (42, -1000)); +select count(1), min(i64), max(i64) from file('02892.orc') where i64 in (42, -1000); select count(), sum(number) from file('02892.orc') where indexHint(i64 between -150 and 250); select count(1), min(i64), max(i64) from file('02892.orc') where i64 between -150 and 250; @@ -117,17 +111,17 @@ select count(1), min(d128), max(128) from file('02892.orc') where (d128 between select count(), sum(number) from file('02892.orc') where indexHint(0); select count(), min(number), max(number) from file('02892.orc') where indexHint(0); -select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or u64 == 2000); -select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or u64 == 2000); +select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or i64 == 2000); +select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or i64 == 2000); select count(), sum(number) from file('02892.orc') where indexHint(s like 'z%'); select count(), min(s), max(s) from file('02892.orc') where (s like 'z%'); -select count(), sum(number) from file('02892.orc') where indexHint(u8 == 10 or 1 == 1); -select count(), min(u8), max(u8) from file('02892.orc') where (u8 == 10 or 1 == 1); +select count(), sum(number) from file('02892.orc') where indexHint(i8 == 10 or 1 == 1); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 == 10 or 1 == 1); -select count(), sum(number) from file('02892.orc') where indexHint(u8 < 0); -select count(), min(u8), max(u8) from file('02892.orc') where (u8 < 0); +select count(), sum(number) from file('02892.orc') where indexHint(i8 < 0); +select count(), min(i8), max(i8) from file('02892.orc') where (i8 < 0); -- { echoOff } -- Nullable and LowCardinality. diff --git a/tests/queries/0_stateless/02893_vertical_final_array_join.reference b/tests/queries/0_stateless/02893_vertical_final_bugs.reference similarity index 99% rename from tests/queries/0_stateless/02893_vertical_final_array_join.reference rename to tests/queries/0_stateless/02893_vertical_final_bugs.reference index 27b54a2e42e..ab23116aa5f 100644 --- a/tests/queries/0_stateless/02893_vertical_final_array_join.reference +++ b/tests/queries/0_stateless/02893_vertical_final_bugs.reference @@ -1,3 +1,4 @@ +1 2 b 1 -- { echo ON } SELECT arrayJoin([(k1, v), (k2, v)]) AS row, row.1 as k FROM t FINAL WHERE k1 != 3 AND k = 1 ORDER BY row SETTINGS enable_vertical_final = 0; (1,4) 1 diff --git a/tests/queries/0_stateless/02893_vertical_final_array_join.sql b/tests/queries/0_stateless/02893_vertical_final_bugs.sql similarity index 65% rename from tests/queries/0_stateless/02893_vertical_final_array_join.sql rename to tests/queries/0_stateless/02893_vertical_final_bugs.sql index cc2e37fdc6e..e82ab674c58 100644 --- a/tests/queries/0_stateless/02893_vertical_final_array_join.sql +++ b/tests/queries/0_stateless/02893_vertical_final_bugs.sql @@ -1,3 +1,15 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64543 +DROP TABLE IF EXISTS foo; +DROP TABLE IF EXISTS bar; +CREATE TABLE foo (id UInt64, seq UInt64) ENGINE = Memory; +CREATE TABLE bar (id UInt64, seq UInt64, name String) ENGINE = ReplacingMergeTree ORDER BY id; +INSERT INTO foo VALUES (1, 1); +INSERT INTO bar VALUES (1, 1, 'a') (2, 2, 'b'); +INSERT INTO bar VALUES (1, 2, 'b') (2, 3, 'c'); +SELECT * FROM bar INNER JOIN foo USING id WHERE bar.seq > foo.seq SETTINGS final = 1; + +-- Same problem possible can happen with array join +DROP TABLE IF EXISTS t; CREATE TABLE t (k1 UInt64, k2 UInt64, v UInt64) ENGINE = ReplacingMergeTree() ORDER BY (k1, k2); SET optimize_on_insert = 0; INSERT INTO t VALUES (1, 2, 3) (1, 2, 4) (2, 3, 4), (2, 3, 5); diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference index caf11f5c15a..e537236478d 100644 --- a/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference @@ -1,2 +1,3 @@ 1 2 3 1 5 +300 diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql index 5fb628eeb67..5440872e052 100644 --- a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql @@ -30,3 +30,7 @@ WHERE (time_stamp_utc >= toDateTime('2024-04-25 00:00:00')) AND (time_stamp_utc GROUP BY time_stamp_utc ORDER BY Impressions DESC LIMIT 1000; + +drop table test_table; +create table test_table engine MergeTree order by sum as select 100 as sum union all select 200 as sum; +select sum as sum from (select sum(sum) as sum from test_table); diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index 144831a2cdc..a247c99a818 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-parallel, no-fasttest, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_1.reference b/tests/queries/0_stateless/02915_analyzer_fuzz_1.reference new file mode 100644 index 00000000000..ac3f57c1a2e --- /dev/null +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_1.reference @@ -0,0 +1 @@ +With ba\0 diff --git a/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql b/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql new file mode 100644 index 00000000000..94849453063 --- /dev/null +++ b/tests/queries/0_stateless/02915_analyzer_fuzz_1.sql @@ -0,0 +1,2 @@ +set allow_experimental_analyzer=1; +SELECT concat('With ', materialize(_CAST('ba\0', 'LowCardinality(FixedString(3))'))) AS `concat('With ', materialize(CAST('ba\\0', 'LowCardinality(FixedString(3))')))` FROM system.one GROUP BY 'With '; diff --git a/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference b/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference index 786a6b3bf25..7278018f1d6 100644 --- a/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference +++ b/tests/queries/0_stateless/02918_optimize_count_for_merge_tables.reference @@ -7,6 +7,9 @@ Expression ((Projection + Before ORDER BY)) Aggregating Expression (Before GROUP BY) ReadFromMerge - ReadFromMergeTree (default.mt1) - ReadFromMergeTree (default.mt2) - ReadFromStorage (TinyLog) + Expression + ReadFromMergeTree (default.mt1) + Expression + ReadFromMergeTree (default.mt2) + Expression + ReadFromStorage (TinyLog) diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index bb013dccb65..dad4e6747e0 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -58,9 +58,9 @@ function filter_temporary_locks() function insert_duplicates() { - $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --send_logs_level="error" & + $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --allow_repeated_settings --send_logs_level="error" & - $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --send_logs_level="error" + $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --allow_repeated_settings --send_logs_level="error" wait @@ -137,8 +137,8 @@ function list_keeper_nodes() { list_keeper_nodes "${table_shared_id}" -$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --send_logs_level="error" & -$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --allow_repeated_settings --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --allow_repeated_settings --send_logs_level="error" & wait list_keeper_nodes "${table_shared_id}" diff --git a/tests/queries/0_stateless/02922_server_exit_code.sh b/tests/queries/0_stateless/02922_server_exit_code.sh index 60049902410..ded0dc4763f 100755 --- a/tests/queries/0_stateless/02922_server_exit_code.sh +++ b/tests/queries/0_stateless/02922_server_exit_code.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We will check that the server's exit code corresponds to the exception code if it was terminated after exception. # In this example, we provide an invalid path to the server's config, ignore its logs and check the exit code. -# The exception code is 400 = CANNOT_STAT, so the exit code will be 400 % 256. +# The exception code is 76 = CANNOT_OPEN_FILE, so the exit code will be 76 % 256. -${CLICKHOUSE_SERVER_BINARY} -- --path /dev/null 2>/dev/null; [[ "$?" == "$((400 % 256))" ]] && echo 'Ok' || echo 'Fail' +${CLICKHOUSE_SERVER_BINARY} -- --path /dev/null 2>/dev/null; [[ "$?" == "$((76 % 256))" ]] && echo 'Ok' || echo 'Fail' diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh index 22ca909a26e..723de45eaad 100755 --- a/tests/queries/0_stateless/02941_variant_type_1.sh +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test1_insert() { @@ -115,11 +115,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index 91ba0285bd8..f43cd2bb0d6 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test4_insert() { @@ -61,11 +61,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index 8a039a02d6d..f4b2b304f56 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test5_insert() { @@ -63,11 +63,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index e38db8fda54..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -7,7 +7,8 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { @@ -57,11 +58,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02942_variant_cast.sql b/tests/queries/0_stateless/02942_variant_cast.sql index fc2d1d63657..33587e3e438 100644 --- a/tests/queries/0_stateless/02942_variant_cast.sql +++ b/tests/queries/0_stateless/02942_variant_cast.sql @@ -1,5 +1,4 @@ set allow_experimental_variant_type=1; -set allow_experimental_analyzer=0; -- It's currently doesn't work with analyzer because of the way it works with constants, but it will be refactored and fixed in future select NULL::Variant(String, UInt64); select 42::UInt64::Variant(String, UInt64); diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference index 0e1954cde62..5b7ad7ddce0 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference @@ -1,5 +1,5 @@ -1 Hello ClickHouse -2 Hello World +1 Well, Hello ClickHouse ! +2 Well, Hello World ! 1 Hello ClickHouse 2 Hello World Granules: 6/6 @@ -11,9 +11,9 @@ Granules: 6/6 Granules: 2/6 --- -1 Hello ClickHouse -2 Hello World -6 World Champion +1 Well, Hello ClickHouse ! +2 Well, Hello World ! +6 True World Champion 1 Hello ClickHouse 2 Hello World 6 World Champion @@ -26,7 +26,7 @@ Granules: 6/6 Granules: 3/6 --- -5 OLAP Database +5 Its An OLAP Database 5 OLAP Database Granules: 6/6 Granules: 1/6 diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql index 49d39c601ef..42175cbb2c6 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql @@ -21,21 +21,22 @@ ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 1; -INSERT INTO tokenbf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion'); +INSERT INTO tokenbf_tab VALUES (1, 'Well, Hello ClickHouse !'), (2, 'Well, Hello World !'), (3, 'Good Weather !'), (4, 'Say Hello !'), (5, 'Its An OLAP Database'), (6, 'True World Champion'); INSERT INTO ngrambf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion'); -SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id; +SELECT * FROM tokenbf_tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id; SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id; -- Read 2/6 granules -- Required string: 'Hello ' -- Alternatives: 'Hello ClickHouse', 'Hello World' +-- Surrounded by spaces for tokenbf SELECT * FROM ( EXPLAIN PLAN indexes=1 - SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id + SELECT * FROM tokenbf_tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -46,7 +47,7 @@ SELECT * FROM ( EXPLAIN PLAN indexes=1 - SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id + SELECT * FROM tokenbf_tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -78,18 +79,19 @@ SETTINGS SELECT '---'; -SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id; +SELECT * FROM tokenbf_tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id; SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id; -- Read 3/6 granules -- Required string: - -- Alternatives: 'ClickHouse', 'World' +-- Surrounded by spaces for tokenbf SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id + SELECT * FROM tokenbf_tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -100,7 +102,7 @@ SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id + SELECT * FROM tokenbf_tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -131,18 +133,19 @@ SETTINGS SELECT '---'; -SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP.*') ORDER BY id; +SELECT * FROM tokenbf_tab WHERE match(str, ' OLAP .*') ORDER BY id; SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP.*') ORDER BY id; -- Read 1/6 granules -- Required string: 'OLAP' -- Alternatives: - +-- Surrounded by spaces for tokenbf SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id + SELECT * FROM tokenbf_tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id ) WHERE explain LIKE '%Granules: %' @@ -152,7 +155,7 @@ SELECT * FROM ( EXPLAIN PLAN indexes = 1 - SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id + SELECT * FROM tokenbf_tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id ) WHERE explain LIKE '%Granules: %' diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index e985cf365dd..49ea5f2769c 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,5 +1,3 @@ -set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. It wil be fixed after refactoring around constants in analyzer. - set allow_experimental_variant_type=1; set use_variant_as_common_type=1; diff --git a/tests/queries/0_stateless/02969_auto_format_detection.reference b/tests/queries/0_stateless/02969_auto_format_detection.reference index 865db11defc..4b86be04996 100644 --- a/tests/queries/0_stateless/02969_auto_format_detection.reference +++ b/tests/queries/0_stateless/02969_auto_format_detection.reference @@ -82,7 +82,8 @@ CSV c1 Nullable(UInt64) c2 Nullable(String) c3 Array(Nullable(UInt64)) -c4 Tuple(Nullable(UInt64), Nullable(String)) +c4 Nullable(UInt64) +c5 Nullable(String) a Nullable(String) b Nullable(String) c Array(Nullable(String)) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql index d00cc00e097..f30e217ca0f 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql @@ -1,5 +1,9 @@ -- Tags: no-parallel +SET output_format_csv_serialize_tuple_into_separate_columns = false; +SET input_format_csv_deserialize_separate_columns_into_tuple = false; +SET input_format_csv_try_infer_strings_from_quoted_tuples = false; + insert into function file('02977_1.csv') select '20240305', 1, ['s', 'd'], map('a', 2), tuple('222', 33, map('abc', 5)) SETTINGS engine_file_truncate_on_insert=1; desc file('02977_1.csv'); select * from file('02977_1.csv') settings max_threads=1; diff --git a/tests/queries/0_stateless/02995_baseline_23_12_1.tsv b/tests/queries/0_stateless/02995_baseline_23_12_1.tsv index 4c0c9125b46..a391473e7c9 100644 --- a/tests/queries/0_stateless/02995_baseline_23_12_1.tsv +++ b/tests/queries/0_stateless/02995_baseline_23_12_1.tsv @@ -41,7 +41,7 @@ allow_experimental_query_deduplication 0 allow_experimental_refreshable_materialized_view 0 allow_experimental_s3queue 1 allow_experimental_shared_merge_tree 0 -allow_experimental_statistic 0 +allow_experimental_statistics 0 allow_experimental_undrop_table_query 1 allow_experimental_usearch_index 0 allow_experimental_window_functions 1 @@ -58,7 +58,7 @@ allow_prefetched_read_pool_for_remote_filesystem 1 allow_push_predicate_when_subquery_contains_with 1 allow_settings_after_format_in_insert 0 allow_simdjson 1 -allow_statistic_optimize 0 +allow_statistics_optimize 0 allow_suspicious_codecs 0 allow_suspicious_fixed_string_types 0 allow_suspicious_indices 0 diff --git a/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql b/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql index 5dc69488cea..46d6bb657c9 100644 --- a/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql +++ b/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 1_000_000 as a FORMAT Pretty; SELECT 1_000_000 as a FORMAT PrettyNoEscapes; SELECT 1_000_000 as a FORMAT PrettyMonoBlock; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference new file mode 100644 index 00000000000..bab1fbd050f --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference @@ -0,0 +1,290 @@ +-- { echoOn } +-- inequality operation +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- BETWEEN +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- Stupid condition +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- Window functions with stupid condition +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 15 3 +2 15 beta 2 10 beta 15 3 +3 20 gamma 0 0 15 3 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 15 3 +2 15 beta 2 10 beta 15 3 +3 20 gamma 0 0 15 3 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 12.5 3 +2 15 beta 2 10 beta 12.5 3 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 12.5 3 +2 15 beta 2 10 beta 12.5 3 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 8.333333333333334 7 +1 10 alpha 1 5 ALPHA 8.333333333333334 7 +2 15 beta 2 10 beta 8.333333333333334 7 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 8.333333333333334 7 +1 10 alpha 1 5 ALPHA 8.333333333333334 7 +2 15 beta 2 10 beta 8.333333333333334 7 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 11.25 7 +1 10 alpha 1 5 ALPHA 11.25 7 +2 15 beta 2 10 beta 11.25 7 +3 20 gamma 0 0 11.25 7 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 11.25 7 +1 10 alpha 1 5 ALPHA 11.25 7 +2 15 beta 2 10 beta 11.25 7 +3 20 gamma 0 0 11.25 7 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 new file mode 100644 index 00000000000..f15fced161c --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 @@ -0,0 +1,82 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma'); +INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); + + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_use_nulls=0; +-- { echoOn } +-- inequality operation +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- BETWEEN +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- Stupid condition +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- Window functions with stupid condition +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference new file mode 100644 index 00000000000..1ec6f911897 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference @@ -0,0 +1,90 @@ +-- { echoOn } + +-- Support for query lower +SET join_algorithm='hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- Subquery JOIN +SET join_algorithm='hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +0 0 2 10 beta 0 0 +0 0 4 25 delta 0 0 +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +0 0 2 10 beta 0 0 +0 0 4 25 delta 0 0 +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 new file mode 100644 index 00000000000..a97153ce3aa --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 @@ -0,0 +1,44 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma'); +INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); + + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_use_nulls=0; +-- { echoOn } + +-- Support for query lower +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + + +-- Subquery JOIN +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) {{ join_type }} JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference new file mode 100644 index 00000000000..0c747d04c58 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference @@ -0,0 +1,163 @@ +-- { echoOn } + +-- These queries work +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 new file mode 100644 index 00000000000..3235019821b --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 @@ -0,0 +1,51 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32 +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32 +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a) VALUES (1, 10), (2, 15), (3, 20); +INSERT INTO t2 (key, a) VALUES (1, 5), (2, 10), (4, 25); + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_algorithm='hash'; +-- { echoOn } + +-- These queries work +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 806596f8a63..46f24f73356 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -38,6 +38,17 @@ key4 f 2 3 4 0 0 \N SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 key1 a 1 1 2 key1 B 2 1 2 @@ -67,6 +78,16 @@ key1 c 3 2 1 key1 D 4 1 6 SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key4 f 2 3 4 key4 F 1 1 1 SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); 0 0 \N key3 a3 1 1 1 key1 a 1 1 2 key1 A 1 2 1 @@ -102,6 +123,17 @@ key1 c 3 2 1 key1 D 4 1 6 SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key4 f 2 3 4 key4 F 1 1 1 SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); 0 0 \N key3 a3 1 1 1 key1 a 1 1 2 key1 A 1 2 1 @@ -146,6 +178,18 @@ key4 f 2 3 4 0 0 \N SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 FULL JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 SET join_algorithm='grace_hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 @@ -185,6 +229,17 @@ key4 f 2 3 4 0 0 \N SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 key1 a 1 1 2 key1 B 2 1 2 @@ -214,6 +269,16 @@ key1 c 3 2 1 key1 D 4 1 6 SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key4 f 2 3 4 key4 F 1 1 1 SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); 0 0 \N key3 a3 1 1 1 key1 a 1 1 2 key1 A 1 2 1 @@ -249,6 +314,17 @@ key1 c 3 2 1 key1 D 4 1 6 SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key4 f 2 3 4 key4 F 1 1 1 SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); 0 0 \N key3 a3 1 1 1 key1 a 1 1 2 key1 A 1 2 1 @@ -293,6 +369,18 @@ key4 f 2 3 4 0 0 \N SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 FULL JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); 1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 C 3 4 5 +key1 b 2 3 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 C 3 4 5 +key1 e 5 5 5 key1 C 3 4 5 +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index d3aa74f5c38..a363101ca69 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -18,6 +18,7 @@ SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.b + SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 3))) ORDER BY ALL; {% endfor -%} {% endfor -%} diff --git a/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference new file mode 100644 index 00000000000..f83d884fd78 --- /dev/null +++ b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference @@ -0,0 +1 @@ +Syntax error diff --git a/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh new file mode 100755 index 00000000000..762201ed5fc --- /dev/null +++ b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --query "SELECT((((((((((SELECT(((((((((SELECT((((((((((SELECT(((((((((SELECT((((((((((SELECT(((((((((SELECT 1+)))))))))))))))))))))))))))))))))))))))))))))))))))))))))'" 2>&1 | grep -o -F 'Syntax error' diff --git a/tests/queries/0_stateless/03022_highlight_digit_groups.sql b/tests/queries/0_stateless/03022_highlight_digit_groups.sql index c48a02e712f..8c371c409c4 100644 --- a/tests/queries/0_stateless/03022_highlight_digit_groups.sql +++ b/tests/queries/0_stateless/03022_highlight_digit_groups.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SET output_format_pretty_row_numbers = 0; SELECT exp10(number) * (number % 2 ? 1 : -1) FROM numbers(30) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1; diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect deleted file mode 100755 index de15a199132..00000000000 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect +++ /dev/null @@ -1,49 +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 60 -match_max 100000 -set stty_init "rows 25 cols 120" - -expect_after { - -i $any_spawn_id eof { exp_continue } - -i $any_spawn_id timeout { exit 1 } -} - -spawn clickhouse-local -expect ":) " - -# Trivial SELECT with LIMIT from system.zeros shows progress bar. -send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -# As well as from generateRandom -send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "exit\r" -expect eof diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference index e69de29bb2d..6ca5ae94f9a 100644 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference @@ -0,0 +1,3 @@ +Matched +Matched +Matched diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh new file mode 100755 index 00000000000..500a12587a2 --- /dev/null +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function run_with_progress_and_match_total_rows() +{ + CURL_RESPONSE=$(echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + + echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" +} + +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' +run_with_progress_and_match_total_rows "SELECT * FROM generateRandom('number UInt64') LIMIT 100" diff --git a/tests/queries/0_stateless/03034_ddls_and_merges_with_unusual_maps.reference b/tests/queries/0_stateless/03034_ddls_and_merges_with_unusual_maps.reference new file mode 100644 index 00000000000..9dc0605fd5a --- /dev/null +++ b/tests/queries/0_stateless/03034_ddls_and_merges_with_unusual_maps.reference @@ -0,0 +1,8 @@ +Map(Nothing, ...) is non-comparable --> not usable as primary key +But Map(Nothing, ...) can be a non-primary-key, it is quite useless though ... +Map(Float32, ...) and Map(LC(String)) are okay as primary key +{1:'a'} {'b':'b'} +{2:'aa'} {'bb':'bb'} +Map(Float32, ...) and Map(LC(String)) as non-primary-key +{1:'a'} {'b':'b'} +{3:'aaa'} {'bb':'bb'} diff --git a/tests/queries/0_stateless/03034_ddls_and_merges_with_unusual_maps.sql b/tests/queries/0_stateless/03034_ddls_and_merges_with_unusual_maps.sql new file mode 100644 index 00000000000..a3cd59df1cd --- /dev/null +++ b/tests/queries/0_stateless/03034_ddls_and_merges_with_unusual_maps.sql @@ -0,0 +1,33 @@ +-- Tests maps with "unusual" key types (Float32, Nothing, LowCardinality(String)) + +SET mutations_sync = 2; + +DROP TABLE IF EXISTS tab; + +SELECT 'Map(Nothing, ...) is non-comparable --> not usable as primary key'; +CREATE TABLE tab (m1 Map(Nothing, String)) ENGINE = MergeTree ORDER BY m1; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } + +SELECT 'But Map(Nothing, ...) can be a non-primary-key, it is quite useless though ...'; +CREATE TABLE tab (m3 Map(Nothing, String)) ENGINE = MergeTree ORDER BY tuple(); +-- INSERT INTO tab VALUES (map('', 'd')); -- { serverError NOT_IMPLEMENTED } -- The client can't serialize the data and fails. The query + -- doesn't reach the server and we can't check via 'serverError' :-/ +DROP TABLE tab; + +SELECT 'Map(Float32, ...) and Map(LC(String)) are okay as primary key'; +CREATE TABLE tab (m1 Map(Float32, String), m2 Map(LowCardinality(String), String)) ENGINE = MergeTree ORDER BY (m1, m2); +INSERT INTO tab VALUES (map(1.0, 'a'), map('b', 'b')); +INSERT INTO tab VALUES (map(2.0, 'aa'), map('bb', 'bb')); + +-- Test merge +OPTIMIZE TABLE tab FINAL; +SELECT * FROM tab ORDER BY m1, m2; + +DROP TABLE tab; + +SELECT 'Map(Float32, ...) and Map(LC(String)) as non-primary-key'; +CREATE TABLE tab (m1 Map(Float32, String), m2 Map(LowCardinality(String), String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO tab VALUES (map(1.0, 'a'), map('b', 'b')), (map(2.0, 'aa'), map('bb', 'bb')); +ALTER TABLE tab UPDATE m1 = map(3.0, 'aaa') WHERE m1 = map(2.0, 'aa'); +SELECT * FROM tab ORDER BY m1, m2; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 00740e6380f..80f4e309505 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -33,10 +33,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -50,10 +50,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -100,10 +100,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -117,10 +117,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -168,12 +168,12 @@ Positions: 4 2 0 1 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3 - ALIAS id :: 0 -> __table1.id UInt64 : 4 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 4, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 - FUNCTION equals(__table1.id : 4, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 + ALIAS id : 0 -> __table1.id UInt64 : 4 + ALIAS value :: 1 -> __table1.value String : 5 + FUNCTION equals(id : 0, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 + FUNCTION equals(id :: 0, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 FUNCTION and(equals(__table1.id, 5_UInt8) :: 2, equals(__table1.id, 6_UInt8) :: 1) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 3 - Positions: 3 4 0 + Positions: 3 4 5 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -188,12 +188,12 @@ Positions: 4 2 0 1 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3 - ALIAS id :: 0 -> __table2.id UInt64 : 4 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 4, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - FUNCTION equals(__table2.id : 4, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 + ALIAS id : 0 -> __table2.id UInt64 : 4 + ALIAS value :: 1 -> __table2.value String : 5 + FUNCTION equals(id : 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + FUNCTION equals(id :: 0, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 FUNCTION and(equals(__table2.id, 6_UInt8) :: 2, equals(__table2.id, 5_UInt8) :: 1) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 3 - Positions: 3 4 0 + Positions: 3 4 5 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -237,10 +237,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -254,10 +254,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -452,10 +452,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -469,10 +469,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 7c1ac41cfdc..887b2ed94d7 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 927ceac72b5..371ae87c2ef 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh b/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh index ca313307a6d..60248f4453a 100755 --- a/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh +++ b/tests/queries/0_stateless/03039_dynamic_versioned_collapsing_merge_tree.sh @@ -7,6 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh + # Fix some settings to avoid timeouts because of some settings randomization CH_CLIENT="$CLICKHOUSE_CLIENT --allow_merge_tree_settings --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --index_granularity 8128 --merge_max_block_size 8128" @@ -32,7 +33,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index 6ca5b6ef572..d35bdeff98b 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -25,13 +25,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -42,20 +41,18 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -66,9 +63,8 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 -- 2 Value_2 2 Value_2 -- @@ -99,13 +95,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -116,20 +111,18 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -140,9 +133,8 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 -- 2 Value_2 2 Value_2 -- @@ -173,13 +165,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -190,22 +181,20 @@ Positions: 4 0 2 1 Prewhere filter column: and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 4 - FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 4, notEquals(__table1.id, 0_UInt8) :: 3) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1 - Positions: 2 0 1 + FUNCTION notEquals(id : 0, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 + FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 3, notEquals(__table1.id, 0_UInt8) :: 2) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1 + Positions: 0 1 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -216,10 +205,9 @@ Positions: 4 0 2 1 Prewhere filter column: and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 4 - FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 4, notEquals(__table2.id, 0_UInt8) :: 3) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1 - Positions: 2 0 1 + FUNCTION notEquals(id : 0, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 + FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 3, notEquals(__table2.id, 0_UInt8) :: 2) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1 + Positions: 0 1 -- 2 Value_2 2 Value_2 diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference new file mode 100644 index 00000000000..9b93c75ea56 --- /dev/null +++ b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.reference @@ -0,0 +1 @@ +CREATE VIEW default.test_table_comment AS (SELECT toString({date_from:String})) COMMENT \'test comment\' diff --git a/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql new file mode 100644 index 00000000000..98318e99e4a --- /dev/null +++ b/tests/queries/0_stateless/03142_alter_comment_parameterized_view.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS test_table_comment; +CREATE VIEW test_table_comment AS SELECT toString({date_from:String}); +ALTER TABLE test_table_comment MODIFY COMMENT 'test comment'; +SELECT create_table_query FROM system.tables WHERE name = 'test_table_comment' AND database = currentDatabase(); +DROP TABLE test_table_comment; diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference index 864f62d3113..6d375fd471a 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.reference @@ -1,5 +1,10 @@ ┏━━━┓ ┃ x ┃ ┡━━━┩ -1. │ █ │ +1. │ █ │ └───┘ + ┏━━━━━━━━━┳━━━━━━━━━━┓ + ┃ 'Hello' ┃ x ┃ + ┡━━━━━━━━━╇━━━━━━━━━━┩ +1. │ Hello │ █ test █ │ + └─────────┴──────────┘ diff --git a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql index e37b0db08e9..49f689a4cc5 100644 --- a/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql +++ b/tests/queries/0_stateless/03142_skip_ANSI_in_UTF8_compute_width.sql @@ -1 +1,2 @@ -SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 0) AS x FORMAT Pretty; +SELECT format('\x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x FORMAT Pretty; +SELECT 'Hello', format('\x1b[38;2;{0};{1};{2}m█\x1b[0m test \x1b[38;2;{0};{1};{2}m█\x1b[0m', 255, 128, 128) AS x FORMAT Pretty; diff --git a/tests/queries/0_stateless/03143_prewhere_profile_events.sh b/tests/queries/0_stateless/03143_prewhere_profile_events.sh index 863fcc1fe01..00daa0fe7cc 100755 --- a/tests/queries/0_stateless/03143_prewhere_profile_events.sh +++ b/tests/queries/0_stateless/03143_prewhere_profile_events.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-merge-tree-settings +# Tags: no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -nq " DROP TABLE IF EXISTS t; - CREATE TABLE t(a UInt32, b UInt32, c UInt32, d UInt32) ENGINE=MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part=1, min_rows_for_wide_part=1; + CREATE TABLE t(a UInt32, b UInt32, c UInt32, d UInt32) ENGINE=MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=0; INSERT INTO t SELECT number, number, number, number FROM numbers_mt(1e7); diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.reference b/tests/queries/0_stateless/03146_parameterized_view_with_date.reference new file mode 100644 index 00000000000..1d6227dbbcb --- /dev/null +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.reference @@ -0,0 +1 @@ +2 2024-04-01 01:00:00 diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql new file mode 100644 index 00000000000..2cfadb70b24 --- /dev/null +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql @@ -0,0 +1,14 @@ + +drop table if exists table_pv; +create table table_pv (id Int32, timestamp_field DateTime) engine = Memory(); + +insert into table_pv values(1, '2024-03-01 00:00:00'); +insert into table_pv values (2, '2024-04-01 01:00:00'); + +create view pv as select * from table_pv where timestamp_field > {timestamp_param:DateTime}; + +select * from pv (timestamp_param=toDateTime('2024-04-01 00:00:01')); + +select * from pv (timestamp_param=toDateTime('2024-040')); -- { serverError CANNOT_PARSE_DATETIME } + +drop table table_pv; diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.reference b/tests/queries/0_stateless/03155_analyzer_interpolate.reference index 791aaa5b2a2..eade3b45d26 100644 --- a/tests/queries/0_stateless/03155_analyzer_interpolate.reference +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.reference @@ -11,3 +11,8 @@ 5 [5] 5.5 [5] 7 [7] +2 +100500 +18 +26 +34 diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.sql b/tests/queries/0_stateless/03155_analyzer_interpolate.sql index b3c1d233f47..30423cb86ff 100644 --- a/tests/queries/0_stateless/03155_analyzer_interpolate.sql +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.sql @@ -10,3 +10,6 @@ SELECT n, number+5 AS inter FROM ( -- { serverError NOT_AN_AGGREGATE } SELECT toFloat32(number % 10) AS n, number, number*2 AS mn FROM numbers(10) WHERE number % 3 = 1 ) GROUP BY n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS mn * 2); + +-- https://github.com/ClickHouse/ClickHouse/issues/64636 +select sum(number) as s from remote('127.0.0.{1,2}', numbers(10)) where (intDiv(number, 2) as key) != 1 group by key order by key with fill interpolate (s as 100500); diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.reference b/tests/queries/0_stateless/03155_test_move_to_prewhere.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh new file mode 100755 index 00000000000..b6980b3a23a --- /dev/null +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -nq " + CREATE TABLE event_envoy + ( + timestamp_interval DateTime CODEC(DoubleDelta), + region LowCardinality(String), + cluster LowCardinality(String) + ) + ENGINE = MergeTree + ORDER BY (timestamp_interval) + SETTINGS index_granularity = 8192; + + INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5); +" + +${CLICKHOUSE_CLIENT} -nq " + CREATE TABLE event_envoy_remote + ( + timestamp_interval DateTime CODEC(DoubleDelta), + region LowCardinality(String), + cluster LowCardinality(String) + ) AS remote('127.0.0.1', '${CLICKHOUSE_DATABASE}', event_envoy); +" + +${CLICKHOUSE_CLIENT} -q " + CREATE TABLE global_event_envoy + ( + timestamp_interval DateTime, + region LowCardinality(String), + cluster LowCardinality(String) + ) + ENGINE = Merge('${CLICKHOUSE_DATABASE}', 'event_envoy.*'); +" + +${CLICKHOUSE_CLIENT} --prefer_localhost_replica 1 -q " + EXPLAIN indexes=1 + SELECT timestamp_interval + FROM global_event_envoy + WHERE timestamp_interval <= now() - 54321 AND region = 'us-east-1' +" | grep -c 'Condition.*timestamp_interval' + diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference index b5b2aec9c12..18830a293bd 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.reference @@ -10,3 +10,5 @@ Hello 1 Hello 1 Hello 2 Hello 2 +2020-01-01 a 2 +2020-01-01 b 4 diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql index f605a369822..55f9877b2ac 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql @@ -8,3 +8,21 @@ SELECT s, arr, a FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) AR SELECT s, arr FROM remote('127.0.0.2', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; SELECT s, arr FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; + +create table hourly( + hour datetime, + `metric.names` Array(String), + `metric.values` Array(Int64) +) Engine=Memory +as select '2020-01-01', ['a', 'b'], [1,2]; + +SELECT + toDate(hour) AS day, + `metric.names`, + sum(`metric.values`) +FROM remote('127.0.0.{1,2}', currentDatabase(), hourly) +ARRAY JOIN metric +GROUP BY + day, + metric.names +ORDER BY metric.names; diff --git a/tests/queries/0_stateless/03156_group_concat.reference b/tests/queries/0_stateless/03156_group_concat.reference new file mode 100644 index 00000000000..75b347be0c4 --- /dev/null +++ b/tests/queries/0_stateless/03156_group_concat.reference @@ -0,0 +1,14 @@ +0 95 abc [1,2,3] +1 \N a [993,986,979,972] +2 123 makson95 [] +95123 +abcamakson95 +[1,2,3][993,986,979,972][] +95,123 +abc,a,makson95 +[1,2,3],[993,986,979,972] +\N +951239512395123 +abc,a,makson95,abc,a,makson95,abc,a,makson95 +[1,2,3][993,986,979,972][][1,2,3][993,986,979,972][][1,2,3][993,986,979,972][] +488890 diff --git a/tests/queries/0_stateless/03156_group_concat.sql b/tests/queries/0_stateless/03156_group_concat.sql new file mode 100644 index 00000000000..c14fde8943a --- /dev/null +++ b/tests/queries/0_stateless/03156_group_concat.sql @@ -0,0 +1,40 @@ +DROP TABLE IF EXISTS test_groupConcat; +CREATE TABLE test_groupConcat +( + id UInt64, + p_int Int32 NULL, + p_string String, + p_array Array(Int32) +) ENGINE = MergeTree ORDER BY id; + +SET max_insert_threads = 1, max_threads = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; +INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []); + +SELECT * FROM test_groupConcat; + +SELECT groupConcat(p_int) FROM test_groupConcat; +SELECT groupConcat(p_string) FROM test_groupConcat; +SELECT groupConcat(p_array) FROM test_groupConcat; + +SELECT groupConcat(',')(p_int) FROM test_groupConcat; +SELECT groupConcat(',')(p_string) FROM test_groupConcat; +SELECT groupConcat(',', 2)(p_array) FROM test_groupConcat; + +SELECT groupConcat(p_int) FROM test_groupConcat WHERE id = 1; + +INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []); +INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []); + +SELECT groupConcat(p_int) FROM test_groupConcat; +SELECT groupConcat(',')(p_string) FROM test_groupConcat; +SELECT groupConcat(p_array) FROM test_groupConcat; + +SELECT groupConcat(123)(number) FROM numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT groupConcat(',', '3')(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS } +SELECT groupConcat(',', 0)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS } +SELECT groupConcat(',', -1)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS } +SELECT groupConcat(',', 3, 3)(number) FROM numbers(10); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION } + +SELECT length(groupConcat(number)) FROM numbers(100000); + +DROP TABLE IF EXISTS test_groupConcat; diff --git a/tests/queries/0_stateless/03156_nullable_number_tips.sql b/tests/queries/0_stateless/03156_nullable_number_tips.sql index e6f2fa36d86..9a494e3292b 100644 --- a/tests/queries/0_stateless/03156_nullable_number_tips.sql +++ b/tests/queries/0_stateless/03156_nullable_number_tips.sql @@ -1,3 +1,4 @@ +SET output_format_pretty_display_footer_column_names=0; SELECT 123456789 AS x FORMAT PrettyCompact; SELECT toNullable(123456789) AS x FORMAT PrettyCompact; SELECT toLowCardinality(toNullable(123456789)) AS x FORMAT PrettyCompact; diff --git a/tests/queries/0_stateless/03160_pretty_format_tty.sh b/tests/queries/0_stateless/03160_pretty_format_tty.sh index bbc4b96eb90..200bd52f3fa 100755 --- a/tests/queries/0_stateless/03160_pretty_format_tty.sh +++ b/tests/queries/0_stateless/03160_pretty_format_tty.sh @@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # default output_format_pretty_max_rows is 10K -$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1" | wc -l +$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1, output_format_pretty_display_footer_column_names=0" | wc -l diff --git a/tests/queries/0_stateless/03161_decimal_binary_math.reference b/tests/queries/0_stateless/03161_decimal_binary_math.reference new file mode 100644 index 00000000000..f7d9761c7c5 --- /dev/null +++ b/tests/queries/0_stateless/03161_decimal_binary_math.reference @@ -0,0 +1,75 @@ +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +42.4242 2.42 8686.104718 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +0.4242 0.24 0.514871 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 2.42 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +42.4242 2.42 42.4242 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 +0.4242 0.4242 0.599909 diff --git a/tests/queries/0_stateless/03161_decimal_binary_math.sql b/tests/queries/0_stateless/03161_decimal_binary_math.sql new file mode 100644 index 00000000000..5484cc6a9bb --- /dev/null +++ b/tests/queries/0_stateless/03161_decimal_binary_math.sql @@ -0,0 +1,79 @@ +SELECT toDecimal32('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(pow(x, y), 6); +SELECT toDecimal64('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(pow(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, toDecimal64('2.42', 2) AS y, round(pow(x, y), 6); +SELECT toDecimal64('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(pow(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(pow(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, toDecimal32('2.42', 2) AS y, round(pow(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(pow(x, y), 6); +SELECT 42.4242 AS x, toDecimal32('2.42', 2) AS y, round(pow(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, 2.42 AS y, round(pow(x, y), 6); +SELECT materialize(42.4242) AS x, toDecimal32('2.42', 2) AS y, round(pow(x, y), 6); +SELECT 42.4242 AS x, materialize(toDecimal32('2.42', 2)) AS y, round(pow(x, y), 6); +SELECT materialize(42.4242) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(pow(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, 2.42 AS y, round(pow(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, materialize(2.42) AS y, round(pow(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, materialize(2.42) AS y, round(pow(x, y), 6); + +SELECT toDecimal32('0.4242', 4) AS x, toDecimal32('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT toDecimal64('0.4242', 4) AS x, toDecimal32('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT toDecimal32('0.4242', 4) AS x, toDecimal64('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT toDecimal64('0.4242', 4) AS x, toDecimal64('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT toDecimal32('0.4242', 4) AS x, materialize(toDecimal32('0.24', 2)) AS y, round(atan2(y, x), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, toDecimal32('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, materialize(toDecimal32('0.24', 2)) AS y, round(atan2(y, x), 6); +SELECT 0.4242 AS x, toDecimal32('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT toDecimal32('0.4242', 4) AS x, 0.24 AS y, round(atan2(y, x), 6); +SELECT materialize(0.4242) AS x, toDecimal32('0.24', 2) AS y, round(atan2(y, x), 6); +SELECT 0.4242 AS x, materialize(toDecimal32('0.24', 2)) AS y, round(atan2(y, x), 6); +SELECT materialize(0.4242) AS x, materialize(toDecimal32('0.24', 2)) AS y, round(atan2(y, x), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, 0.24 AS y, round(atan2(y, x), 6); +SELECT toDecimal32('0.4242', 4) AS x, materialize(0.24) AS y, round(atan2(y, x), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, materialize(0.24) AS y, round(atan2(y, x), 6); + +SELECT toDecimal32('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(min2(x, y), 6); +SELECT toDecimal64('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(min2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, toDecimal64('2.42', 2) AS y, round(min2(x, y), 6); +SELECT toDecimal64('42.4242', 4) AS x, toDecimal64('2.42', 2) AS y, round(min2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(min2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, toDecimal32('2.42', 2) AS y, round(min2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(min2(x, y), 6); +SELECT 42.4242 AS x, toDecimal32('2.42', 2) AS y, round(min2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, 2.42 AS y, round(min2(x, y), 6); +SELECT materialize(42.4242) AS x, toDecimal32('2.42', 2) AS y, round(min2(x, y), 6); +SELECT 42.4242 AS x, materialize(toDecimal32('2.42', 2)) AS y, round(min2(x, y), 6); +SELECT materialize(42.4242) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(min2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, 2.42 AS y, round(min2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, materialize(2.42) AS y, round(min2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, materialize(2.42) AS y, round(min2(x, y), 6); + +SELECT toDecimal32('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(max2(x, y), 6); +SELECT toDecimal64('42.4242', 4) AS x, toDecimal32('2.42', 2) AS y, round(max2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, toDecimal64('2.42', 2) AS y, round(max2(x, y), 6); +SELECT toDecimal64('42.4242', 4) AS x, toDecimal64('2.42', 2) AS y, round(max2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(max2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, toDecimal32('2.42', 2) AS y, round(max2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(max2(x, y), 6); +SELECT 42.4242 AS x, toDecimal32('2.42', 2) AS y, round(max2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, 2.42 AS y, round(max2(x, y), 6); +SELECT materialize(42.4242) AS x, toDecimal32('2.42', 2) AS y, round(max2(x, y), 6); +SELECT 42.4242 AS x, materialize(toDecimal32('2.42', 2)) AS y, round(max2(x, y), 6); +SELECT materialize(42.4242) AS x, materialize(toDecimal32('2.42', 2)) AS y, round(max2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, 2.42 AS y, round(max2(x, y), 6); +SELECT toDecimal32('42.4242', 4) AS x, materialize(2.42) AS y, round(max2(x, y), 6); +SELECT materialize(toDecimal32('42.4242', 4)) AS x, materialize(2.42) AS y, round(max2(x, y), 6); + +SELECT toDecimal32('0.4242', 4) AS x, toDecimal32('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT toDecimal64('0.4242', 4) AS x, toDecimal32('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT toDecimal32('0.4242', 4) AS x, toDecimal64('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT toDecimal64('0.4242', 4) AS x, toDecimal64('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT toDecimal32('0.4242', 4) AS x, materialize(toDecimal32('0.4242', 4)) AS y, round(hypot(x, y), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, toDecimal32('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, materialize(toDecimal32('0.4242', 4)) AS y, round(hypot(x, y), 6); +SELECT 0.4242 AS x, toDecimal32('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT toDecimal32('0.4242', 4) AS x, 0.4242 AS y, round(hypot(x, y), 6); +SELECT materialize(0.4242) AS x, toDecimal32('0.4242', 4) AS y, round(hypot(x, y), 6); +SELECT 0.4242 AS x, materialize(toDecimal32('0.4242', 4)) AS y, round(hypot(x, y), 6); +SELECT materialize(0.4242) AS x, materialize(toDecimal32('0.4242', 4)) AS y, round(hypot(x, y), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, 0.4242 AS y, round(hypot(x, y), 6); +SELECT toDecimal32('0.4242', 4) AS x, materialize(0.4242) AS y, round(hypot(x, y), 6); +SELECT materialize(toDecimal32('0.4242', 4)) AS x, materialize(0.4242) AS y, round(hypot(x, y), 6); diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference new file mode 100644 index 00000000000..ef9b07ba955 --- /dev/null +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.reference @@ -0,0 +1,4 @@ +65409 +16 +128 +2363 diff --git a/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql new file mode 100644 index 00000000000..fa098b64702 --- /dev/null +++ b/tests/queries/0_stateless/03164_adapting_parquet_reader_output_size.sql @@ -0,0 +1,25 @@ +-- Tags: no-fasttest, no-parallel, no-random-settings + +set max_insert_threads=1; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet); +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_max_block_size=16; +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_prefer_block_bytes=30; +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; +CREATE TABLE test_parquet (col1 String, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=File(Parquet) settings input_format_parquet_prefer_block_bytes=30720; +INSERT INTO test_parquet SELECT rand(),rand(),rand(),rand(),rand(),rand(),rand() FROM numbers(100000); +SELECT max(blockSize()) FROM test_parquet; + +DROP TABLE IF EXISTS test_parquet; \ No newline at end of file diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference new file mode 100644 index 00000000000..227b118bb7f --- /dev/null +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.reference @@ -0,0 +1 @@ +ReadFromPreparedSource (Optimized trivial count) diff --git a/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql new file mode 100644 index 00000000000..dbffbc1af71 --- /dev/null +++ b/tests/queries/0_stateless/03164_early_constant_folding_analyzer.sql @@ -0,0 +1,30 @@ +CREATE TABLE checks +( + `pull_request_number` UInt32, + `commit_sha` LowCardinality(String), + `check_name` LowCardinality(String), + `check_status` LowCardinality(String), + `check_duration_ms` UInt64, + `check_start_time` DateTime, + `test_name` LowCardinality(String), + `test_status` LowCardinality(String), + `test_duration_ms` UInt64, + `report_url` String, + `pull_request_url` String, + `commit_url` String, + `task_url` String, + `base_ref` String, + `base_repo` String, + `head_ref` String, + `head_repo` String, + `test_context_raw` String, + `instance_type` LowCardinality(String), + `instance_id` String, + `date` Date MATERIALIZED toDate(check_start_time) +) +ENGINE = MergeTree ORDER BY (date, pull_request_number, commit_sha, check_name, test_name, check_start_time); + +insert into checks select * from generateRandom() limit 1; + + +select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL) where explain like '%ReadFromPreparedSource%' SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0; diff --git a/tests/queries/0_stateless/03164_linestring_geometry.reference b/tests/queries/0_stateless/03164_linestring_geometry.reference new file mode 100644 index 00000000000..1f68df04614 --- /dev/null +++ b/tests/queries/0_stateless/03164_linestring_geometry.reference @@ -0,0 +1,11 @@ +-- { echoOn } +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +[(1,1),(2,2),(3,3),(1,1)] +SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +LineString +SELECT wkt(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +LINESTRING(1 1,2 2,3 3,1 1) +-- Native Array(Tuple(Float64, Float64)) is threated as Ring, not as LineString. +WITH wkt(CAST([(1, 1), (2, 2), (3, 3)], 'Array(Tuple(Float64, Float64))')) as x +SELECT x, toTypeName(x), readWKTRing(x) as y, toTypeName(y); +POLYGON((1 1,2 2,3 3)) String [(1,1),(2,2),(3,3)] Ring diff --git a/tests/queries/0_stateless/03164_linestring_geometry.sql b/tests/queries/0_stateless/03164_linestring_geometry.sql new file mode 100644 index 00000000000..e4f1d1295e7 --- /dev/null +++ b/tests/queries/0_stateless/03164_linestring_geometry.sql @@ -0,0 +1,8 @@ +-- { echoOn } +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +SELECT wkt(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); + +-- Native Array(Tuple(Float64, Float64)) is threated as Ring, not as LineString. +WITH wkt(CAST([(1, 1), (2, 2), (3, 3)], 'Array(Tuple(Float64, Float64))')) as x +SELECT x, toTypeName(x), readWKTRing(x) as y, toTypeName(y); diff --git a/tests/queries/0_stateless/03164_materialize_statistics.reference b/tests/queries/0_stateless/03164_materialize_statistics.reference index c209d2e8b63..5e969cf41cb 100644 --- a/tests/queries/0_stateless/03164_materialize_statistics.reference +++ b/tests/queries/0_stateless/03164_materialize_statistics.reference @@ -1,10 +1,10 @@ 10 10 10 -statistic not used Condition less(b, 10_UInt8) moved to PREWHERE -statistic not used Condition less(a, 10_UInt8) moved to PREWHERE -statistic used after merge Condition less(a, 10_UInt8) moved to PREWHERE -statistic used after merge Condition less(b, 10_UInt8) moved to PREWHERE -statistic used after materialize Condition less(a, 10_UInt8) moved to PREWHERE -statistic used after materialize Condition less(b, 10_UInt8) moved to PREWHERE +statistics not used Condition less(b, 10_UInt8) moved to PREWHERE +statistics not used Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE +statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE 2 0 diff --git a/tests/queries/0_stateless/03164_materialize_statistics.sql b/tests/queries/0_stateless/03164_materialize_statistics.sql index 763644d16ab..43c5724dd59 100644 --- a/tests/queries/0_stateless/03164_materialize_statistics.sql +++ b/tests/queries/0_stateless/03164_materialize_statistics.sql @@ -1,34 +1,34 @@ -DROP TABLE IF EXISTS t_statistic_materialize; +DROP TABLE IF EXISTS t_statistics_materialize; SET allow_experimental_analyzer = 1; -SET allow_experimental_statistic = 1; -SET allow_statistic_optimize = 1; +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; SET materialize_statistics_on_insert = 0; -CREATE TABLE t_statistic_materialize +CREATE TABLE t_statistics_materialize ( - a Int64 STATISTIC(tdigest), - b Int16 STATISTIC(tdigest), + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), ) ENGINE = MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. -INSERT INTO t_statistic_materialize SELECT number, -number FROM system.numbers LIMIT 10000; +INSERT INTO t_statistics_materialize SELECT number, -number FROM system.numbers LIMIT 10000; -SELECT count(*) FROM t_statistic_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistic not used'; +SELECT count(*) FROM t_statistics_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; -OPTIMIZE TABLE t_statistic_materialize FINAL; +OPTIMIZE TABLE t_statistics_materialize FINAL; -SELECT count(*) FROM t_statistic_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistic used after merge'; +SELECT count(*) FROM t_statistics_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; -TRUNCATE TABLE t_statistic_materialize; +TRUNCATE TABLE t_statistics_materialize; SET mutations_sync = 2; -INSERT INTO t_statistic_materialize SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE t_statistic_materialize MATERIALIZE STATISTIC a, b TYPE tdigest; +INSERT INTO t_statistics_materialize SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE t_statistics_materialize MATERIALIZE STATISTICS a, b; -SELECT count(*) FROM t_statistic_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistic used after materialize'; +SELECT count(*) FROM t_statistics_materialize WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; -DROP TABLE t_statistic_materialize; +DROP TABLE t_statistics_materialize; SYSTEM FLUSH LOGS; @@ -36,7 +36,7 @@ SELECT log_comment, message FROM system.text_log JOIN ( SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log WHERE current_database = currentDatabase() - AND query LIKE 'SELECT count(*) FROM t_statistic_materialize%' + AND query LIKE 'SELECT count(*) FROM t_statistics_materialize%' AND type = 'QueryFinish' ) AS query_log USING (query_id) WHERE message LIKE '%moved to PREWHERE%' @@ -45,5 +45,5 @@ ORDER BY event_time_microseconds; SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) FROM system.query_log WHERE current_database = currentDatabase() - AND query LIKE 'INSERT INTO t_statistic_materialize SELECT%' + AND query LIKE 'INSERT INTO t_statistics_materialize SELECT%' AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/03164_optimize_read_in_order_nullable.reference b/tests/queries/0_stateless/03164_optimize_read_in_order_nullable.reference new file mode 100644 index 00000000000..7e866b496a8 --- /dev/null +++ b/tests/queries/0_stateless/03164_optimize_read_in_order_nullable.reference @@ -0,0 +1,32 @@ +-- Reproducer result: +\N Mark 50 +1 John 33 +2 Ksenia 48 + +-- Read in order, no sort required: +0 0 +1 \N +4 4 +\N 2 +\N \N + +-- Read in order, partial sort for second key: +0 0 +1 \N +4 4 +\N \N +\N 2 + +-- No reading in order, sort for first key: +\N 2 +\N \N +0 0 +1 \N +4 4 + +-- Reverse order, partial sort for the second key: +\N 2 +\N \N +4 4 +1 \N +0 0 diff --git a/tests/queries/0_stateless/03164_optimize_read_in_order_nullable.sql b/tests/queries/0_stateless/03164_optimize_read_in_order_nullable.sql new file mode 100644 index 00000000000..7af6e55bf98 --- /dev/null +++ b/tests/queries/0_stateless/03164_optimize_read_in_order_nullable.sql @@ -0,0 +1,55 @@ +-- Reproducer from https://github.com/ClickHouse/ClickHouse/issues/63460 +DROP TABLE IF EXISTS 03164_users; +CREATE TABLE 03164_users (uid Nullable(Int16), name String, age Int16) ENGINE=MergeTree ORDER BY (uid) SETTINGS allow_nullable_key=1; + +INSERT INTO 03164_users VALUES (1, 'John', 33); +INSERT INTO 03164_users VALUES (2, 'Ksenia', 48); +INSERT INTO 03164_users VALUES (NULL, 'Mark', 50); +OPTIMIZE TABLE 03164_users FINAL; + +SELECT '-- Reproducer result:'; + +SELECT * FROM 03164_users ORDER BY uid ASC NULLS FIRST LIMIT 10 SETTINGS optimize_read_in_order = 1; + +DROP TABLE IF EXISTS 03164_users; + +DROP TABLE IF EXISTS 03164_multi_key; +CREATE TABLE 03164_multi_key (c1 Nullable(UInt32), c2 Nullable(UInt32)) ENGINE = MergeTree ORDER BY (c1, c2) SETTINGS allow_nullable_key=1; + +INSERT INTO 03164_multi_key VALUES (0, 0), (1, NULL), (NULL, 2), (NULL, NULL), (4, 4); +-- Just in case +OPTIMIZE TABLE 03164_multi_key FINAL; + +SELECT ''; +SELECT '-- Read in order, no sort required:'; + +SELECT c1, c2 +FROM 03164_multi_key +ORDER BY c1 ASC NULLS LAST, c2 ASC NULLS LAST +SETTINGS optimize_read_in_order = 1; + +SELECT ''; +SELECT '-- Read in order, partial sort for second key:'; + +SELECT c1, c2 +FROM 03164_multi_key +ORDER BY c1 ASC NULLS LAST, c2 ASC NULLS FIRST +SETTINGS optimize_read_in_order = 1; + +SELECT ''; +SELECT '-- No reading in order, sort for first key:'; + +SELECT c1, c2 +FROM 03164_multi_key +ORDER BY c1 ASC NULLS FIRST, c2 ASC NULLS LAST +SETTINGS optimize_read_in_order = 1; + +SELECT ''; +SELECT '-- Reverse order, partial sort for the second key:'; + +SELECT c1, c2 +FROM 03164_multi_key +ORDER BY c1 DESC NULLS FIRST, c2 DESC NULLS LAST +SETTINGS optimize_read_in_order = 1; + +DROP TABLE IF EXISTS 03164_multi_key; diff --git a/tests/queries/0_stateless/03164_orc_signedness.reference b/tests/queries/0_stateless/03164_orc_signedness.reference new file mode 100644 index 00000000000..3ee822a94c1 --- /dev/null +++ b/tests/queries/0_stateless/03164_orc_signedness.reference @@ -0,0 +1,41 @@ +-- { echoOn } +select x from file('i8.orc') where indexHint(x = -128); +-128 +select x from file('i8.orc') where indexHint(x = 128); +select x from file('u8.orc') where indexHint(x = -128); +-128 +select x from file('u8.orc') where indexHint(x = 128); +select x from file('i16.orc') where indexHint(x = -32768); +-32768 +select x from file('i16.orc') where indexHint(x = 32768); +select x from file('u16.orc') where indexHint(x = -32768); +-32768 +select x from file('u16.orc') where indexHint(x = 32768); +select x from file('i32.orc') where indexHint(x = -2147483648); +-2147483648 +select x from file('i32.orc') where indexHint(x = 2147483648); +select x from file('u32.orc') where indexHint(x = -2147483648); +-2147483648 +select x from file('u32.orc') where indexHint(x = 2147483648); +select x from file('i64.orc') where indexHint(x = -9223372036854775808); +-9223372036854775808 +select x from file('i64.orc') where indexHint(x = 9223372036854775808); +-9223372036854775808 +select x from file('u64.orc') where indexHint(x = -9223372036854775808); +-9223372036854775808 +select x from file('u64.orc') where indexHint(x = 9223372036854775808); +-9223372036854775808 +select x from file('u8.orc', ORC, 'x UInt8') where indexHint(x > 10); +128 +select x from file('u8.orc', ORC, 'x UInt64') where indexHint(x > 10); +18446744073709551488 +select x from file('u16.orc', ORC, 'x UInt16') where indexHint(x > 10); +32768 +select x from file('u16.orc', ORC, 'x UInt64') where indexHint(x > 10); +18446744073709518848 +select x from file('u32.orc', ORC, 'x UInt32') where indexHint(x > 10); +2147483648 +select x from file('u32.orc', ORC, 'x UInt64') where indexHint(x > 10); +18446744071562067968 +select x from file('u64.orc', ORC, 'x UInt64') where indexHint(x > 10); +9223372036854775808 diff --git a/tests/queries/0_stateless/03164_orc_signedness.sql b/tests/queries/0_stateless/03164_orc_signedness.sql new file mode 100644 index 00000000000..ae2d0428ca5 --- /dev/null +++ b/tests/queries/0_stateless/03164_orc_signedness.sql @@ -0,0 +1,42 @@ +-- Tags: no-fasttest, no-parallel + +set input_format_orc_filter_push_down = 1; +set engine_file_truncate_on_insert = 1; + +insert into function file('i8.orc') select materialize(-128)::Int8 as x; +insert into function file('u8.orc') select materialize(128)::UInt8 as x; +insert into function file('i16.orc') select materialize(-32768)::Int16 as x; +insert into function file('u16.orc') select materialize(32768)::UInt16 as x; +insert into function file('i32.orc') select materialize(-2147483648)::Int32 as x; +insert into function file('u32.orc') select materialize(2147483648)::UInt32 as x; +insert into function file('i64.orc') select materialize(-9223372036854775808)::Int64 as x; +insert into function file('u64.orc') select materialize(9223372036854775808)::UInt64 as x; + +-- { echoOn } +select x from file('i8.orc') where indexHint(x = -128); +select x from file('i8.orc') where indexHint(x = 128); +select x from file('u8.orc') where indexHint(x = -128); +select x from file('u8.orc') where indexHint(x = 128); + +select x from file('i16.orc') where indexHint(x = -32768); +select x from file('i16.orc') where indexHint(x = 32768); +select x from file('u16.orc') where indexHint(x = -32768); +select x from file('u16.orc') where indexHint(x = 32768); + +select x from file('i32.orc') where indexHint(x = -2147483648); +select x from file('i32.orc') where indexHint(x = 2147483648); +select x from file('u32.orc') where indexHint(x = -2147483648); +select x from file('u32.orc') where indexHint(x = 2147483648); + +select x from file('i64.orc') where indexHint(x = -9223372036854775808); +select x from file('i64.orc') where indexHint(x = 9223372036854775808); +select x from file('u64.orc') where indexHint(x = -9223372036854775808); +select x from file('u64.orc') where indexHint(x = 9223372036854775808); + +select x from file('u8.orc', ORC, 'x UInt8') where indexHint(x > 10); +select x from file('u8.orc', ORC, 'x UInt64') where indexHint(x > 10); +select x from file('u16.orc', ORC, 'x UInt16') where indexHint(x > 10); +select x from file('u16.orc', ORC, 'x UInt64') where indexHint(x > 10); +select x from file('u32.orc', ORC, 'x UInt32') where indexHint(x > 10); +select x from file('u32.orc', ORC, 'x UInt64') where indexHint(x > 10); +select x from file('u64.orc', ORC, 'x UInt64') where indexHint(x > 10); diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference new file mode 100644 index 00000000000..4dded9eda81 --- /dev/null +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference @@ -0,0 +1,10 @@ +10 +10 +10 +10 +10 +10 +10 +10 +13 +4 diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql new file mode 100644 index 00000000000..58143395e44 --- /dev/null +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -0,0 +1,125 @@ +DROP TABLE IF EXISTS range_filter_custom_range_test; + +CREATE TABLE range_filter_custom_range_test (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper = 15 +); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper = 14 +); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper = 17 +); + + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 15 +); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper = 25 +); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 5 +); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper = 10000 +); + + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 10, parallel_replicas_custom_key_range_upper = 13 +); + +DROP TABLE range_filter_custom_range_test; + +DROP TABLE IF EXISTS range_filter_custom_range_test_2; + +CREATE TABLE range_filter_custom_range_test_2 (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test_2 SELECT number from numbers(13); + +SELECT count() +FROM +( + SELECT * + FROM cluster(parallel_replicas, currentDatabase(), range_filter_custom_range_test_2) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 13 +); + +DROP TABLE range_filter_custom_range_test_2; + +DROP TABLE IF EXISTS range_filter_custom_range_test_3; + +CREATE TABLE range_filter_custom_range_test_3 (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test_3 SELECT number from numbers(4); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test_3) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 4 +); + +DROP TABLE range_filter_custom_range_test_3; \ No newline at end of file diff --git a/tests/queries/0_stateless/03165_order_by_duplicate.reference b/tests/queries/0_stateless/03165_order_by_duplicate.reference new file mode 100644 index 00000000000..5d5e7a33f4a --- /dev/null +++ b/tests/queries/0_stateless/03165_order_by_duplicate.reference @@ -0,0 +1,39 @@ +QUERY id: 0 + PROJECTION COLUMNS + id UInt64 + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test, final: 1 + WHERE + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + QUERY id: 6, is_subquery: 1, is_distinct: 1 + PROJECTION COLUMNS + id UInt64 + PROJECTION + LIST id: 7, nodes: 1 + COLUMN id: 8, column_name: id, result_type: UInt64, source_id: 9 + JOIN TREE + TABLE id: 9, alias: __table1, table_name: default.test, final: 1 + ORDER BY + LIST id: 10, nodes: 1 + SORT id: 11, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + COLUMN id: 8, column_name: id, result_type: UInt64, source_id: 9 + LIMIT + CONSTANT id: 12, constant_value: UInt64_4, constant_value_type: UInt64 + ORDER BY + LIST id: 13, nodes: 1 + SORT id: 14, sort_direction: ASCENDING, with_fill: 0 + EXPRESSION + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + LIMIT BY LIMIT + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt64 + LIMIT BY + LIST id: 16, nodes: 1 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/03165_order_by_duplicate.sql b/tests/queries/0_stateless/03165_order_by_duplicate.sql new file mode 100644 index 00000000000..0054cbc36a6 --- /dev/null +++ b/tests/queries/0_stateless/03165_order_by_duplicate.sql @@ -0,0 +1,16 @@ +CREATE TABLE test +ENGINE = ReplacingMergeTree +PRIMARY KEY id +AS SELECT number AS id FROM numbers(100); + +EXPLAIN QUERY TREE SELECT id +FROM test FINAL +WHERE id IN ( + SELECT DISTINCT id + FROM test FINAL + ORDER BY id ASC + LIMIT 4 +) +ORDER BY id ASC +LIMIT 1 BY id +SETTINGS allow_experimental_analyzer = 1; diff --git a/tests/queries/0_stateless/03165_parseReadableSize.reference b/tests/queries/0_stateless/03165_parseReadableSize.reference new file mode 100644 index 00000000000..57f17ecc5d3 --- /dev/null +++ b/tests/queries/0_stateless/03165_parseReadableSize.reference @@ -0,0 +1,60 @@ +1.00 B +1.00 KiB +1.00 MiB +1.00 GiB +1.00 TiB +1.00 PiB +1.00 EiB +1.00 B +1.00 KB +1.00 MB +1.00 GB +1.00 TB +1.00 PB +1.00 EB +1.00 MiB +1024 +3072 +1024 +1024 +1024 +1024 +1024 +\N +3217 +3217 +1000 +5 +2048 +8192 +0 0 0 +1 B 1 +1 KiB 1024 +1 MiB 1048576 +1 GiB 1073741824 +1 TiB 1099511627776 +1 PiB 1125899906842624 +1 EiB 1152921504606846976 +invalid \N +1 Joe \N +1KB 1000 + 1 GiB \N +1 TiB with fries \N +NaN KiB \N +Inf KiB \N +0xa123 KiB \N +1 B 1 +1 KiB 1024 +1 MiB 1048576 +1 GiB 1073741824 +1 TiB 1099511627776 +1 PiB 1125899906842624 +1 EiB 1152921504606846976 +invalid 0 +1 Joe 0 +1KB 1000 + 1 GiB 0 +1 TiB with fries 0 +NaN KiB 0 +Inf KiB 0 +0xa123 KiB 0 diff --git a/tests/queries/0_stateless/03165_parseReadableSize.sql b/tests/queries/0_stateless/03165_parseReadableSize.sql new file mode 100644 index 00000000000..33386268aa4 --- /dev/null +++ b/tests/queries/0_stateless/03165_parseReadableSize.sql @@ -0,0 +1,121 @@ +-- Should be the inverse of formatReadableSize +SELECT formatReadableSize(parseReadableSize('1 B')); +SELECT formatReadableSize(parseReadableSize('1 KiB')); +SELECT formatReadableSize(parseReadableSize('1 MiB')); +SELECT formatReadableSize(parseReadableSize('1 GiB')); +SELECT formatReadableSize(parseReadableSize('1 TiB')); +SELECT formatReadableSize(parseReadableSize('1 PiB')); +SELECT formatReadableSize(parseReadableSize('1 EiB')); + +-- Should be the inverse of formatReadableDecimalSize +SELECT formatReadableDecimalSize(parseReadableSize('1 B')); +SELECT formatReadableDecimalSize(parseReadableSize('1 KB')); +SELECT formatReadableDecimalSize(parseReadableSize('1 MB')); +SELECT formatReadableDecimalSize(parseReadableSize('1 GB')); +SELECT formatReadableDecimalSize(parseReadableSize('1 TB')); +SELECT formatReadableDecimalSize(parseReadableSize('1 PB')); +SELECT formatReadableDecimalSize(parseReadableSize('1 EB')); + +-- Is case-insensitive +SELECT formatReadableSize(parseReadableSize('1 mIb')); + +-- Should be able to parse decimals +SELECT parseReadableSize('1.00 KiB'); -- 1024 +SELECT parseReadableSize('3.00 KiB'); -- 3072 + +-- Infix whitespace is ignored +SELECT parseReadableSize('1 KiB'); +SELECT parseReadableSize('1KiB'); + +-- Can parse LowCardinality +SELECT parseReadableSize(toLowCardinality('1 KiB')); + +-- Can parse nullable fields +SELECT parseReadableSize(toNullable('1 KiB')); + +-- Can parse non-const columns fields +SELECT parseReadableSize(materialize('1 KiB')); + +-- Output is NULL if NULL arg is passed +SELECT parseReadableSize(NULL); + +-- Can parse more decimal places than Float64's precision +SELECT parseReadableSize('3.14159265358979323846264338327950288419716939937510 KiB'); + +-- Can parse sizes prefixed with a plus sign +SELECT parseReadableSize('+3.1415 KiB'); + +-- Can parse amounts in scientific notation +SELECT parseReadableSize('10e2 B'); + +-- Can parse floats with no decimal points +SELECT parseReadableSize('5. B'); + +-- Can parse numbers with leading zeroes +SELECT parseReadableSize('002 KiB'); + +-- Can parse octal-like +SELECT parseReadableSize('08 KiB'); + +-- Can parse various flavours of zero +SELECT parseReadableSize('0 KiB'), parseReadableSize('+0 KiB'), parseReadableSize('-0 KiB'); + +-- ERRORS +-- No arguments +SELECT parseReadableSize(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- Too many arguments +SELECT parseReadableSize('1 B', '2 B'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +-- Wrong Type +SELECT parseReadableSize(12); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- Invalid input - overall garbage +SELECT parseReadableSize('oh no'); -- { serverError CANNOT_PARSE_NUMBER } +-- Invalid input - unknown unit +SELECT parseReadableSize('12.3 rb'); -- { serverError CANNOT_PARSE_TEXT } +-- Invalid input - Leading whitespace +SELECT parseReadableSize(' 1 B'); -- { serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED } +-- Invalid input - Trailing characters +SELECT parseReadableSize('1 B leftovers'); -- { serverError UNEXPECTED_DATA_AFTER_PARSED_VALUE } +-- Invalid input - Negative sizes are not allowed +SELECT parseReadableSize('-1 KiB'); -- { serverError BAD_ARGUMENTS } +-- Invalid input - Input too large to fit in UInt64 +SELECT parseReadableSize('1000 EiB'); -- { serverError BAD_ARGUMENTS } +-- Invalid input - Hexadecimal is not supported +SELECT parseReadableSize('0xa123 KiB'); -- { serverError CANNOT_PARSE_TEXT } +-- Invalid input - NaN is not supported, with or without sign and with different capitalizations +SELECT parseReadableSize('nan KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('+nan KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('-nan KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('NaN KiB'); -- { serverError BAD_ARGUMENTS } +-- Invalid input - Infinite is not supported, with or without sign, in all its forms +SELECT parseReadableSize('inf KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('+inf KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('-inf KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('infinite KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('+infinite KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('-infinite KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('Inf KiB'); -- { serverError BAD_ARGUMENTS } +SELECT parseReadableSize('Infinite KiB'); -- { serverError BAD_ARGUMENTS } + + +-- OR NULL +-- Works as the regular version when inputs are correct +SELECT + arrayJoin(['1 B', '1 KiB', '1 MiB', '1 GiB', '1 TiB', '1 PiB', '1 EiB']) AS readable_sizes, + parseReadableSizeOrNull(readable_sizes) AS filesize; + +-- Returns NULL on invalid values +SELECT + arrayJoin(['invalid', '1 Joe', '1KB', ' 1 GiB', '1 TiB with fries', 'NaN KiB', 'Inf KiB', '0xa123 KiB']) AS readable_sizes, + parseReadableSizeOrNull(readable_sizes) AS filesize; + + +-- OR ZERO +-- Works as the regular version when inputs are correct +SELECT + arrayJoin(['1 B', '1 KiB', '1 MiB', '1 GiB', '1 TiB', '1 PiB', '1 EiB']) AS readable_sizes, + parseReadableSizeOrZero(readable_sizes) AS filesize; + +-- Returns NULL on invalid values +SELECT + arrayJoin(['invalid', '1 Joe', '1KB', ' 1 GiB', '1 TiB with fries', 'NaN KiB', 'Inf KiB', '0xa123 KiB']) AS readable_sizes, + parseReadableSizeOrZero(readable_sizes) AS filesize; \ No newline at end of file diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.reference b/tests/queries/0_stateless/03165_round_scale_as_column.reference new file mode 100644 index 00000000000..9ad25ed466a --- /dev/null +++ b/tests/queries/0_stateless/03165_round_scale_as_column.reference @@ -0,0 +1,2165 @@ +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-1 -1 -1 -1 -1 -1 +-0.9 -1 -1 -1 -0 -0 +-0.8 -1 -1 -1 -0 -0 +-0.7 -1 -1 -1 -0 -0 +-0.6 -1 -1 -1 -0 -0 +-0.5 -0 -0 -1 -0 -0 +-0.4 -0 -0 -1 -0 -0 +-0.3 -0 -0 -1 -0 -0 +-0.2 -0 -0 -1 -0 -0 +-0.1 -0 -0 -1 -0 -0 +0 0 0 0 0 0 +0.1 0 0 0 1 0 +0.2 0 0 0 1 0 +0.3 0 0 0 1 0 +0.4 0 0 0 1 0 +0.5 0 0 0 1 0 +0.6 1 1 0 1 0 +0.7 1 1 0 1 0 +0.8 1 1 0 1 0 +0.9 1 1 0 1 0 +-1 -1 -1 -1 -1 -1 +-0.9 -1 -1 -1 -0 -0 +-0.8 -1 -1 -1 -0 -0 +-0.7 -1 -1 -1 -0 -0 +-0.6 -1 -1 -1 -0 -0 +-0.5 -0 -0 -1 -0 -0 +-0.4 -0 -0 -1 -0 -0 +-0.3 -0 -0 -1 -0 -0 +-0.2 -0 -0 -1 -0 -0 +-0.1 -0 -0 -1 -0 -0 +0 0 0 0 0 0 +0.1 0 0 0 1 0 +0.2 0 0 0 1 0 +0.3 0 0 0 1 0 +0.4 0 0 0 1 0 +0.5 0 0 0 1 0 +0.6 1 1 0 1 0 +0.7 1 1 0 1 0 +0.8 1 1 0 1 0 +0.9 1 1 0 1 0 +-1 -1 -1 -1 -1 -1 +-0.9 -0.9 -0.9 -0.9 -0.9 -0.9 +-0.8 -0.8 -0.8 -0.8 -0.8 -0.8 +-0.7 -0.7 -0.7 -0.7 -0.7 -0.7 +-0.6 -0.6 -0.6 -0.6 -0.6 -0.6 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 +-0.4 -0.4 -0.4 -0.4 -0.4 -0.4 +-0.3 -0.3 -0.3 -0.3 -0.3 -0.3 +-0.2 -0.2 -0.2 -0.2 -0.2 -0.2 +-0.1 -0.1 -0.1 -0.1 -0.1 -0.1 +0 0 0 0 0 0 +0.1 0.1 0.1 0.1 0.1 0.1 +0.2 0.2 0.2 0.2 0.2 0.2 +0.3 0.3 0.3 0.3 0.3 0.3 +0.4 0.4 0.4 0.4 0.4 0.4 +0.5 0.5 0.5 0.5 0.5 0.5 +0.6 0.6 0.6 0.6 0.6 0.6 +0.7 0.7 0.7 0.7 0.7 0.7 +0.8 0.8 0.8 0.8 0.8 0.8 +0.9 0.9 0.9 0.9 0.9 0.9 +-1 -1 -1 -1 -1 -1 +-0.9 -0.9 -0.9 -0.9 -0.9 -0.9 +-0.8 -0.8 -0.8 -0.8 -0.8 -0.8 +-0.7 -0.7 -0.7 -0.7 -0.7 -0.7 +-0.6 -0.6 -0.6 -0.6 -0.6 -0.6 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 +-0.4 -0.4 -0.4 -0.4 -0.4 -0.4 +-0.3 -0.3 -0.3 -0.3 -0.3 -0.3 +-0.2 -0.2 -0.2 -0.2 -0.2 -0.2 +-0.1 -0.1 -0.1 -0.1 -0.1 -0.1 +0 0 0 0 0 0 +0.1 0.1 0.1 0.1 0.1 0.1 +0.2 0.2 0.2 0.2 0.2 0.2 +0.3 0.3 0.3 0.3 0.3 0.3 +0.4 0.4 0.4 0.4 0.4 0.4 +0.5 0.5 0.5 0.5 0.5 0.5 +0.6 0.6 0.6 0.6 0.6 0.6 +0.7 0.7 0.7 0.7 0.7 0.7 +0.8 0.8 0.8 0.8 0.8 0.8 +0.9 0.9 0.9 0.9 0.9 0.9 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 -0 -0 +-8 -10 -10 -10 -0 -0 +-7 -10 -10 -10 -0 -0 +-6 -10 -10 -10 -0 -0 +-5 -0 -0 -10 -0 -0 +-4 -0 -0 -10 -0 -0 +-3 -0 -0 -10 -0 -0 +-2 -0 -0 -10 -0 -0 +-1 -0 -0 -10 -0 -0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 0 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 -0 -0 +-8 -10 -10 -10 -0 -0 +-7 -10 -10 -10 -0 -0 +-6 -10 -10 -10 -0 -0 +-5 -0 -0 -10 -0 -0 +-4 -0 -0 -10 -0 -0 +-3 -0 -0 -10 -0 -0 +-2 -0 -0 -10 -0 -0 +-1 -0 -0 -10 -0 -0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 0 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 -0 -0 -100 -0 -0 +-9 -0 -0 -100 -0 -0 +-8 -0 -0 -100 -0 -0 +-7 -0 -0 -100 -0 -0 +-6 -0 -0 -100 -0 -0 +-5 -0 -0 -100 -0 -0 +-4 -0 -0 -100 -0 -0 +-3 -0 -0 -100 -0 -0 +-2 -0 -0 -100 -0 -0 +-1 -0 -0 -100 -0 -0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 -0 -0 -100 -0 -0 +-9 -0 -0 -100 -0 -0 +-8 -0 -0 -100 -0 -0 +-7 -0 -0 -100 -0 -0 +-6 -0 -0 -100 -0 -0 +-5 -0 -0 -100 -0 -0 +-4 -0 -0 -100 -0 -0 +-3 -0 -0 -100 -0 -0 +-2 -0 -0 -100 -0 -0 +-1 -0 -0 -100 -0 -0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +CHECKPOINT1 +id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 244 4 244 244 244 244 244 +id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 -56 4 -56 -56 -56 -56 -56 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 -12 4 -12 -12 -12 -12 -12 +id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -1 0 -1 -1 -1 -1 -1 +21 -0.9 0 -1 -1 -1 -0 -0 +22 -0.8 0 -1 -1 -1 -0 -0 +23 -0.7 0 -1 -1 -1 -0 -0 +24 -0.6 0 -1 -1 -1 -0 -0 +25 -0.5 0 -0 -0 -1 -0 -0 +26 -0.4 0 -0 -0 -1 -0 -0 +27 -0.3 0 -0 -0 -1 -0 -0 +28 -0.2 0 -0 -0 -1 -0 -0 +29 -0.1 0 -0 -0 -1 -0 -0 +30 0 0 0 0 0 0 0 +31 0.1 0 0 0 0 1 0 +32 0.2 0 0 0 0 1 0 +33 0.3 0 0 0 0 1 0 +34 0.4 0 0 0 0 1 0 +35 0.5 0 0 0 0 1 0 +36 0.6 0 1 1 0 1 0 +37 0.7 0 1 1 0 1 0 +38 0.8 0 1 1 0 1 0 +39 0.9 0 1 1 0 1 0 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 0 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -1 -1 -0 -0 -10 -0 -0 +61 -0.9 -1 -0 -0 -10 -0 -0 +62 -0.8 -1 -0 -0 -10 -0 -0 +63 -0.7 -1 -0 -0 -10 -0 -0 +64 -0.6 -1 -0 -0 -10 -0 -0 +65 -0.5 -1 -0 -0 -10 -0 -0 +66 -0.4 -1 -0 -0 -10 -0 -0 +67 -0.3 -1 -0 -0 -10 -0 -0 +68 -0.2 -1 -0 -0 -10 -0 -0 +69 -0.1 -1 -0 -0 -10 -0 -0 +70 0 -1 0 0 0 0 0 +71 0.1 -1 0 0 0 10 0 +72 0.2 -1 0 0 0 10 0 +73 0.3 -1 0 0 0 10 0 +74 0.4 -1 0 0 0 10 0 +75 0.5 -1 0 0 0 10 0 +76 0.6 -1 0 0 0 10 0 +77 0.7 -1 0 0 0 10 0 +78 0.8 -1 0 0 0 10 0 +79 0.9 -1 0 0 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -1 -2 -0 -0 -100 -0 -0 +101 -0.9 -2 -0 -0 -100 -0 -0 +102 -0.8 -2 -0 -0 -100 -0 -0 +103 -0.7 -2 -0 -0 -100 -0 -0 +104 -0.6 -2 -0 -0 -100 -0 -0 +105 -0.5 -2 -0 -0 -100 -0 -0 +106 -0.4 -2 -0 -0 -100 -0 -0 +107 -0.3 -2 -0 -0 -100 -0 -0 +108 -0.2 -2 -0 -0 -100 -0 -0 +109 -0.1 -2 -0 -0 -100 -0 -0 +110 0 -2 0 0 0 0 0 +111 0.1 -2 0 0 0 100 0 +112 0.2 -2 0 0 0 100 0 +113 0.3 -2 0 0 0 100 0 +114 0.4 -2 0 0 0 100 0 +115 0.5 -2 0 0 0 100 0 +116 0.6 -2 0 0 0 100 0 +117 0.7 -2 0 0 0 100 0 +118 0.8 -2 0 0 0 100 0 +119 0.9 -2 0 0 0 100 0 +200 12345.679 0 12346 12346 12345 12346 12345 +201 12345.679 -1 12350 12350 12340 12350 12340 +202 12345.679 -2 12300 12300 12300 12400 12300 +203 12345.679 -3 12000 12000 12000 13000 12000 +204 12345.679 -4 10000 10000 10000 20000 10000 +205 12345.679 -5 0 0 0 100000 0 +206 12345.679 -6 0 0 0 1000000 0 +207 12345.679 -7 0 0 0 10000000 0 +208 12345.679 -8 0 0 0 100000000 0 +209 12345.679 -9 0 0 0 1000000000 0 +210 12345.679 0 12346 12346 12345 12346 12345 +211 12345.679 -1 12350 12350 12340 12350 12340 +212 12345.679 -2 12300 12300 12300 12400 12300 +213 12345.679 -3 12000 12000 12000 13000 12000 +214 12345.679 -4 10000 10000 10000 20000 10000 +215 12345.679 -5 0 0 0 100000 0 +216 12345.679 -6 0 0 0 1000000 0 +217 12345.679 -7 0 0 0 10000000 0 +218 12345.679 -8 0 0 0 100000000 0 +219 12345.679 -9 0 0 0 1000000000 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -1 0 -1 -1 -1 -1 -1 +21 -0.9 0 -1 -1 -1 -0 -0 +22 -0.8 0 -1 -1 -1 -0 -0 +23 -0.7 0 -1 -1 -1 -0 -0 +24 -0.6 0 -1 -1 -1 -0 -0 +25 -0.5 0 -0 -0 -1 -0 -0 +26 -0.4 0 -0 -0 -1 -0 -0 +27 -0.3 0 -0 -0 -1 -0 -0 +28 -0.2 0 -0 -0 -1 -0 -0 +29 -0.1 0 -0 -0 -1 -0 -0 +30 0 0 0 0 0 0 0 +31 0.1 0 0 0 0 1 0 +32 0.2 0 0 0 0 1 0 +33 0.3 0 0 0 0 1 0 +34 0.4 0 0 0 0 1 0 +35 0.5 0 0 0 0 1 0 +36 0.6 0 1 1 0 1 0 +37 0.7 0 1 1 0 1 0 +38 0.8 0 1 1 0 1 0 +39 0.9 0 1 1 0 1 0 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 0 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -1 -1 -0 -0 -10 -0 -0 +61 -0.9 -1 -0 -0 -10 -0 -0 +62 -0.8 -1 -0 -0 -10 -0 -0 +63 -0.7 -1 -0 -0 -10 -0 -0 +64 -0.6 -1 -0 -0 -10 -0 -0 +65 -0.5 -1 -0 -0 -10 -0 -0 +66 -0.4 -1 -0 -0 -10 -0 -0 +67 -0.3 -1 -0 -0 -10 -0 -0 +68 -0.2 -1 -0 -0 -10 -0 -0 +69 -0.1 -1 -0 -0 -10 -0 -0 +70 0 -1 0 0 0 0 0 +71 0.1 -1 0 0 0 10 0 +72 0.2 -1 0 0 0 10 0 +73 0.3 -1 0 0 0 10 0 +74 0.4 -1 0 0 0 10 0 +75 0.5 -1 0 0 0 10 0 +76 0.6 -1 0 0 0 10 0 +77 0.7 -1 0 0 0 10 0 +78 0.8 -1 0 0 0 10 0 +79 0.9 -1 0 0 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -1 -2 -0 -0 -100 -0 -0 +101 -0.9 -2 -0 -0 -100 -0 -0 +102 -0.8 -2 -0 -0 -100 -0 -0 +103 -0.7 -2 -0 -0 -100 -0 -0 +104 -0.6 -2 -0 -0 -100 -0 -0 +105 -0.5 -2 -0 -0 -100 -0 -0 +106 -0.4 -2 -0 -0 -100 -0 -0 +107 -0.3 -2 -0 -0 -100 -0 -0 +108 -0.2 -2 -0 -0 -100 -0 -0 +109 -0.1 -2 -0 -0 -100 -0 -0 +110 0 -2 0 0 0 0 0 +111 0.1 -2 0 0 0 100 0 +112 0.2 -2 0 0 0 100 0 +113 0.3 -2 0 0 0 100 0 +114 0.4 -2 0 0 0 100 0 +115 0.5 -2 0 0 0 100 0 +116 0.6 -2 0 0 0 100 0 +117 0.7 -2 0 0 0 100 0 +118 0.8 -2 0 0 0 100 0 +119 0.9 -2 0 0 0 100 0 +200 12345.6789 0 12346 12346 12345 12346 12345 +201 12345.6789 -1 12350 12350 12340 12350 12340 +202 12345.6789 -2 12300 12300 12300 12400 12300 +203 12345.6789 -3 12000 12000 12000 13000 12000 +204 12345.6789 -4 10000 10000 10000 20000 10000 +205 12345.6789 -5 0 0 0 100000 0 +206 12345.6789 -6 0 0 0 1000000 0 +207 12345.6789 -7 0 0 0 10000000 0 +208 12345.6789 -8 0 0 0 100000000 0 +209 12345.6789 -9 0 0 0 1000000000 0 +210 12345.6789 0 12346 12346 12345 12346 12345 +211 12345.6789 -1 12350 12350 12340 12350 12340 +212 12345.6789 -2 12300 12300 12300 12400 12300 +213 12345.6789 -3 12000 12000 12000 13000 12000 +214 12345.6789 -4 10000 10000 10000 20000 10000 +215 12345.6789 -5 0 0 0 100000 0 +216 12345.6789 -6 0 0 0 1000000 0 +217 12345.6789 -7 0 0 0 10000000 0 +218 12345.6789 -8 0 0 0 100000000 0 +219 12345.6789 -9 0 0 0 1000000000 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +CHECKPOINT2 +1 42.42 42.42 42.42 42.42 +2 0.0084 0.0084 0.0084 0.0084 +3 6.513 6.513 6.513 6.513 +4 115.6011 115.6011 115.6011 115.6011 +5 1.6029 1.6029 1.6029 1.6029 +6 -0.9999 -0.9999 -0.9999 -0.9999 +7 0.0084 0.0084 0.0084 0.0084 +8 3.7476 3.7476 3.7476 3.7476 +9 5.4066 5.4066 5.4066 5.4066 +10 1.6275 1.6275 1.6275 1.6275 +1 +1 +1 diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql new file mode 100644 index 00000000000..229f705808d --- /dev/null +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -0,0 +1,123 @@ +-- Tests functions round(), roundBankers(), floor(), ceil() and trunc() with default 'scale' argument +SELECT toUInt8(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt8(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + +SELECT toFloat32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat32((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat64((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + +-- Functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' arguments +SELECT toFloat32((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; +SELECT toFloat64((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; +SELECT toUInt8(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; + +SELECT toInt8(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; + +SELECT toUInt8(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt8(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; + +SELECT toString('CHECKPOINT1'); + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + id Int32, + scale Int16, + u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, + i8 Int8, i16 Int16, i32 Int32, i64 Int64, + f32 Float32, f64 Float64 +) ENGINE = Memory; + +INSERT INTO tab SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; + +INSERT INTO tab SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tab SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; + +INSERT INTO tab VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); +INSERT INTO tab VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); +INSERT INTO tab VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); +INSERT INTO tab VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); +INSERT INTO tab VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); +INSERT INTO tab VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); + +SELECT toString('id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; +SELECT toString('id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; + +DROP TABLE tab; +-- +SELECT toString('CHECKPOINT2'); + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + id Int32, + scale Int16, + d32 Decimal32(4), d64 Decimal64(4), d128 Decimal128(4), d256 Decimal256(4) +) ENGINE = Memory; + +INSERT INTO tab VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); +INSERT INTO tab SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tab WHERE id = 1; +INSERT INTO tab SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tab WHERE id = 1; + +SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tab ORDER BY id; + +DROP TABLE tab; + +SELECT round(1, 1); +SELECT round(materialize(1), materialize(1)); +SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} +SELECT round(materialize(1), 1); +SELECT materialize(10.1) AS x, ceil(x, toUInt256(123)); --{serverError ILLEGAL_TYPE_OF_ARGUMENT} diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference new file mode 100644 index 00000000000..4cd7f2cb141 --- /dev/null +++ b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference @@ -0,0 +1,8 @@ +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql new file mode 100644 index 00000000000..97651d1b0fd --- /dev/null +++ b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql @@ -0,0 +1,41 @@ +-- Tags: distributed + +DROP TABLE IF EXISTS ids; +DROP TABLE IF EXISTS data; +DROP TABLE IF EXISTS data2; + +CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); + +CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); + +CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN merge(currentDatabase(), 'data*') AS s ON l.id = s.id +WHERE (status IN ['CREATED', 'CREATING']) +ORDER BY event_time DESC +; + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN clusterAllReplicas(test_cluster_two_shards, merge(currentDatabase(), 'data*')) AS s ON l.id = s.id +WHERE (status IN ['CREATED', 'CREATING']) +ORDER BY event_time DESC +; + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN view(SELECT * FROM merge(currentDatabase(), 'data*')) AS s ON l.id = s.id +WHERE (status IN ['CREATED', 'CREATING']) +ORDER BY event_time DESC +; diff --git a/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.reference b/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.reference new file mode 100644 index 00000000000..4fb6812cb4f --- /dev/null +++ b/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.reference @@ -0,0 +1,83 @@ +-------- Bloom filter -------- + +-- No skip for prefix +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for prefix with complete token +Parts: 1/1 +Parts: 0/1 + +-- No skip for suffix +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for suffix with complete token +Parts: 1/1 +Parts: 0/1 + +-- No skip for substring +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for substring with complete token +Parts: 1/1 +Parts: 0/1 + +-- No skip for multiple substrings +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for multiple substrings with complete tokens +Parts: 1/1 +Parts: 0/1 + +-- No skip for multiple non-existsing substrings, only one with complete token +Parts: 1/1 +Parts: 1/1 + +-------- GIN filter -------- + +-- No skip for prefix +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for prefix with complete token +Parts: 1/1 +Parts: 0/1 + +-- No skip for suffix +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for suffix with complete token +Parts: 1/1 +Parts: 0/1 + +-- No skip for substring +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for substring with complete token +Parts: 1/1 +Parts: 0/1 + +-- No skip for multiple substrings +Parts: 1/1 +Parts: 1/1 +1 Service is not ready + +-- Skip for multiple substrings with complete tokens +Parts: 1/1 +Parts: 0/1 + +-- No skip for multiple non-existsing substrings, only one with complete token +Parts: 1/1 +Parts: 1/1 diff --git a/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql b/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql new file mode 100644 index 00000000000..bae98bd1eb6 --- /dev/null +++ b/tests/queries/0_stateless/03165_string_functions_with_token_text_indexes.sql @@ -0,0 +1,231 @@ +SELECT '-------- Bloom filter --------'; +SELECT ''; +DROP TABLE IF EXISTS 03165_token_bf; + +SET allow_experimental_full_text_index=1; + +CREATE TABLE 03165_token_bf +( + id Int64, + message String, + INDEX idx_message message TYPE tokenbf_v1(32768, 3, 2) GRANULARITY 1 +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO 03165_token_bf VALUES(1, 'Service is not ready'); + +SELECT '-- No skip for prefix'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv'); + +SELECT ''; +SELECT '-- Skip for prefix with complete token'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv i') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv i'); + +SELECT ''; +SELECT '-- No skip for suffix'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE endsWith(message, 'eady') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE endsWith(message, 'eady'); + +SELECT ''; +SELECT '-- Skip for suffix with complete token'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE endsWith(message, ' eady') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE endsWith(message, ' eady'); + +SELECT ''; +SELECT '-- No skip for substring'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE match(message, 'no') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE match(message, 'no'); + +SELECT ''; +SELECT '-- Skip for substring with complete token'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE match(message, ' xyz ') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE match(message, ' xyz '); + +SELECT ''; +SELECT '-- No skip for multiple substrings'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, ['ce', 'no']) +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, ['ce', 'no']); + +SELECT ''; +SELECT '-- Skip for multiple substrings with complete tokens'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', ' yz ']) +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', ' yz ']); + +SELECT ''; +SELECT '-- No skip for multiple non-existsing substrings, only one with complete token'; +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', 'yz']) +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', 'yz']); + +DROP TABLE IF EXISTS 03165_token_bf; + +SELECT ''; +SELECT '-------- GIN filter --------'; +SELECT ''; + +SET allow_experimental_inverted_index=1; +DROP TABLE IF EXISTS 03165_token_ft; +CREATE TABLE 03165_token_ft +( + id Int64, + message String, + INDEX idx_message message TYPE full_text() GRANULARITY 1 +) +ENGINE = MergeTree +ORDER BY id +-- Full text index works only with full parts. +SETTINGS min_bytes_for_full_part_storage=0; + +INSERT INTO 03165_token_ft VALUES(1, 'Service is not ready'); + +SELECT '-- No skip for prefix'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv'); + +SELECT ''; +SELECT '-- Skip for prefix with complete token'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv i') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv i'); + +SELECT ''; +SELECT '-- No skip for suffix'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE endsWith(message, 'eady') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE endsWith(message, 'eady'); + +SELECT ''; +SELECT '-- Skip for suffix with complete token'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE endsWith(message, ' eady') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE endsWith(message, ' eady'); + +SELECT ''; +SELECT '-- No skip for substring'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE match(message, 'no') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE match(message, 'no'); + +SELECT ''; +SELECT '-- Skip for substring with complete token'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE match(message, ' xyz ') +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE match(message, ' xyz '); + +SELECT ''; +SELECT '-- No skip for multiple substrings'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, ['ce', 'no']) +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, ['ce', 'no']); + +SELECT ''; +SELECT '-- Skip for multiple substrings with complete tokens'; + +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', ' yz ']) +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', ' yz ']); + +SELECT ''; +SELECT '-- No skip for multiple non-existsing substrings, only one with complete token'; +SELECT trim(explain) +FROM ( + EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', 'yz']) +) +WHERE explain LIKE '%Parts:%'; + +SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', 'yz']); diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference new file mode 100644 index 00000000000..b50fdcee209 --- /dev/null +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference @@ -0,0 +1,2 @@ +a b +a b diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql new file mode 100644 index 00000000000..e32d23920dd --- /dev/null +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql @@ -0,0 +1,7 @@ +create table src (x Int64) engine = Log; +create table dst (s String, lc LowCardinality(String)) engine MergeTree order by s; +create materialized view mv to dst (s String, lc String) as select 'a' as s, toLowCardinality('b') as lc from src; +insert into src values (1); + +select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=0; +select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=1; diff --git a/tests/queries/0_stateless/03166_optimize_row_order_during_insert.reference b/tests/queries/0_stateless/03166_optimize_row_order_during_insert.reference new file mode 100644 index 00000000000..bbd87fb450c --- /dev/null +++ b/tests/queries/0_stateless/03166_optimize_row_order_during_insert.reference @@ -0,0 +1,78 @@ +Simple test +Egor 1 +Egor 2 +Igor 1 +Igor 2 +Igor 3 +Cardinalities test +Alex 1 63 0 +Alex 1 65 0 +Alex 1 239 0 +Alex 2 224 0 +Alex 4 83 0 +Alex 4 134 0 +Alex 4 192 0 +Bob 2 53 0 +Bob 4 100 0 +Bob 4 177 0 +Bob 4 177 0 +Nikita 1 173 0 +Nikita 1 228 0 +Nikita 2 148 0 +Nikita 2 148 0 +Nikita 2 208 0 +Alex 1 63 1 +Alex 1 65 1 +Alex 1 239 1 +Alex 2 128 1 +Alex 2 128 1 +Alex 2 224 1 +Alex 4 83 1 +Alex 4 83 1 +Alex 4 134 1 +Alex 4 134 1 +Alex 4 192 1 +Bob 2 53 1 +Bob 2 53 1 +Bob 2 187 1 +Bob 2 187 1 +Bob 4 100 1 +Nikita 1 173 1 +Nikita 1 228 1 +Nikita 2 54 1 +Nikita 2 54 1 +Nikita 2 148 1 +Nikita 2 208 1 +Equivalence classes test +AB 1 9.81 0 +A\0 0 2.7 1 +A\0 1 2.7 1 +B\0 0 2.7 1 +B\0 1 2.7 1 +A\0 1 42 1 +B\0 0 42 1 +A\0 0 3.14 \N +B\0 -1 3.14 \N +B\0 2 3.14 \N +AB 0 42 \N +AB 0 42 \N +B\0 0 42 \N +A\0 1 42 \N +A\0 1 42 \N +B\0 1 42 \N +Many types test +A\0\0\0\0\0 2020-01-01 [0,1.1] 10 some string {'key':'value'} (123) +A\0\0\0\0\0 2020-01-01 [0,1.1] \N example {} (26) +A\0\0\0\0\0 2020-01-01 [2.2,1.1] 1 some other string {'key2':'value2'} (5) +A\0\0\0\0\0 2020-01-02 [2.2,1.1] 1 some other string {'key2':'value2'} (5) +A\0\0\0\0\0 2020-01-02 [0,1.1] 10 some string {'key':'value'} (123) +A\0\0\0\0\0 2020-01-02 [0,2.2] 10 example {} (26) +B\0\0\0\0\0 2020-01-04 [0,2.2] \N example {} (26) +B\0\0\0\0\0 2020-01-04 [0,1.1] 10 some string {'key':'value'} (123) +B\0\0\0\0\0 2020-01-04 [2.2,1.1] 1 some string {'key2':'value2'} (5) +B\0\0\0\0\0 2020-01-05 [0,1.1] 10 some string {'key':'value'} (123) +B\0\0\0\0\0 2020-01-05 [0,2.2] \N example {} (26) +B\0\0\0\0\0 2020-01-05 [2.2,1.1] 1 some other string {'key':'value'} (5) +C\0\0\0\0\0 2020-01-04 [0,1.1] 10 some string {'key':'value'} (5) +C\0\0\0\0\0 2020-01-04 [0,2.2] \N example {} (26) +C\0\0\0\0\0 2020-01-04 [2.2,1.1] 1 some other string {'key2':'value2'} (5) diff --git a/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql b/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql new file mode 100644 index 00000000000..5fc71598e47 --- /dev/null +++ b/tests/queries/0_stateless/03166_optimize_row_order_during_insert.sql @@ -0,0 +1,98 @@ +-- Checks that no bad things happen when the table optimizes the row order to improve compressability during insert. + + +-- Below SELECTs intentionally only ORDER BY the table primary key and rely on read-in-order optimization +SET optimize_read_in_order = 1; + +-- Just simple check, that optimization works correctly for table with 2 columns and 2 equivalence classes. +SELECT 'Simple test'; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + name String, + event Int8 +) ENGINE = MergeTree +ORDER BY name +SETTINGS optimize_row_order = true; +INSERT INTO tab VALUES ('Igor', 3), ('Egor', 1), ('Egor', 2), ('Igor', 2), ('Igor', 1); + +SELECT * FROM tab ORDER BY name SETTINGS max_threads=1; + +DROP TABLE tab; + +-- Checks that RowOptimizer correctly selects the order for columns according to cardinality, with an empty ORDER BY. +-- There are 4 columns with cardinalities {name : 3, timestamp": 3, money: 17, flag: 2}, so the columns order must be {flag, name, timestamp, money}. +SELECT 'Cardinalities test'; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + name String, + timestamp Int64, + money UInt8, + flag String +) ENGINE = MergeTree +ORDER BY () +SETTINGS optimize_row_order = True; +INSERT INTO tab VALUES ('Bob', 4, 100, '1'), ('Nikita', 2, 54, '1'), ('Nikita', 1, 228, '1'), ('Alex', 4, 83, '1'), ('Alex', 4, 134, '1'), ('Alex', 1, 65, '0'), ('Alex', 4, 134, '1'), ('Bob', 2, 53, '0'), ('Alex', 4, 83, '0'), ('Alex', 1, 63, '1'), ('Bob', 2, 53, '1'), ('Alex', 4, 192, '1'), ('Alex', 2, 128, '1'), ('Nikita', 2, 148, '0'), ('Bob', 4, 177, '0'), ('Nikita', 1, 173, '0'), ('Alex', 1, 239, '0'), ('Alex', 1, 63, '0'), ('Alex', 2, 224, '1'), ('Bob', 4, 177, '0'), ('Alex', 2, 128, '1'), ('Alex', 4, 134, '0'), ('Alex', 4, 83, '1'), ('Bob', 4, 100, '0'), ('Nikita', 2, 54, '1'), ('Alex', 1, 239, '1'), ('Bob', 2, 187, '1'), ('Alex', 1, 65, '1'), ('Bob', 2, 53, '1'), ('Alex', 2, 224, '0'), ('Alex', 4, 192, '0'), ('Nikita', 1, 173, '1'), ('Nikita', 2, 148, '1'), ('Bob', 2, 187, '1'), ('Nikita', 2, 208, '1'), ('Nikita', 2, 208, '0'), ('Nikita', 1, 228, '0'), ('Nikita', 2, 148, '0'); + +SELECT * FROM tab SETTINGS max_threads=1; + +DROP TABLE tab; + +-- Checks that RowOptimizer correctly selects the order for columns according to cardinality in each equivalence class obtained using SortDescription. +-- There are two columns in the SortDescription: {flag, money} in this order. +-- So there are 5 equivalence classes: {9.81, 9}, {2.7, 1}, {42, 1}, {3.14, Null}, {42, Null}. +-- For the first three of them cardinalities of the other 2 columns are equal, so they are sorted in order {0, 1} in these classes. +-- In the fourth class cardinalities: {name : 2, timestamp : 3}, so they are sorted in order {name, timestamp} in this class. +-- In the fifth class cardinalities: {name : 3, timestamp : 2}, so they are sorted in order {timestamp, name} in this class. +SELECT 'Equivalence classes test'; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + name FixedString(2), + timestamp Float32, + money Float64, + flag Nullable(Int32) +) ENGINE = MergeTree +ORDER BY (flag, money) +SETTINGS optimize_row_order = True, allow_nullable_key = True; +INSERT INTO tab VALUES ('AB', 0, 42, Null), ('AB', 0, 42, Null), ('A', 1, 42, Null), ('AB', 1, 9.81, 0), ('B', 0, 42, Null), ('B', -1, 3.14, Null), ('B', 1, 2.7, 1), ('B', 0, 42, 1), ('A', 1, 42, 1), ('B', 1, 42, Null), ('B', 0, 2.7, 1), ('A', 0, 2.7, 1), ('B', 2, 3.14, Null), ('A', 0, 3.14, Null), ('A', 1, 2.7, 1), ('A', 1, 42, Null); + +SELECT * FROM tab ORDER BY (flag, money) SETTINGS max_threads=1; + +DROP TABLE tab; + +-- Checks that no bad things happen when the table optimizes the row order to improve compressability during insert for many different column types. +-- For some of these types estimateCardinalityInPermutedRange returns just the size of the current equal range. +-- There are 5 equivalence classes, each of them has equal size = 3. +-- In the first of them cardinality of the vector_array column equals 2, other cardinalities equals 3. +-- In the second of them cardinality of the nullable_int column equals 2, other cardinalities equals 3. +-- ... +-- In the fifth of them cardinality of the tuple_column column equals 2, other cardinalities equals 3. +-- So, for all of this classes for columns with cardinality equals 2 such that estimateCardinalityInPermutedRange methid is implemented, +-- this column must be the first in the column order, all others must be in the stable order. +-- For all other classes columns must be in the stable order. +SELECT 'Many types test'; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab ( + fixed_str FixedString(6), + event_date Date, + vector_array Array(Float32), + nullable_int Nullable(Int128), + low_card_string LowCardinality(String), + map_column Map(String, String), + tuple_column Tuple(UInt256) +) ENGINE = MergeTree() +ORDER BY (fixed_str, event_date) +SETTINGS optimize_row_order = True; + +INSERT INTO tab VALUES ('A', '2020-01-01', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('A', '2020-01-01', [0.0, 1.1], NULL, 'example', {}, (26)), ('A', '2020-01-01', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)), ('A', '2020-01-02', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('A', '2020-01-02', [0.0, 2.2], 10, 'example', {}, (26)), ('A', '2020-01-02', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)), ('B', '2020-01-04', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('B', '2020-01-04', [0.0, 2.2], Null, 'example', {}, (26)), ('B', '2020-01-04', [2.2, 1.1], 1, 'some string', {'key2':'value2'}, (5)), ('B', '2020-01-05', [0.0, 1.1], 10, 'some string', {'key':'value'}, (123)), ('B', '2020-01-05', [0.0, 2.2], Null, 'example', {}, (26)), ('B', '2020-01-05', [2.2, 1.1], 1, 'some other string', {'key':'value'}, (5)), ('C', '2020-01-04', [0.0, 1.1], 10, 'some string', {'key':'value'}, (5)), ('C', '2020-01-04', [0.0, 2.2], Null, 'example', {}, (26)), ('C', '2020-01-04', [2.2, 1.1], 1, 'some other string', {'key2':'value2'}, (5)); + +SELECT * FROM tab ORDER BY (fixed_str, event_date) SETTINGS max_threads=1; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.reference b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.reference new file mode 100644 index 00000000000..86f79bea4ba --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.reference @@ -0,0 +1,33 @@ +200 +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.t_ind_merge_1) + Indexes: + PrimaryKey + Condition: true + Parts: 2/2 + Granules: 32/32 + Skip + Name: idx_b + Description: minmax GRANULARITY 1 + Parts: 2/2 + Granules: 4/32 +200 +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.t_ind_merge_1) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 32/32 + Skip + Name: idx_b + Description: minmax GRANULARITY 1 + Parts: 1/1 + Granules: 4/32 +4 1 3 diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql new file mode 100644 index 00000000000..d3e3b38a3cb --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -0,0 +1,39 @@ +DROP TABLE IF EXISTS t_ind_merge_1; + +SET allow_experimental_analyzer = 1; + +CREATE TABLE t_ind_merge_1 (a UInt64, b UInt64, c UInt64, d UInt64, INDEX idx_b b TYPE minmax) +ENGINE = MergeTree +ORDER BY a SETTINGS + index_granularity = 64, + merge_max_block_size = 8192, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + min_bytes_for_wide_part = 0, + min_bytes_for_full_part_storage = 0; + +INSERT INTO t_ind_merge_1 SELECT number, number, rand(), rand() FROM numbers(1000); +INSERT INTO t_ind_merge_1 SELECT number, number, rand(), rand() FROM numbers(1000); + +SELECT count() FROM t_ind_merge_1 WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; +EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge_1 WHERE b < 100; + +OPTIMIZE TABLE t_ind_merge_1 FINAL; + +SELECT count() FROM t_ind_merge_1 WHERE b < 100 SETTINGS force_data_skipping_indices = 'idx_b'; +EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge_1 WHERE b < 100; + +SYSTEM FLUSH LOGS; + +WITH + (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge_1') AS uuid, + extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups +SELECT + groups[1] AS total, + groups[2] AS merged, + groups[3] AS gathered +FROM system.text_log +WHERE ((query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_1::all_1_2_1')) AND notEmpty(groups) +ORDER BY event_time_microseconds; + +DROP TABLE t_ind_merge_1; diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference new file mode 100644 index 00000000000..4c2f01294a4 --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.reference @@ -0,0 +1 @@ +6 3 3 diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql new file mode 100644 index 00000000000..b749e0c84b0 --- /dev/null +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -0,0 +1,42 @@ +DROP TABLE IF EXISTS t_ind_merge_2; + +CREATE TABLE t_ind_merge_2 ( + a UInt64, + b UInt64, + c UInt64, + d UInt64, + e UInt64, + f UInt64, + INDEX idx_a a TYPE minmax, + INDEX idx_b b TYPE minmax, + INDEX idx_cd c * d TYPE minmax, + INDEX idx_d1 d TYPE minmax, + INDEX idx_d2 d + 7 TYPE set(3), + INDEX idx_e e * 3 TYPE set(3)) +ENGINE = MergeTree +ORDER BY a SETTINGS + index_granularity = 64, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + min_bytes_for_wide_part = 0, + min_bytes_for_full_part_storage = 0; + +INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() FROM numbers(1000); +INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() FROM numbers(1000); + +OPTIMIZE TABLE t_ind_merge_2 FINAL; +SYSTEM FLUSH LOGS; + +--- merged: a, c, d; gathered: b, e, f +WITH + (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge_2') AS uuid, + extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups +SELECT + groups[1] AS total, + groups[2] AS merged, + groups[3] AS gathered +FROM system.text_log +WHERE ((query_id = uuid || '::all_1_2_1') OR (query_id = currentDatabase() || '.t_ind_merge_2::all_1_2_1')) AND notEmpty(groups) +ORDER BY event_time_microseconds; + +DROP TABLE t_ind_merge_2; diff --git a/tests/queries/0_stateless/03167_base64_url_functions.reference b/tests/queries/0_stateless/03167_base64_url_functions.reference new file mode 100644 index 00000000000..2a0d0013609 --- /dev/null +++ b/tests/queries/0_stateless/03167_base64_url_functions.reference @@ -0,0 +1,10 @@ +https://clickhouse.com aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ https://clickhouse.com https://clickhouse.com +12? MTI_ 12? 12? +https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode aHR0cHM6Ly93d3cuZ29vZ2xlLmNvbS9zZWFyY2g_cT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGUmc2NhX2Vzdj03MzlmOGJiMzgwZTRjN2VkJmVpPVRmUmlacUNESXJtbndQQVAyS0xSa0E4JnZlZD0wYWhVS0V3amczWkhpdHNtR0F4VzVFeEFJSFZoUkZQSVE0ZFVEQ0JBJnVhY3Q9NSZvcT1jbGlja2hvdXNlK2Jhc2U2NCtkZWNvZGU https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode +aHR0cHM6Ly9jbGlj https://clic https://clic +aHR0cHM6Ly9jbGlja2g https://clickh https://clickh +aHR0cHM6Ly9jbGljaw https://click https://click + + + +https://clickhouse.com aHR0cHM6Ly9jbGlja2hvdXNlLmNvbQ https://clickhouse.com https://clickhouse.com diff --git a/tests/queries/0_stateless/03167_base64_url_functions.sql b/tests/queries/0_stateless/03167_base64_url_functions.sql new file mode 100644 index 00000000000..674f1ae498b --- /dev/null +++ b/tests/queries/0_stateless/03167_base64_url_functions.sql @@ -0,0 +1,36 @@ +-- Tags: no-fasttest +-- no-fasttest because aklomp-base64 library is required + +-- incorrect number of arguments +SELECT base64UrlEncode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryBase64UrlDecode(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64UrlEncode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT base64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tryBase64UrlDecode('foo', 'excess argument'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +-- test with valid inputs + +SELECT 'https://clickhouse.com' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +SELECT '12?' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +SELECT 'https://www.google.com/search?q=clickhouse+base64+decode&sca_esv=739f8bb380e4c7ed&ei=TfRiZqCDIrmnwPAP2KLRkA8&ved=0ahUKEwjg3ZHitsmGAxW5ExAIHVhRFPIQ4dUDCBA&uact=5&oq=clickhouse+base64+decode' AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); + +-- encoded value has no padding +SELECT 'aHR0cHM6Ly9jbGlj' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +-- encoded value has one-byte padding +SELECT 'aHR0cHM6Ly9jbGlja2g' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); +-- encoded value has two-bytes padding +SELECT 'aHR0cHM6Ly9jbGljaw' AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); + +-- test with invalid inputs + +SELECT base64UrlDecode('https://clickhouse.com'); -- { serverError INCORRECT_DATA } +SELECT tryBase64UrlDecode('https://clickhouse.com'); +SELECT base64UrlDecode('12?'); -- { serverError INCORRECT_DATA } +SELECT tryBase64UrlDecode('12?'); +SELECT base64UrlDecode('aHR0cHM6Ly9jbGlja'); -- { serverError INCORRECT_DATA } +SELECT tryBase64UrlDecode('aHR0cHM6Ly9jbGlja'); + +-- test FixedString argument + +SELECT toFixedString('https://clickhouse.com', 22) AS original, base64UrlEncode(original) AS encoded, base64UrlDecode(encoded), tryBase64UrlDecode(encoded); diff --git a/tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 b/tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 new file mode 100644 index 00000000000..71dc879f28e --- /dev/null +++ b/tests/queries/0_stateless/03167_boom_filter_index_with_map.reference.j2 @@ -0,0 +1,4 @@ +{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%} +{'xxx':56} +{56:'xxx'} +{% endfor -%} diff --git a/tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 b/tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 new file mode 100644 index 00000000000..4147bd84e8e --- /dev/null +++ b/tests/queries/0_stateless/03167_boom_filter_index_with_map.sql.j2 @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS boom_filter_map_1; +DROP TABLE IF EXISTS boom_filter_map_2; + +{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%} + +CREATE TABLE boom_filter_map_1 +( + `m` Map(String, {{ type }}), + INDEX index_models_value_bloom_filter mapValues(m) TYPE bloom_filter GRANULARITY 1 +) + ENGINE = MergeTree +ORDER BY tuple(); + +CREATE TABLE boom_filter_map_2 +( + `m` Map({{ type }}, String), + INDEX index_models_value_bloom_filter mapKeys(m) TYPE bloom_filter GRANULARITY 1 +) + ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO boom_filter_map_1 (m) values (map('xxx', 56)); +INSERT INTO boom_filter_map_2 (m) values (map(56, 'xxx')); + +SELECT m FROM boom_filter_map_1 WHERE (m['xxx']) = 56; +SELECT m FROM boom_filter_map_2 WHERE (m[56]) = 'xxx'; + +DROP TABLE IF EXISTS boom_filter_map_1; +DROP TABLE IF EXISTS boom_filter_map_2; + +{% endfor -%} diff --git a/tests/queries/0_stateless/03167_empty_tuple_concat.reference b/tests/queries/0_stateless/03167_empty_tuple_concat.reference new file mode 100644 index 00000000000..6a452c185a8 --- /dev/null +++ b/tests/queries/0_stateless/03167_empty_tuple_concat.reference @@ -0,0 +1 @@ +() diff --git a/tests/queries/0_stateless/03167_empty_tuple_concat.sql b/tests/queries/0_stateless/03167_empty_tuple_concat.sql new file mode 100644 index 00000000000..f6fce86f332 --- /dev/null +++ b/tests/queries/0_stateless/03167_empty_tuple_concat.sql @@ -0,0 +1 @@ +SELECT ()||(); diff --git a/tests/queries/0_stateless/03167_fancy_quotes_off_by_one.reference b/tests/queries/0_stateless/03167_fancy_quotes_off_by_one.reference new file mode 100644 index 00000000000..9daeafb9864 --- /dev/null +++ b/tests/queries/0_stateless/03167_fancy_quotes_off_by_one.reference @@ -0,0 +1 @@ +test diff --git a/tests/queries/0_stateless/03167_fancy_quotes_off_by_one.sql b/tests/queries/0_stateless/03167_fancy_quotes_off_by_one.sql new file mode 100644 index 00000000000..6f563d8f2a1 --- /dev/null +++ b/tests/queries/0_stateless/03167_fancy_quotes_off_by_one.sql @@ -0,0 +1 @@ +SELECT ‘test’ AS “column” \ No newline at end of file diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.reference b/tests/queries/0_stateless/03167_transactions_are_really_disabled.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql new file mode 100644 index 00000000000..e3c86a2d5be --- /dev/null +++ b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS mv_table; +DROP TABLE IF EXISTS null_table; + +CREATE TABLE null_table (str String) ENGINE = Null; +CREATE MATERIALIZED VIEW mv_table (str String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/transactions_disabled_rmt', '{replica}') ORDER BY str AS SELECT str AS str FROM null_table; + +SET implicit_transaction=1; +set throw_on_unsupported_query_inside_transaction=0; + +INSERT INTO null_table VALUES ('test'); --{serverError NOT_IMPLEMENTED} + +DROP TABLE IF EXISTS mv_table; +DROP TABLE IF EXISTS null_table; diff --git a/tests/queries/0_stateless/03168_cld2_tsan.reference b/tests/queries/0_stateless/03168_cld2_tsan.reference new file mode 100644 index 00000000000..6c3cafd4a6d --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.reference @@ -0,0 +1,2 @@ +{'ja':0.62,'fr':0.36} +{'ja':0.62,'fr':0.36} diff --git a/tests/queries/0_stateless/03168_cld2_tsan.sql b/tests/queries/0_stateless/03168_cld2_tsan.sql new file mode 100644 index 00000000000..701a781c472 --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: depends on cld2 + +-- https://github.com/ClickHouse/ClickHouse/issues/64931 +SELECT detectLanguageMixed(materialize('二兎を追う者は一兎をも得ず二兎を追う者は一兎をも得ず A vaincre sans peril, on triomphe sans gloire.')) +GROUP BY + GROUPING SETS ( + ('a', toUInt256(1)), + (stringToH3(toFixedString(toFixedString('85283473ffffff', 14), 14)))) +SETTINGS allow_experimental_nlp_functions = 1; diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql new file mode 100644 index 00000000000..4e4cc291e9b --- /dev/null +++ b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql @@ -0,0 +1,6 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64946 +SELECT + multiIf((number % toLowCardinality(toNullable(toUInt128(2)))) = (number % toNullable(2)), toInt8(1), (number % materialize(toLowCardinality(3))) = toUInt128(toNullable(0)), toInt8(materialize(materialize(2))), toInt64(toUInt128(3))) +FROM system.numbers +LIMIT 44857 +FORMAT Null; diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql new file mode 100644 index 00000000000..da4626ad897 --- /dev/null +++ b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel + +DROP DATABASE IF EXISTS 03147_db; +CREATE DATABASE IF NOT EXISTS 03147_db; +CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; +INSERT INTO 03147_db.t SELECT * FROM numbers(10); +USE 03147_db; + +SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 100; + +SELECT * FROM loop(03147_db.t) LIMIT 15 FORMAT Null; diff --git a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference new file mode 100644 index 00000000000..e3ac97f9945 --- /dev/null +++ b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference @@ -0,0 +1,5 @@ +1 +3168 8613 +[] ['SELECT(a, b) ON default.d_03168_query_log'] +[] [] +['SELECT(a, b) ON default.d_03168_query_log'] [] diff --git a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh new file mode 100755 index 00000000000..9abc635a874 --- /dev/null +++ b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +user_name="u_03168_query_log" +table_name="default.d_03168_query_log" +test_query="select a, b from ${table_name}" + +${CLICKHOUSE_CLIENT_BINARY} --query "drop user if exists ${user_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "create user ${user_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "drop table if exists ${table_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "create table ${table_name} (a UInt64, b UInt64) order by a" + +${CLICKHOUSE_CLIENT_BINARY} --query "insert into table ${table_name} values (3168, 8613)" + +error="$(${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" 2>&1 >/dev/null)" +echo "${error}" | grep -Fc "ACCESS_DENIED" + +${CLICKHOUSE_CLIENT_BINARY} --query "grant select(a, b) on ${table_name} to ${user_name}" + +${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" + +${CLICKHOUSE_CLIENT_BINARY} --query "system flush logs" +${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'ExceptionBeforeStart' and current_database = currentDatabase() order by event_time desc limit 1" +${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryStart' and current_database = currentDatabase() order by event_time desc limit 1" +${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryFinish' and current_database = currentDatabase() order by event_time desc limit 1" + +${CLICKHOUSE_CLIENT_BINARY} --query "drop table ${table_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "drop user ${user_name}" diff --git a/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.reference b/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql b/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql new file mode 100644 index 00000000000..8463d13d251 --- /dev/null +++ b/tests/queries/0_stateless/03169_cache_complex_dict_short_circuit_bug.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS complex_key_simple_attributes_source_short_circuit_table; +DROP DICTIONARY IF EXISTS cache_dictionary_complex_key_simple_attributes_short_circuit; + +CREATE TABLE complex_key_simple_attributes_source_short_circuit_table +( + id UInt64, + id_key String, + value_first String, + value_second String +) + ENGINE = TinyLog; + +INSERT INTO complex_key_simple_attributes_source_short_circuit_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); + +CREATE DICTIONARY cache_dictionary_complex_key_simple_attributes_short_circuit +( + `id` UInt64, + `id_key` String, + `value_first` String DEFAULT 'value_first_default', + `value_second` String DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(TABLE 'complex_key_simple_attributes_source_short_circuit_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10)); + +SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes_short_circuit', 'value_first', (number, concat(toString(number))), toString(materialize('default'))) AS value_first FROM system.numbers LIMIT 20 FORMAT Null; +SELECT dictGetOrDefault('cache_dictionary_complex_key_simple_attributes_short_circuit', 'value_first', (number, concat(toString(number))), toString(materialize('default'))) AS value_first FROM system.numbers LIMIT 20 FORMAT Null; + +DROP TABLE IF EXISTS complex_key_simple_attributes_source_short_circuit_table; +DROP DICTIONARY IF EXISTS cache_dictionary_complex_key_simple_attributes_short_circuit; diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.reference b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference new file mode 100644 index 00000000000..7a9f413a900 --- /dev/null +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.reference @@ -0,0 +1,2382 @@ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ +10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ +11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ +12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ +13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ +14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ +15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ +16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ +17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ +18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ +19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ +20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ +21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ +22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ +23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ +24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ +25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ +26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ +27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ +28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ +29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ +30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ +31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ +32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ +33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ +34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ +35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ +36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ +37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ +38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ +39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ +40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ +41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ +42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ +43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ +44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ +45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ +46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ +47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ +48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ +49. │ 48 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ +10. │ 9 │ UInt64 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ + ┏━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━┓ + ┃ number ┃ toTypeName(number) ┃ + ┡━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━┩ + 1. │ 0 │ UInt64 │ + ├────────┼────────────────────┤ + 2. │ 1 │ UInt64 │ + ├────────┼────────────────────┤ + 3. │ 2 │ UInt64 │ + ├────────┼────────────────────┤ + 4. │ 3 │ UInt64 │ + ├────────┼────────────────────┤ + 5. │ 4 │ UInt64 │ + ├────────┼────────────────────┤ + 6. │ 5 │ UInt64 │ + ├────────┼────────────────────┤ + 7. │ 6 │ UInt64 │ + ├────────┼────────────────────┤ + 8. │ 7 │ UInt64 │ + ├────────┼────────────────────┤ + 9. │ 8 │ UInt64 │ + ├────────┼────────────────────┤ + 10. │ 9 │ UInt64 │ + ├────────┼────────────────────┤ + 11. │ 10 │ UInt64 │ + ├────────┼────────────────────┤ + 12. │ 11 │ UInt64 │ + ├────────┼────────────────────┤ + 13. │ 12 │ UInt64 │ + ├────────┼────────────────────┤ + 14. │ 13 │ UInt64 │ + ├────────┼────────────────────┤ + 15. │ 14 │ UInt64 │ + ├────────┼────────────────────┤ + 16. │ 15 │ UInt64 │ + ├────────┼────────────────────┤ + 17. │ 16 │ UInt64 │ + ├────────┼────────────────────┤ + 18. │ 17 │ UInt64 │ + ├────────┼────────────────────┤ + 19. │ 18 │ UInt64 │ + ├────────┼────────────────────┤ + 20. │ 19 │ UInt64 │ + ├────────┼────────────────────┤ + 21. │ 20 │ UInt64 │ + ├────────┼────────────────────┤ + 22. │ 21 │ UInt64 │ + ├────────┼────────────────────┤ + 23. │ 22 │ UInt64 │ + ├────────┼────────────────────┤ + 24. │ 23 │ UInt64 │ + ├────────┼────────────────────┤ + 25. │ 24 │ UInt64 │ + ├────────┼────────────────────┤ + 26. │ 25 │ UInt64 │ + ├────────┼────────────────────┤ + 27. │ 26 │ UInt64 │ + ├────────┼────────────────────┤ + 28. │ 27 │ UInt64 │ + ├────────┼────────────────────┤ + 29. │ 28 │ UInt64 │ + ├────────┼────────────────────┤ + 30. │ 29 │ UInt64 │ + ├────────┼────────────────────┤ + 31. │ 30 │ UInt64 │ + ├────────┼────────────────────┤ + 32. │ 31 │ UInt64 │ + ├────────┼────────────────────┤ + 33. │ 32 │ UInt64 │ + ├────────┼────────────────────┤ + 34. │ 33 │ UInt64 │ + ├────────┼────────────────────┤ + 35. │ 34 │ UInt64 │ + ├────────┼────────────────────┤ + 36. │ 35 │ UInt64 │ + ├────────┼────────────────────┤ + 37. │ 36 │ UInt64 │ + ├────────┼────────────────────┤ + 38. │ 37 │ UInt64 │ + ├────────┼────────────────────┤ + 39. │ 38 │ UInt64 │ + ├────────┼────────────────────┤ + 40. │ 39 │ UInt64 │ + ├────────┼────────────────────┤ + 41. │ 40 │ UInt64 │ + ├────────┼────────────────────┤ + 42. │ 41 │ UInt64 │ + ├────────┼────────────────────┤ + 43. │ 42 │ UInt64 │ + ├────────┼────────────────────┤ + 44. │ 43 │ UInt64 │ + ├────────┼────────────────────┤ + 45. │ 44 │ UInt64 │ + ├────────┼────────────────────┤ + 46. │ 45 │ UInt64 │ + ├────────┼────────────────────┤ + 47. │ 46 │ UInt64 │ + ├────────┼────────────────────┤ + 48. │ 47 │ UInt64 │ + ├────────┼────────────────────┤ + 49. │ 48 │ UInt64 │ + ├────────┼────────────────────┤ + 50. │ 49 │ UInt64 │ + ├────────┼────────────────────┤ + 51. │ 50 │ UInt64 │ + ├────────┼────────────────────┤ + 52. │ 51 │ UInt64 │ + ├────────┼────────────────────┤ + 53. │ 52 │ UInt64 │ + ├────────┼────────────────────┤ + 54. │ 53 │ UInt64 │ + ├────────┼────────────────────┤ + 55. │ 54 │ UInt64 │ + ├────────┼────────────────────┤ + 56. │ 55 │ UInt64 │ + ├────────┼────────────────────┤ + 57. │ 56 │ UInt64 │ + ├────────┼────────────────────┤ + 58. │ 57 │ UInt64 │ + ├────────┼────────────────────┤ + 59. │ 58 │ UInt64 │ + ├────────┼────────────────────┤ + 60. │ 59 │ UInt64 │ + ├────────┼────────────────────┤ + 61. │ 60 │ UInt64 │ + ├────────┼────────────────────┤ + 62. │ 61 │ UInt64 │ + ├────────┼────────────────────┤ + 63. │ 62 │ UInt64 │ + ├────────┼────────────────────┤ + 64. │ 63 │ UInt64 │ + ├────────┼────────────────────┤ + 65. │ 64 │ UInt64 │ + ├────────┼────────────────────┤ + 66. │ 65 │ UInt64 │ + ├────────┼────────────────────┤ + 67. │ 66 │ UInt64 │ + ├────────┼────────────────────┤ + 68. │ 67 │ UInt64 │ + ├────────┼────────────────────┤ + 69. │ 68 │ UInt64 │ + ├────────┼────────────────────┤ + 70. │ 69 │ UInt64 │ + ├────────┼────────────────────┤ + 71. │ 70 │ UInt64 │ + ├────────┼────────────────────┤ + 72. │ 71 │ UInt64 │ + ├────────┼────────────────────┤ + 73. │ 72 │ UInt64 │ + ├────────┼────────────────────┤ + 74. │ 73 │ UInt64 │ + ├────────┼────────────────────┤ + 75. │ 74 │ UInt64 │ + ├────────┼────────────────────┤ + 76. │ 75 │ UInt64 │ + ├────────┼────────────────────┤ + 77. │ 76 │ UInt64 │ + ├────────┼────────────────────┤ + 78. │ 77 │ UInt64 │ + ├────────┼────────────────────┤ + 79. │ 78 │ UInt64 │ + ├────────┼────────────────────┤ + 80. │ 79 │ UInt64 │ + ├────────┼────────────────────┤ + 81. │ 80 │ UInt64 │ + ├────────┼────────────────────┤ + 82. │ 81 │ UInt64 │ + ├────────┼────────────────────┤ + 83. │ 82 │ UInt64 │ + ├────────┼────────────────────┤ + 84. │ 83 │ UInt64 │ + ├────────┼────────────────────┤ + 85. │ 84 │ UInt64 │ + ├────────┼────────────────────┤ + 86. │ 85 │ UInt64 │ + ├────────┼────────────────────┤ + 87. │ 86 │ UInt64 │ + ├────────┼────────────────────┤ + 88. │ 87 │ UInt64 │ + ├────────┼────────────────────┤ + 89. │ 88 │ UInt64 │ + ├────────┼────────────────────┤ + 90. │ 89 │ UInt64 │ + ├────────┼────────────────────┤ + 91. │ 90 │ UInt64 │ + ├────────┼────────────────────┤ + 92. │ 91 │ UInt64 │ + ├────────┼────────────────────┤ + 93. │ 92 │ UInt64 │ + ├────────┼────────────────────┤ + 94. │ 93 │ UInt64 │ + ├────────┼────────────────────┤ + 95. │ 94 │ UInt64 │ + ├────────┼────────────────────┤ + 96. │ 95 │ UInt64 │ + ├────────┼────────────────────┤ + 97. │ 96 │ UInt64 │ + ├────────┼────────────────────┤ + 98. │ 97 │ UInt64 │ + ├────────┼────────────────────┤ + 99. │ 98 │ UInt64 │ + ├────────┼────────────────────┤ +100. │ 99 │ UInt64 │ + ┣━━━━━━━━╋━━━━━━━━━━━━━━━━━━━━┫ + ┃ number ┃ toTypeName(number) ┃ + ┗━━━━━━━━┻━━━━━━━━━━━━━━━━━━━━┛ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └────────┴────────────────────┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └─number─┴─toTypeName(number)─┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └─number─┴─toTypeName(number)─┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └─number─┴─toTypeName(number)─┘ + ┌─number─┬─toTypeName(number)─┐ + 1. │ 0 │ UInt64 │ + 2. │ 1 │ UInt64 │ + 3. │ 2 │ UInt64 │ + 4. │ 3 │ UInt64 │ + 5. │ 4 │ UInt64 │ + 6. │ 5 │ UInt64 │ + 7. │ 6 │ UInt64 │ + 8. │ 7 │ UInt64 │ + 9. │ 8 │ UInt64 │ + 10. │ 9 │ UInt64 │ + 11. │ 10 │ UInt64 │ + 12. │ 11 │ UInt64 │ + 13. │ 12 │ UInt64 │ + 14. │ 13 │ UInt64 │ + 15. │ 14 │ UInt64 │ + 16. │ 15 │ UInt64 │ + 17. │ 16 │ UInt64 │ + 18. │ 17 │ UInt64 │ + 19. │ 18 │ UInt64 │ + 20. │ 19 │ UInt64 │ + 21. │ 20 │ UInt64 │ + 22. │ 21 │ UInt64 │ + 23. │ 22 │ UInt64 │ + 24. │ 23 │ UInt64 │ + 25. │ 24 │ UInt64 │ + 26. │ 25 │ UInt64 │ + 27. │ 26 │ UInt64 │ + 28. │ 27 │ UInt64 │ + 29. │ 28 │ UInt64 │ + 30. │ 29 │ UInt64 │ + 31. │ 30 │ UInt64 │ + 32. │ 31 │ UInt64 │ + 33. │ 32 │ UInt64 │ + 34. │ 33 │ UInt64 │ + 35. │ 34 │ UInt64 │ + 36. │ 35 │ UInt64 │ + 37. │ 36 │ UInt64 │ + 38. │ 37 │ UInt64 │ + 39. │ 38 │ UInt64 │ + 40. │ 39 │ UInt64 │ + 41. │ 40 │ UInt64 │ + 42. │ 41 │ UInt64 │ + 43. │ 42 │ UInt64 │ + 44. │ 43 │ UInt64 │ + 45. │ 44 │ UInt64 │ + 46. │ 45 │ UInt64 │ + 47. │ 46 │ UInt64 │ + 48. │ 47 │ UInt64 │ + 49. │ 48 │ UInt64 │ + 50. │ 49 │ UInt64 │ + 51. │ 50 │ UInt64 │ + 52. │ 51 │ UInt64 │ + 53. │ 52 │ UInt64 │ + 54. │ 53 │ UInt64 │ + 55. │ 54 │ UInt64 │ + 56. │ 55 │ UInt64 │ + 57. │ 56 │ UInt64 │ + 58. │ 57 │ UInt64 │ + 59. │ 58 │ UInt64 │ + 60. │ 59 │ UInt64 │ + 61. │ 60 │ UInt64 │ + 62. │ 61 │ UInt64 │ + 63. │ 62 │ UInt64 │ + 64. │ 63 │ UInt64 │ + 65. │ 64 │ UInt64 │ + 66. │ 65 │ UInt64 │ + 67. │ 66 │ UInt64 │ + 68. │ 67 │ UInt64 │ + 69. │ 68 │ UInt64 │ + 70. │ 69 │ UInt64 │ + 71. │ 70 │ UInt64 │ + 72. │ 71 │ UInt64 │ + 73. │ 72 │ UInt64 │ + 74. │ 73 │ UInt64 │ + 75. │ 74 │ UInt64 │ + 76. │ 75 │ UInt64 │ + 77. │ 76 │ UInt64 │ + 78. │ 77 │ UInt64 │ + 79. │ 78 │ UInt64 │ + 80. │ 79 │ UInt64 │ + 81. │ 80 │ UInt64 │ + 82. │ 81 │ UInt64 │ + 83. │ 82 │ UInt64 │ + 84. │ 83 │ UInt64 │ + 85. │ 84 │ UInt64 │ + 86. │ 85 │ UInt64 │ + 87. │ 86 │ UInt64 │ + 88. │ 87 │ UInt64 │ + 89. │ 88 │ UInt64 │ + 90. │ 89 │ UInt64 │ + 91. │ 90 │ UInt64 │ + 92. │ 91 │ UInt64 │ + 93. │ 92 │ UInt64 │ + 94. │ 93 │ UInt64 │ + 95. │ 94 │ UInt64 │ + 96. │ 95 │ UInt64 │ + 97. │ 96 │ UInt64 │ + 98. │ 97 │ UInt64 │ + 99. │ 98 │ UInt64 │ +100. │ 99 │ UInt64 │ + └─number─┴─toTypeName(number)─┘ + number toTypeName(number) + + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + number toTypeName(number) + + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + + number toTypeName(number) + number toTypeName(number) + + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + + number toTypeName(number) + number toTypeName(number) + + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + + number toTypeName(number) + number toTypeName(number) + + 1. 0 UInt64 + 2. 1 UInt64 + 3. 2 UInt64 + 4. 3 UInt64 + 5. 4 UInt64 + 6. 5 UInt64 + 7. 6 UInt64 + 8. 7 UInt64 + 9. 8 UInt64 + 10. 9 UInt64 + 11. 10 UInt64 + 12. 11 UInt64 + 13. 12 UInt64 + 14. 13 UInt64 + 15. 14 UInt64 + 16. 15 UInt64 + 17. 16 UInt64 + 18. 17 UInt64 + 19. 18 UInt64 + 20. 19 UInt64 + 21. 20 UInt64 + 22. 21 UInt64 + 23. 22 UInt64 + 24. 23 UInt64 + 25. 24 UInt64 + 26. 25 UInt64 + 27. 26 UInt64 + 28. 27 UInt64 + 29. 28 UInt64 + 30. 29 UInt64 + 31. 30 UInt64 + 32. 31 UInt64 + 33. 32 UInt64 + 34. 33 UInt64 + 35. 34 UInt64 + 36. 35 UInt64 + 37. 36 UInt64 + 38. 37 UInt64 + 39. 38 UInt64 + 40. 39 UInt64 + 41. 40 UInt64 + 42. 41 UInt64 + 43. 42 UInt64 + 44. 43 UInt64 + 45. 44 UInt64 + 46. 45 UInt64 + 47. 46 UInt64 + 48. 47 UInt64 + 49. 48 UInt64 + 50. 49 UInt64 + 51. 50 UInt64 + 52. 51 UInt64 + 53. 52 UInt64 + 54. 53 UInt64 + 55. 54 UInt64 + 56. 55 UInt64 + 57. 56 UInt64 + 58. 57 UInt64 + 59. 58 UInt64 + 60. 59 UInt64 + 61. 60 UInt64 + 62. 61 UInt64 + 63. 62 UInt64 + 64. 63 UInt64 + 65. 64 UInt64 + 66. 65 UInt64 + 67. 66 UInt64 + 68. 67 UInt64 + 69. 68 UInt64 + 70. 69 UInt64 + 71. 70 UInt64 + 72. 71 UInt64 + 73. 72 UInt64 + 74. 73 UInt64 + 75. 74 UInt64 + 76. 75 UInt64 + 77. 76 UInt64 + 78. 77 UInt64 + 79. 78 UInt64 + 80. 79 UInt64 + 81. 80 UInt64 + 82. 81 UInt64 + 83. 82 UInt64 + 84. 83 UInt64 + 85. 84 UInt64 + 86. 85 UInt64 + 87. 86 UInt64 + 88. 87 UInt64 + 89. 88 UInt64 + 90. 89 UInt64 + 91. 90 UInt64 + 92. 91 UInt64 + 93. 92 UInt64 + 94. 93 UInt64 + 95. 94 UInt64 + 96. 95 UInt64 + 97. 96 UInt64 + 98. 97 UInt64 + 99. 98 UInt64 +100. 99 UInt64 + + number toTypeName(number) diff --git a/tests/queries/0_stateless/03169_display_column_names_in_footer.sql b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql new file mode 100644 index 00000000000..9e4ec09c21c --- /dev/null +++ b/tests/queries/0_stateless/03169_display_column_names_in_footer.sql @@ -0,0 +1,19 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/65035 +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 49) FORMAT Pretty; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 10) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names_min_rows=9; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapes; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapes; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapesMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace SETTINGS output_format_pretty_display_footer_column_names=0; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapes; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceMonoBlock; +SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapesMonoBlock; diff --git a/tests/queries/0_stateless/03169_modify_column_data_loss.reference b/tests/queries/0_stateless/03169_modify_column_data_loss.reference new file mode 100644 index 00000000000..2126a658c16 --- /dev/null +++ b/tests/queries/0_stateless/03169_modify_column_data_loss.reference @@ -0,0 +1,4 @@ +1 one 0 +2 two 0 +3 \N 0 +1 one 1 0 diff --git a/tests/queries/0_stateless/03169_modify_column_data_loss.sql b/tests/queries/0_stateless/03169_modify_column_data_loss.sql new file mode 100644 index 00000000000..def0a25a1b4 --- /dev/null +++ b/tests/queries/0_stateless/03169_modify_column_data_loss.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS column_modify_test; + +CREATE TABLE column_modify_test (id UInt64, val String, other_col UInt64) engine=MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part=0; +INSERT INTO column_modify_test VALUES (1,'one',0); +INSERT INTO column_modify_test VALUES (2,'two',0); + +-- on 21.9 that was done via mutations mechanism +ALTER TABLE column_modify_test MODIFY COLUMN val Nullable(String); + +INSERT INTO column_modify_test VALUES (3,Null,0); + +-- till now everythings looks ok +SELECT * FROM column_modify_test order by id, val, other_col; + +-- Now we do mutation. It will affect one of the parts, and will update columns.txt to the latest / correct state w/o updating the column file! +alter table column_modify_test update other_col=1 where id = 1 SETTINGS mutations_sync=1; + +-- row 1 is damaged now the column file & columns.txt is out of sync! +SELECT *, throwIf(val <> 'one') as issue FROM column_modify_test WHERE id = 1; diff --git a/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference new file mode 100644 index 00000000000..e58e9764b39 --- /dev/null +++ b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.reference @@ -0,0 +1,2 @@ +100 +100 diff --git a/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql new file mode 100644 index 00000000000..50d99b851a6 --- /dev/null +++ b/tests/queries/0_stateless/03169_optimize_injective_functions_inside_uniq_crash.sql @@ -0,0 +1,21 @@ +SELECT sum(u) +FROM +( + SELECT + intDiv(number, 4096) AS k, + uniqCombined(tuple(materialize(toLowCardinality(toNullable(16))))) AS u + FROM numbers(4096 * 100) + GROUP BY k +) +SETTINGS allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq=0; + +SELECT sum(u) +FROM +( + SELECT + intDiv(number, 4096) AS k, + uniqCombined(tuple(materialize(toLowCardinality(toNullable(16))))) AS u + FROM numbers(4096 * 100) + GROUP BY k +) +SETTINGS allow_experimental_analyzer = 1, optimize_injective_functions_inside_uniq=1; diff --git a/tests/queries/0_stateless/03169_time_virtual_column.reference b/tests/queries/0_stateless/03169_time_virtual_column.reference new file mode 100644 index 00000000000..4482956b706 --- /dev/null +++ b/tests/queries/0_stateless/03169_time_virtual_column.reference @@ -0,0 +1 @@ +4 1 diff --git a/tests/queries/0_stateless/03169_time_virtual_column.sh b/tests/queries/0_stateless/03169_time_virtual_column.sh new file mode 100755 index 00000000000..fef1de8c6f2 --- /dev/null +++ b/tests/queries/0_stateless/03169_time_virtual_column.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +sleep 1 +$CLICKHOUSE_LOCAL -nm -q " +select _size, (dateDiff('millisecond', _time, now()) < 600000 AND dateDiff('millisecond', _time, now()) > 0) from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv'); +" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv diff --git a/tests/queries/0_stateless/03170_float_schema_inference_small_block.reference b/tests/queries/0_stateless/03170_float_schema_inference_small_block.reference new file mode 100644 index 00000000000..9ee16da8728 --- /dev/null +++ b/tests/queries/0_stateless/03170_float_schema_inference_small_block.reference @@ -0,0 +1,15 @@ +Int64 +x Nullable(Int64) +x Nullable(Int64) +x Nullable(Int64) +Float64 +x Nullable(Float64) +x Nullable(Float64) +x Nullable(Float64) +x Nullable(Float64) +Float64.explicit File +x Nullable(Float64) +Float64.pipe +x Nullable(Float64) +Float64.default max_read_buffer_size +x Nullable(Float64) diff --git a/tests/queries/0_stateless/03170_float_schema_inference_small_block.sh b/tests/queries/0_stateless/03170_float_schema_inference_small_block.sh new file mode 100755 index 00000000000..88f9bfad7ed --- /dev/null +++ b/tests/queries/0_stateless/03170_float_schema_inference_small_block.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# do not fallback to float always +echo "Int64" +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : 1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : +1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : -1}' + +echo "Float64" +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : 1.1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : +1.1}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : 1.111}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' <<<'{"x" : +1.111}' + +# this is requried due to previously clickhouse-local does not interprets +# --max_read_buffer_size for fds [1] +# +# [1]: https://github.com/ClickHouse/ClickHouse/pull/64532 +echo "Float64.explicit File" +tmp_path=$(mktemp "$CUR_DIR/03170_float_schema_inference_small_block.json.XXXXXX") +trap 'rm -f $tmp_path' EXIT +cat > "$tmp_path" <<<'{"x" : 1.111}' +$CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' --file "$tmp_path" + +echo "Float64.pipe" +echo '{"x" : 1.1}' | $CLICKHOUSE_LOCAL --storage_file_read_method read --max_read_buffer_size 1 --input-format JSONEachRow 'desc "table"' +echo "Float64.default max_read_buffer_size" +echo '{"x" : 1.1}' | $CLICKHOUSE_LOCAL --storage_file_read_method read --input-format JSONEachRow 'desc "table"' diff --git a/tests/queries/0_stateless/03170_part_offset_as_table_column.reference b/tests/queries/0_stateless/03170_part_offset_as_table_column.reference new file mode 100644 index 00000000000..435187cb39b --- /dev/null +++ b/tests/queries/0_stateless/03170_part_offset_as_table_column.reference @@ -0,0 +1,30 @@ +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 diff --git a/tests/queries/0_stateless/03170_part_offset_as_table_column.sql b/tests/queries/0_stateless/03170_part_offset_as_table_column.sql new file mode 100644 index 00000000000..36cbc156744 --- /dev/null +++ b/tests/queries/0_stateless/03170_part_offset_as_table_column.sql @@ -0,0 +1,25 @@ +CREATE TABLE test_table +( + `key` UInt32, + `_part_offset` DEFAULT 0 +) +ENGINE = MergeTree +ORDER BY key; + +INSERT INTO test_table (key) SELECT number +FROM numbers(10); + +set allow_experimental_analyzer=0; + +SELECT * +FROM test_table; + +set allow_experimental_analyzer=1; + +SELECT * +FROM test_table; + +SELECT + key, + _part_offset +FROM test_table; diff --git a/tests/queries/0_stateless/03171_condition_pushdown.reference b/tests/queries/0_stateless/03171_condition_pushdown.reference new file mode 100644 index 00000000000..c7e6479af3b --- /dev/null +++ b/tests/queries/0_stateless/03171_condition_pushdown.reference @@ -0,0 +1,2 @@ +123 123 +456 456 diff --git a/tests/queries/0_stateless/03171_condition_pushdown.sql b/tests/queries/0_stateless/03171_condition_pushdown.sql new file mode 100644 index 00000000000..9cfe41ce921 --- /dev/null +++ b/tests/queries/0_stateless/03171_condition_pushdown.sql @@ -0,0 +1,6 @@ +-- This query succeeds only if it is correctly optimized. +SET allow_experimental_analyzer = 1; +SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (123, 456) AND t2.number = t1.number ORDER BY ALL; + +-- Still TODO: +-- SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (SELECT 123 UNION ALL SELECT 456) AND t2.number = t1.number ORDER BY ALL; diff --git a/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.reference b/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql b/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql new file mode 100644 index 00000000000..034d48d2b8b --- /dev/null +++ b/tests/queries/0_stateless/03171_direct_dict_short_circuit_bug.sql @@ -0,0 +1,37 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/65201 +SET short_circuit_function_evaluation='enable'; + +DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes; +DROP TABLE IF EXISTS simple_key_simple_attributes_source_table; + +CREATE TABLE simple_key_simple_attributes_source_table +( + id UInt64, + value_first String, + value_second String +) + ENGINE = TinyLog; + +INSERT INTO simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + + +CREATE DICTIONARY direct_dictionary_simple_key_simple_attributes +( + `id` UInt64, + `value_first` String DEFAULT 'value_first_default', + `value_second` String DEFAULT 'value_second_default' +) + PRIMARY KEY id + SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table')) + LAYOUT(DIRECT()); + +SELECT + toUInt128(1), + dictGetOrDefault('direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString(toFixedString(toFixedString(toFixedString(materialize(toNullable('default')), 7), 7), toNullable(toNullable(toNullable(toUInt128(7))))))) AS value_second +FROM system.numbers LIMIT 255 +FORMAT Null; + +DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes; +DROP TABLE IF EXISTS simple_key_simple_attributes_source_table; diff --git a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference new file mode 100644 index 00000000000..a2ac115060f --- /dev/null +++ b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.reference @@ -0,0 +1,6 @@ +100 1 1 +300 3 0 +200 2 2 +100 1 1 +300 3 0 +200 2 2 diff --git a/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql new file mode 100644 index 00000000000..e1b5531a442 --- /dev/null +++ b/tests/queries/0_stateless/03171_hashed_dictionary_short_circuit_bug_fix.sql @@ -0,0 +1,30 @@ +-- Tags: no-parallel + +CREATE TABLE x ( hash_id UInt64, user_result Decimal(3, 2) ) ENGINE = Memory(); + +CREATE TABLE y ( hash_id UInt64, user_result DECIMAL(18, 6) ) ENGINE = Memory(); + +INSERT INTO x values (100, 1), (200, 2); +INSERT INTO y values (100, 1), (300, 3), (200, 2); + +CREATE DICTIONARY d1 (hash_id UInt64, user_result Decimal(3, 2) ) +PRIMARY KEY hash_id +SOURCE(CLICKHOUSE(TABLE 'x')) +LIFETIME(0) +LAYOUT(HASHED()); + +SELECT hash_id, + dictGetOrDefault(d1, 'user_result', toUInt64(hash_id), toFloat64(user_result)), + dictGet(d1, 'user_result', toUInt64(hash_id)) +FROM y; + +CREATE DICTIONARY d2 (hash_id UInt64, user_result Decimal(3, 2) ) +PRIMARY KEY hash_id +SOURCE(CLICKHOUSE(TABLE 'x')) +LIFETIME(0) +LAYOUT(HASHED_ARRAY()); + +SELECT hash_id, + dictGetOrDefault(d2, 'user_result', toUInt64(hash_id), toFloat64(user_result)), + dictGet(d2, 'user_result', toUInt64(hash_id)) +FROM y; diff --git a/tests/queries/0_stateless/03172_bcrypt_validation.reference b/tests/queries/0_stateless/03172_bcrypt_validation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03172_bcrypt_validation.sql b/tests/queries/0_stateless/03172_bcrypt_validation.sql new file mode 100644 index 00000000000..37dd0c9bb5d --- /dev/null +++ b/tests/queries/0_stateless/03172_bcrypt_validation.sql @@ -0,0 +1,3 @@ +-- Tags: no-fasttest +DROP USER IF EXISTS 03172_user_invalid_bcrypt_hash; +CREATE USER 03172_user_invalid_bcrypt_hash IDENTIFIED WITH bcrypt_hash BY '012345678901234567890123456789012345678901234567890123456789'; -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/03173_distinct_combinator_alignment.reference b/tests/queries/0_stateless/03173_distinct_combinator_alignment.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03173_distinct_combinator_alignment.sql b/tests/queries/0_stateless/03173_distinct_combinator_alignment.sql new file mode 100644 index 00000000000..4a066be5086 --- /dev/null +++ b/tests/queries/0_stateless/03173_distinct_combinator_alignment.sql @@ -0,0 +1 @@ +SELECT toTypeName(topKDistinctState(toNullable(10))(toString(number)) IGNORE NULLS) FROM numbers(100) GROUP BY tuple((map((materialize(toNullable(1)), 2), 4, (3, 4), 5), 3)), map((1, 2), 4, (3, 4), toNullable(5)) WITH CUBE WITH TOTALS FORMAT Null diff --git a/tests/queries/0_stateless/03173_forbid_qualify.reference b/tests/queries/0_stateless/03173_forbid_qualify.reference new file mode 100644 index 00000000000..c2f595d8c4b --- /dev/null +++ b/tests/queries/0_stateless/03173_forbid_qualify.reference @@ -0,0 +1,3 @@ +100 +49 +100 diff --git a/tests/queries/0_stateless/03173_forbid_qualify.sql b/tests/queries/0_stateless/03173_forbid_qualify.sql new file mode 100644 index 00000000000..d8cb2bad2ea --- /dev/null +++ b/tests/queries/0_stateless/03173_forbid_qualify.sql @@ -0,0 +1,11 @@ +drop table if exists test_qualify; +create table test_qualify (number Int64) ENGINE = MergeTree ORDER BY (number); + +insert into test_qualify SELECT * FROM numbers(100); + +select count() from test_qualify; -- 100 +select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 1; -- 49 +select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } + +delete from test_qualify where number in (select number from test_qualify qualify row_number() over (order by number) = 50); -- { serverError UNFINISHED } +select count() from test_qualify; -- 100 diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 5c8589dfca1..ac0896f2e93 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -8,7 +8,7 @@ sys.path.insert(0, os.path.join(CURDIR)) import uexpect -prompt = ":\) " +prompt = ":\\) " end_of_block = r".*\r\n.*\r\n" @@ -21,7 +21,7 @@ class client(object): self.client.eol("\r") self.client.logger(log, prefix=name) self.client.timeout(120) - self.client.expect("[#\$] ", timeout=60) + self.client.expect("[#\\$] ", timeout=60) self.client.send(command) def __enter__(self): diff --git a/tests/queries/0_stateless/helpers/shell.py b/tests/queries/0_stateless/helpers/shell.py index befb3dcd543..c3fff61ffc9 100644 --- a/tests/queries/0_stateless/helpers/shell.py +++ b/tests/queries/0_stateless/helpers/shell.py @@ -10,7 +10,7 @@ import uexpect class shell(object): - def __init__(self, command=None, name="", log=None, prompt="[#\$] "): + def __init__(self, command=None, name="", log=None, prompt="[#\\$] "): if command is None: command = ["/bin/bash", "--noediting"] self.prompt = prompt diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index b11b2e6b852..9eeea87b52d 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -37,7 +37,7 @@ function wait_for_all_mutations() echo "Timed out while waiting for mutation to execute!" fi - sleep 0.1 + sleep 0.3 done } diff --git a/tests/queries/1_stateful/00166_explain_estimate.sql b/tests/queries/1_stateful/00166_explain_estimate.sql index c4071271736..136433c16ee 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.sql +++ b/tests/queries/1_stateful/00166_explain_estimate.sql @@ -1,6 +1,8 @@ -- Tags: no-replicated-database -- Tag no-replicated-database: Requires investigation +SET optimize_use_implicit_projections = 0; + EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID = 29103473; EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID != 29103473; EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID > 29103473; diff --git a/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql b/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql index 63eca96414f..5733bf6dd18 100644 --- a/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql +++ b/tests/queries/1_stateful/00175_counting_resources_in_subqueries.sql @@ -1,3 +1,5 @@ +SET optimize_use_implicit_projections = 0; + -- the work for scalar subquery is properly accounted: SET max_rows_to_read = 1000000; SELECT 1 = (SELECT count() FROM test.hits WHERE NOT ignore(AdvEngineID)); -- { serverError TOO_MANY_ROWS } diff --git a/tests/sqllogic/test_parser.py b/tests/sqllogic/test_parser.py index c0abcaecd25..bd30674b23a 100755 --- a/tests/sqllogic/test_parser.py +++ b/tests/sqllogic/test_parser.py @@ -526,7 +526,7 @@ class QueryResult: for row in rows: res_row = [] for c, t in zip(row, types): - logger.debug("Builging row. c:%s t:%s", c, t) + logger.debug("Building row. c:%s t:%s", c, t) if c is None: res_row.append("NULL") continue diff --git a/utils/backupview/clickhouse_backupview.py b/utils/backupview/clickhouse_backupview.py index 4ba1f391d02..d1331e2ab49 100755 --- a/utils/backupview/clickhouse_backupview.py +++ b/utils/backupview/clickhouse_backupview.py @@ -8,6 +8,7 @@ import shutil import zipfile # For reading backups from zip archives import boto3 # For reading backups from S3 +import botocore ## Examples: diff --git a/utils/changelog/README.md b/utils/changelog/README.md index ccc235c4990..4b16c39a3fe 100644 --- a/utils/changelog/README.md +++ b/utils/changelog/README.md @@ -6,7 +6,7 @@ Generate github token: Dependencies: ``` sudo apt-get update -sudo apt-get install git python3 python3-fuzzywuzzy python3-github +sudo apt-get install git python3 python3-thefuzz python3-github python3 changelog.py -h ``` @@ -15,10 +15,7 @@ Usage example: Note: The working directory is ClickHouse/utils/changelog ```bash -export GITHUB_TOKEN="" - -git fetch --tags # changelog.py depends on having the tags available, this will fetch them. - # If you are working from a branch in your personal fork, then you may need `git fetch --all` +GITHUB_TOKEN="" python3 changelog.py --output=changelog-v22.4.1.2305-prestable.md --gh-user-or-token="$GITHUB_TOKEN" v21.6.2.7-prestable python3 changelog.py --output=changelog-v22.4.1.2305-prestable.md --gh-user-or-token="$USER" --gh-password="$PASSWORD" v21.6.2.7-prestable diff --git a/utils/changelog/changelog.py b/utils/changelog/changelog.py index 304ab568e3c..b79e4139bcc 100755 --- a/utils/changelog/changelog.py +++ b/utils/changelog/changelog.py @@ -1,430 +1,15 @@ #!/usr/bin/env python3 # In our CI this script runs in style-test containers -import argparse -import logging -import os -import os.path as p -import re -from datetime import date, timedelta -from subprocess import DEVNULL, CalledProcessError -from typing import Dict, List, Optional, TextIO +# The main script is moved to tests/ci/changelog.py +# It depends on the ci scripts too hard to keep it here +# Here's only a wrapper around it for the people who used to it -from fuzzywuzzy.fuzz import ratio # type: ignore -from github.GithubException import RateLimitExceededException, UnknownObjectException -from github.NamedUser import NamedUser - -from git_helper import git_runner as runner -from git_helper import is_shallow -from github_helper import GitHub, PullRequest, PullRequests, Repository - -# This array gives the preferred category order, and is also used to -# normalize category names. -# Categories are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there -# updated accordingly -categories_preferred_order = ( - "Backward Incompatible Change", - "New Feature", - "Performance Improvement", - "Improvement", - "Critical Bug Fix", - "Bug Fix", - "Build/Testing/Packaging Improvement", - "Other", -) - -FROM_REF = "" -TO_REF = "" -SHA_IN_CHANGELOG = [] # type: List[str] -gh = GitHub(create_cache_dir=False) -CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") - - -class Description: - def __init__( - self, number: int, user: NamedUser, html_url: str, entry: str, category: str - ): - self.number = number - self.html_url = html_url - self.user = gh.get_user_cached(user._rawData["login"]) # type: ignore - self.entry = entry - self.category = category - - @property - def formatted_entry(self) -> str: - # Substitute issue links. - # 1) issue number w/o markdown link - entry = re.sub( - r"([^[])#([0-9]{4,})", - r"\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)", - self.entry, - ) - # 2) issue URL w/o markdown link - # including #issuecomment-1 or #event-12 - entry = re.sub( - r"([^(])(https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})[-#a-z0-9]*)", - r"\1[#\3](\2)", - entry, - ) - # It's possible that we face a secondary rate limit. - # In this case we should sleep until we get it - while True: - try: - user_name = self.user.name if self.user.name else self.user.login - break - except UnknownObjectException: - user_name = self.user.login - break - except RateLimitExceededException: - gh.sleep_on_rate_limit() - return ( - f"* {entry} [#{self.number}]({self.html_url}) " - f"([{user_name}]({self.user.html_url}))." - ) - - # Sort PR descriptions by numbers - def __eq__(self, other) -> bool: - if not isinstance(self, type(other)): - return NotImplemented - return self.number == other.number - - def __lt__(self, other: "Description") -> bool: - return self.number < other.number - - -def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: - descriptions = {} # type: Dict[str, List[Description]] - repos = {} # type: Dict[str, Repository] - for pr in prs: - # See https://github.com/PyGithub/PyGithub/issues/2202, - # obj._rawData doesn't spend additional API requests - # We'll save some requests - # pylint: disable=protected-access - repo_name = pr._rawData["base"]["repo"]["full_name"] # type: ignore - # pylint: enable=protected-access - if repo_name not in repos: - repos[repo_name] = pr.base.repo - in_changelog = False - merge_commit = pr.merge_commit_sha - try: - runner.run(f"git rev-parse '{merge_commit}'") - except CalledProcessError: - # It's possible that commit not in the repo, just continue - logging.info("PR %s does not belong to the repo", pr.number) - continue - - in_changelog = merge_commit in SHA_IN_CHANGELOG - if in_changelog: - desc = generate_description(pr, repos[repo_name]) - if desc: - if desc.category not in descriptions: - descriptions[desc.category] = [] - descriptions[desc.category].append(desc) - - for descs in descriptions.values(): - descs.sort() - - return descriptions - - -def parse_args() -> argparse.Namespace: - parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter, - description="Generate a changelog in Markdown format between given tags. " - "It fetches all tags and unshallow the git repository automatically", - ) - parser.add_argument( - "-v", - "--verbose", - action="count", - default=0, - help="set the script verbosity, could be used multiple", - ) - parser.add_argument( - "--debug-helpers", - action="store_true", - help="add debug logging for git_helper and github_helper", - ) - parser.add_argument( - "--output", - type=argparse.FileType("w"), - default="-", - help="output file for changelog", - ) - parser.add_argument( - "--repo", - default="ClickHouse/ClickHouse", - help="a repository to query for pull-requests from GitHub", - ) - parser.add_argument( - "--jobs", - type=int, - default=10, - help="number of jobs to get pull-requests info from GitHub API", - ) - parser.add_argument( - "--gh-user-or-token", - help="user name or GH token to authenticate", - ) - parser.add_argument( - "--gh-password", - help="a password that should be used when user is given", - ) - parser.add_argument( - "--with-testing-tags", - action="store_true", - help="by default '*-testing' tags are ignored, this argument enables them too", - ) - parser.add_argument( - "--from", - dest="from_ref", - help="git ref for a starting point of changelog, by default is calculated " - "automatically to match a previous tag in history", - ) - parser.add_argument( - "to_ref", - metavar="TO_REF", - help="git ref for the changelog end", - ) - args = parser.parse_args() - return args - - -# This function mirrors the PR description checks in ClickhousePullRequestTrigger. -# Returns None if the PR should not be mentioned in changelog. -def generate_description(item: PullRequest, repo: Repository) -> Optional[Description]: - backport_number = item.number - if item.head.ref.startswith("backport/"): - branch_parts = item.head.ref.split("/") - if len(branch_parts) == 3: - try: - item = gh.get_pull_cached(repo, int(branch_parts[-1])) - except Exception as e: - logging.warning("unable to get backpoted PR, exception: %s", e) - else: - logging.warning( - "The branch %s doesn't match backport template, using PR %s as is", - item.head.ref, - item.number, - ) - description = item.body - # Don't skip empty lines because they delimit parts of description - lines = [x.strip() for x in (description.split("\n") if description else [])] - lines = [re.sub(r"\s+", " ", ln) for ln in lines] - - category = "" - entry = "" - - if lines: - i = 0 - while i < len(lines): - if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): - i += 1 - if i >= len(lines): - break - # Can have one empty line between header and the category itself. - # Filter it out. - if not lines[i]: - i += 1 - if i >= len(lines): - break - category = re.sub(r"^[-*\s]*", "", lines[i]) - i += 1 - elif re.match( - r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] - ): - i += 1 - # Can have one empty line between header and the entry itself. - # Filter it out. - if i < len(lines) and not lines[i]: - i += 1 - # All following lines until empty one are the changelog entry. - entry_lines = [] - while i < len(lines) and lines[i]: - entry_lines.append(lines[i]) - i += 1 - entry = " ".join(entry_lines) - else: - i += 1 - - # Remove excessive bullets from the entry. - if re.match(r"^[\-\*] ", entry): - entry = entry[2:] - - # Better style. - if re.match(r"^[a-z]", entry): - entry = entry.capitalize() - - if not category: - # Shouldn't happen, because description check in CI should catch such PRs. - # Fall through, so that it shows up in output and the user can fix it. - category = "NO CL CATEGORY" - - # Filter out the PR categories that are not for changelog. - if re.match( - r"(?i)((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", - category, - ): - category = "NOT FOR CHANGELOG / INSIGNIFICANT" - return Description(item.number, item.user, item.html_url, item.title, category) - - # Normalize bug fixes - if re.match( - r"(?i)bug\Wfix", - category, - ): - category = "Bug Fix (user-visible misbehavior in an official stable release)" - - # Filter out documentations changelog - if re.match( - r"(?i)doc", - category, - ): - return None - - if backport_number != item.number: - entry = f"Backported in #{backport_number}: {entry}" - - if not entry: - # Shouldn't happen, because description check in CI should catch such PRs. - category = "NO CL ENTRY" - entry = "NO CL ENTRY: '" + item.title + "'" - - entry = entry.strip() - if entry[-1] != ".": - entry += "." - - for c in categories_preferred_order: - if ratio(category.lower(), c.lower()) >= 90: - category = c - break - - return Description(item.number, item.user, item.html_url, entry, category) - - -def write_changelog( - fd: TextIO, descriptions: Dict[str, List[Description]], year: int -) -> None: - to_commit = runner(f"git rev-parse {TO_REF}^{{}}")[:11] - from_commit = runner(f"git rev-parse {FROM_REF}^{{}}")[:11] - fd.write( - f"---\nsidebar_position: 1\nsidebar_label: {year}\n---\n\n" - f"# {year} Changelog\n\n" - f"### ClickHouse release {TO_REF} ({to_commit}) FIXME " - f"as compared to {FROM_REF} ({from_commit})\n\n" - ) - - seen_categories = [] # type: List[str] - for category in categories_preferred_order: - if category in descriptions: - seen_categories.append(category) - fd.write(f"#### {category}\n") - for desc in descriptions[category]: - fd.write(f"{desc.formatted_entry}\n") - - fd.write("\n") - - for category in sorted(descriptions): - if category not in seen_categories: - fd.write(f"#### {category}\n\n") - for desc in descriptions[category]: - fd.write(f"{desc.formatted_entry}\n") - - fd.write("\n") - - -def check_refs(from_ref: Optional[str], to_ref: str, with_testing_tags: bool): - global FROM_REF, TO_REF - TO_REF = to_ref - - # Check TO_REF - runner.run(f"git rev-parse {TO_REF}") - - # Check from_ref - if from_ref is None: - # Get all tags pointing to TO_REF - tags = runner.run(f"git tag --points-at '{TO_REF}^{{}}'").split("\n") - logging.info("All tags pointing to %s:\n%s", TO_REF, tags) - if not with_testing_tags: - tags.append("*-testing") - exclude = " ".join([f"--exclude='{tag}'" for tag in tags]) - cmd = f"git describe --abbrev=0 --tags {exclude} '{TO_REF}'" - FROM_REF = runner.run(cmd) - else: - runner.run(f"git rev-parse {FROM_REF}") - FROM_REF = from_ref - - -def set_sha_in_changelog(): - global SHA_IN_CHANGELOG - SHA_IN_CHANGELOG = runner.run( - f"git log --format=format:%H {FROM_REF}..{TO_REF}" - ).split("\n") - - -def get_year(prs: PullRequests) -> int: - if not prs: - return date.today().year - return max(pr.created_at.year for pr in prs) - - -def main(): - log_levels = [logging.WARN, logging.INFO, logging.DEBUG] - args = parse_args() - logging.basicConfig( - format="%(asctime)s %(levelname)-8s [%(filename)s:%(lineno)d]:\n%(message)s", - level=log_levels[min(args.verbose, 2)], - ) - if args.debug_helpers: - logging.getLogger("github_helper").setLevel(logging.DEBUG) - logging.getLogger("git_helper").setLevel(logging.DEBUG) - # Create a cache directory - if not p.isdir(CACHE_PATH): - os.mkdir(CACHE_PATH, 0o700) - - # Get the full repo - if is_shallow(): - logging.info("Unshallow repository") - runner.run("git fetch --unshallow", stderr=DEVNULL) - logging.info("Fetching all tags") - runner.run("git fetch --tags", stderr=DEVNULL) - - check_refs(args.from_ref, args.to_ref, args.with_testing_tags) - set_sha_in_changelog() - - logging.info("Using %s..%s as changelog interval", FROM_REF, TO_REF) - - # use merge-base commit as a starting point, if used ref in another branch - base_commit = runner.run(f"git merge-base '{FROM_REF}^{{}}' '{TO_REF}^{{}}'") - # Get starting and ending dates for gathering PRs - # Add one day after and before to mitigate TZ possible issues - # `tag^{}` format gives commit ref when we have annotated tags - # format %cs gives a committer date, works better for cherry-picked commits - from_date = runner.run(f"git log -1 --format=format:%cs '{base_commit}'") - to_date = runner.run(f"git log -1 --format=format:%cs '{TO_REF}^{{}}'") - merged = ( - date.fromisoformat(from_date) - timedelta(1), - date.fromisoformat(to_date) + timedelta(1), - ) - - # Get all PRs for the given time frame - global gh - gh = GitHub( - args.gh_user_or_token, - args.gh_password, - create_cache_dir=False, - per_page=100, - pool_size=args.jobs, - ) - gh.cache_path = CACHE_PATH - query = f"type:pr repo:{args.repo} is:merged" - prs = gh.get_pulls_from_search(query=query, merged=merged, sort="created") - - descriptions = get_descriptions(prs) - changelog_year = get_year(prs) - - write_changelog(args.output, descriptions, changelog_year) +import subprocess +import sys +from pathlib import Path +SCRIPT_PATH = (Path(__file__).parents[2] / "tests/ci/changelog.py").absolute() if __name__ == "__main__": - main() + subprocess.check_call(["python3", SCRIPT_PATH, *sys.argv[1:]]) diff --git a/utils/changelog/git_helper.py b/utils/changelog/git_helper.py deleted file mode 120000 index 03b05a7eddd..00000000000 --- a/utils/changelog/git_helper.py +++ /dev/null @@ -1 +0,0 @@ -../../tests/ci/git_helper.py \ No newline at end of file diff --git a/utils/changelog/github_helper.py b/utils/changelog/github_helper.py deleted file mode 120000 index 2d44dfe8000..00000000000 --- a/utils/changelog/github_helper.py +++ /dev/null @@ -1 +0,0 @@ -../../tests/ci/github_helper.py \ No newline at end of file diff --git a/utils/changelog/requirements.txt b/utils/changelog/requirements.txt index 106e9e2c72d..53c3bf3206e 100644 --- a/utils/changelog/requirements.txt +++ b/utils/changelog/requirements.txt @@ -1,3 +1,2 @@ -fuzzywuzzy +thefuzz PyGitHub -python-Levenshtein diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 873406358d4..a2a2fcd967a 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -222,6 +222,7 @@ DatabaseOrdinaryThreadsActive DateTime DateTimes DbCL +deallocated Decrypted Deduplicate Deduplication @@ -293,6 +294,7 @@ FilesystemMainPathUsedBytes FilesystemMainPathUsedINodes FixedString FlameGraph +flameGraph Flink ForEach FreeBSD @@ -446,6 +448,7 @@ KafkaLibrdkafkaThreads KafkaProducers KafkaWrites Kahan +Kaser KeeperAliveConnections KeeperMap KeeperOutstandingRequets @@ -466,11 +469,13 @@ LOCALTIME LOCALTIMESTAMP LONGLONG LOONGARCH +Lemir Levenshtein Liao LibFuzzer LightHouse LineAsString +LineString Linf LinfDistance LinfNorm @@ -958,6 +963,7 @@ ToGeoBoundary ToIPv ToParent ToSnowflake +ToSnowflakeID ToString ToUnicode Toolset @@ -1006,6 +1012,8 @@ Updatable Uppercased Uptime Uptrace +UrlDecode +UrlEncode UserID Util VARCHAR @@ -1451,6 +1459,7 @@ datatypes dateName dateTime dateTimeToSnowflake +dateTimeToSnowflakeID datetime datetimes dayofyear @@ -1534,6 +1543,7 @@ dumpColumnStructure durations ecto editDistance +editDistanceUTF embeddings emptyArray emptyArrayDate @@ -1699,6 +1709,7 @@ groupBitmap groupBitmapAnd groupBitmapOr groupBitmapXor +groupConcat groupUniqArray grouparray grouparrayinsertat @@ -1715,6 +1726,7 @@ groupbitmapor groupbitmapxor groupbitor groupbitxor +groupconcat groupuniqarray grpc grpcio @@ -1891,6 +1903,7 @@ lessOrEquals lessorequals levenshtein levenshteinDistance +levenshteinDistanceUTF lexicographically lgamma libFuzzer @@ -1932,6 +1945,8 @@ london lookups loongarch lowcardinality +lowCardinalityIndices +lowCardinalityKeys lowerUTF lowercased lttb @@ -1998,6 +2013,8 @@ minmax mins misconfiguration mispredictions +mlock +mlockall mmap mmapped modularization @@ -2150,6 +2167,9 @@ parseDateTimeInJodaSyntaxOrNull parseDateTimeInJodaSyntaxOrZero parseDateTimeOrNull parseDateTimeOrZero +parseReadableSize +parseReadableSizeOrNull +parseReadableSizeOrZero parseTimeDelta parseable parsers @@ -2303,6 +2323,7 @@ randomStringUTF rankCorr rapidjson rawblob +readWKTLineString readWKTMultiPolygon readWKTPoint readWKTPolygon @@ -2461,6 +2482,7 @@ skewpop skewsamp skippingerrors sleepEachRow +snowflakeIDToDateTime snowflakeToDateTime socketcache soundex diff --git a/utils/check-style/check-flake8 b/utils/check-style/check-flake8 new file mode 100755 index 00000000000..58dd8a99d40 --- /dev/null +++ b/utils/check-style/check-flake8 @@ -0,0 +1,55 @@ +#!/usr/bin/env bash + +function join_by() { local IFS="$1"; shift; echo "$*"; } + +set -e + +# We check only our code, that's why we skip contrib +GIT_ROOT=$(git rev-parse --show-cdup) +GIT_ROOT=${GIT_ROOT:-./} + +# Find all *.py, *.python files and executable files without extension +# that are determined as python scripts by 'file' util +# in the repo except the contrib directory. +find_cmd=( + find "$GIT_ROOT" -type f -not -path "${GIT_ROOT}contrib/*" + \( + \( + -name '*.py' -or -name "*.python" -or + \( + -executable -not -name "*.*" -exec sh -c 'file {} | grep -q "Python script"' \; + \) + \) + # We skip modules generated by the protocol buffer compiler from *.proto files. + -and -not -name '*_pb2.py' -and -not -name '*_pb2_grpc.py' + \) -print0 +) + +ignores=( + E101 # Indentation contains mixed spaces and tabs + E203 # Whitespace before ':' + E226 # missing whitespace around arithmetic operator + E266 # Too many leading '#' for block comment + E401 # Multiple imports on one line + E402 # Module level import not at top of file + E501 # line too long + E711 # Comparison to None should be 'cond is None:' + E712 # Comparison to true should be 'if cond is true:' or 'if cond:' + E713 # Test for membership should be 'not in' + E714 # Test for object identity should be 'is not' + E722 # Do not use bare except, specify exception instead + E731 # Do not assign a lambda expression, use a def + E741 # Do not use variables named 'I', 'O', or 'l' + F401 # Module imported but unused + F403 # 'from module import *' used; unable to detect undefined names + F405 # Name may be undefined, or defined from star imports: module + F522 # .format(...) unused named arguments + F541 # f-string without any placeholders + F811 # redefinition of unused name from line N + F841 # local variable name is assigned to but never used + W191 # Indentation contains tabs + W291 # Trailing whitespace + W293 # Blank line contains whitespace + W503 # Line break occurred before a binary operator +) +"${find_cmd[@]}" | xargs -0 flake8 --ignore "$(join_by , "${ignores[@]}")" diff --git a/utils/check-style/check-mypy b/utils/check-style/check-mypy index 42cb7fbbd15..4434377e627 100755 --- a/utils/check-style/check-mypy +++ b/utils/check-style/check-mypy @@ -11,13 +11,15 @@ GIT_ROOT=${GIT_ROOT:-.} CONFIG="$GIT_ROOT/tests/ci/.mypy.ini" DIRS=("$GIT_ROOT/tests/ci/" "$GIT_ROOT/tests/ci/"*/) tmp=$(mktemp) + for dir in "${DIRS[@]}"; do if ! compgen -G "$dir"/*.py > /dev/null; then continue fi - if ! mypy --config-file="$CONFIG" --sqlite-cache "$dir"/*.py > "$tmp" 2>&1; then + if ! mypy --config-file="$CONFIG" --sqlite-cache $(find "$dir" -maxdepth 1 -name "*.py" | grep -v "test_") > "$tmp" 2>&1; then echo "Errors while processing $dir": cat "$tmp" fi done + rm -rf "$tmp" diff --git a/utils/check-style/check-pylint b/utils/check-style/check-pylint index 7959a414023..8cfbc68ac96 100755 --- a/utils/check-style/check-pylint +++ b/utils/check-style/check-pylint @@ -10,6 +10,7 @@ function xargs-pylint { xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/pyproject.toml" --persistent=no --score=n } -find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 +# exclude ci unittest scripts from check: test_* +find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | grep -v "/test_" | xargs-pylint 50 # Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 23e8b6b2bc4..7f25ca4607c 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -65,6 +65,7 @@ EXTERN_TYPES_EXCLUDES=( ProfileEvents::increment ProfileEvents::incrementForLogMessage ProfileEvents::getName + ProfileEvents::Timer ProfileEvents::Type ProfileEvents::TypeEnum ProfileEvents::dumpToMapColumn @@ -240,8 +241,22 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do esac done -# All the submodules should be from https://github.com/ -find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | grep -v -F 'https://github.com/' && echo 'All the submodules should be from https://github.com/'; done +# All submodules should be from https://github.com/ +git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.url' | \ +while read -r line; do + name=${line#submodule.}; name=${name%.url*} + url=${line#* } + [[ "$url" != 'https://github.com/'* ]] && echo "All submodules should be from https://github.com/, submodule '$name' has '$url'" +done + +# All submodules should be of this form: [submodule "contrib/libxyz"] (for consistency, the submodule name does matter too much) +# - restrict the check to top-level .gitmodules file +git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.path' | \ +while read -r line; do + name=${line#submodule.}; name=${name%.path*} + path=${line#* } + [ "$name" != "$path" ] && echo "Submodule name '$name' is not equal to it's path '$path'" +done # There shouldn't be any code snippets under GPL or LGPL find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" @@ -290,8 +305,6 @@ std_cerr_cout_excludes=( /examples/ /tests/ _fuzzer - # DUMP() - base/base/iostream_debug_helpers.h # OK src/Common/ProgressIndication.cpp # only under #ifdef DBMS_HASH_MAP_DEBUG_RESIZES, that is used only in tests @@ -311,6 +324,7 @@ std_cerr_cout_excludes=( src/Bridge/IBridge.cpp src/Daemon/BaseDaemon.cpp src/Loggers/Loggers.cpp + src/Common/GWPAsan.cpp ) sources_with_std_cerr_cout=( $( find $ROOT_PATH/{src,base} -name '*.h' -or -name '*.cpp' | \ diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index e603084732d..2c349114a59 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -18,6 +18,7 @@ def process_result(result_folder): "style", "pylint", "black", + "flake8", "mypy", "typos", "whitespaces", diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f7d84cce4b1..1bff4cb0b09 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,9 @@ +v24.5.3.5-stable 2024-06-13 +v24.5.2.34-stable 2024-06-13 v24.5.1.1763-stable 2024-06-01 +v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 v24.3.1.2672-lts 2024-03-27 @@ -37,6 +41,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 v23.8.12.13-lts 2024-03-26 diff --git a/utils/security-generator/generate_security.py b/utils/security-generator/generate_security.py index 2b37e28257a..21c6b72e476 100755 --- a/utils/security-generator/generate_security.py +++ b/utils/security-generator/generate_security.py @@ -98,7 +98,7 @@ def generate_supported_versions() -> str: lts.append(version) to_append = f"| {version} | ✔️ |" if to_append: - if len(regular) == max_regular or len(lts) == max_lts: + if len(regular) == max_regular and len(lts) == max_lts: supported_year = year table.append(to_append) continue