Merge branch 'master' into limit_dns_retries

This commit is contained in:
mergify[bot] 2022-04-06 18:23:17 +00:00 committed by GitHub
commit 585cff1e95
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
119 changed files with 2461 additions and 496 deletions

View File

@ -164,6 +164,8 @@ Checks: '-*,
clang-analyzer-unix.cstring.NullArg,
boost-use-to-string,
alpha.security.cert.env.InvalidPtr,
'
WarningsAsErrors: '*'

View File

@ -341,10 +341,15 @@ jobs:
steps:
- name: Set envs
run: |
DEPENDENCIES=$(cat << 'EOF' | jq '. | length'
${{ toJSON(needs) }}
EOF
)
echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV"
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/report_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=ClickHouse build check (actions)
REPORTS_PATH=${{runner.temp}}/reports_dir
TEMP_PATH=${{runner.temp}}/report_check
EOF
- name: Download json reports
uses: actions/download-artifact@v2
@ -360,7 +365,7 @@ jobs:
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cd "$GITHUB_WORKSPACE/tests/ci"
python3 build_report_check.py "$CHECK_NAME"
python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES"
- name: Cleanup
if: always()
run: |

View File

@ -992,10 +992,16 @@ jobs:
steps:
- name: Set envs
run: |
DEPENDENCIES=$(cat << 'EOF' | jq '. | length'
${{ toJSON(needs) }}
EOF
)
echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV"
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/report_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=ClickHouse build check (actions)
REPORTS_PATH=${{runner.temp}}/reports_dir
REPORTS_PATH=${{runner.temp}}/reports_dir
TEMP_PATH=${{runner.temp}}/report_check
EOF
- name: Download json reports
uses: actions/download-artifact@v2
@ -1011,7 +1017,7 @@ jobs:
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cd "$GITHUB_WORKSPACE/tests/ci"
python3 build_report_check.py "$CHECK_NAME"
python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES"
- name: Cleanup
if: always()
run: |

View File

@ -72,3 +72,52 @@ jobs:
with:
name: changed_images
path: ${{ runner.temp }}/changed_images.json
BuilderCoverity:
needs: DockerHubPush
runs-on: [self-hosted, builder]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/build_check
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
CHECK_NAME=ClickHouse build check (actions)
BUILD_NAME=coverity
EOF
- name: Download changed images
uses: actions/download-artifact@v2
with:
name: changed_images
path: ${{ env.IMAGES_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
id: coverity-checkout
uses: actions/checkout@v2
with:
submodules: 'true'
fetch-depth: 0 # otherwise we will have no info about contributors
- name: Build
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" "${{ secrets.COV_TOKEN }}"
- name: Upload Coverity Analysis
if: ${{ success() || failure() }}
run: |
curl --form token='${{ secrets.COV_TOKEN }}' \
--form email='${{ secrets.ROBOT_CLICKHOUSE_EMAIL }}' \
--form file="@$TEMP_PATH/$BUILD_NAME/clickhouse-scan.tgz" \
--form version="${GITHUB_REF#refs/heads/}-${GITHUB_SHA::6}" \
--form description="Nighly Scan: $(date +'%Y-%m-%dT%H:%M:%S')" \
https://scan.coverity.com/builds?project=ClickHouse%2FClickHouse
- name: Cleanup
if: always()
run: |
docker kill "$(docker ps -q)" ||:
docker rm -f "$(docker ps -a -q)" ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"

View File

@ -1044,10 +1044,16 @@ jobs:
steps:
- name: Set envs
run: |
DEPENDENCIES=$(cat << 'EOF' | jq '. | length'
${{ toJSON(needs) }}
EOF
)
echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV"
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/report_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=ClickHouse build check (actions)
REPORTS_PATH=${{runner.temp}}/reports_dir
REPORTS_PATH=${{runner.temp}}/reports_dir
TEMP_PATH=${{runner.temp}}/report_check
EOF
- name: Download json reports
uses: actions/download-artifact@v2
@ -1063,7 +1069,7 @@ jobs:
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cd "$GITHUB_WORKSPACE/tests/ci"
python3 build_report_check.py "$CHECK_NAME"
python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES"
- name: Cleanup
if: always()
run: |

View File

@ -436,10 +436,16 @@ jobs:
steps:
- name: Set envs
run: |
DEPENDENCIES=$(cat << 'EOF' | jq '. | length'
${{ toJSON(needs) }}
EOF
)
echo "DEPENDENCIES=$DEPENDENCIES" >> "$GITHUB_ENV"
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/report_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=ClickHouse build check (actions)
REPORTS_PATH=${{runner.temp}}/reports_dir
REPORTS_PATH=${{runner.temp}}/reports_dir
TEMP_PATH=${{runner.temp}}/report_check
EOF
- name: Download json reports
uses: actions/download-artifact@v2
@ -455,7 +461,7 @@ jobs:
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cd "$GITHUB_WORKSPACE/tests/ci"
python3 build_report_check.py "$CHECK_NAME"
python3 build_report_check.py "$CHECK_NAME" "$DEPENDENCIES"
- name: Cleanup
if: always()
run: |

View File

@ -294,14 +294,19 @@ include(cmake/cpu_features.cmake)
# Enable it explicitly.
set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables")
# Reproducible builds
# If turned `ON`, remap file source paths in debug info, predefined preprocessor macros and __builtin_FILE().
option(ENABLE_BUILD_PATH_MAPPING "Enable remap file source paths in debug info, predefined preprocessor macros and __builtin_FILE(). It's to generate reproducible builds. See https://reproducible-builds.org/docs/build-path" ON)
# Reproducible builds.
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")
set (ENABLE_BUILD_PATH_MAPPING_DEFAULT OFF)
else ()
set (ENABLE_BUILD_PATH_MAPPING_DEFAULT ON)
endif ()
option (ENABLE_BUILD_PATH_MAPPING "Enable remapping of file source paths in debug info, predefined preprocessor macros, and __builtin_FILE(). It's used to generate reproducible builds. See https://reproducible-builds.org/docs/build-path" ${ENABLE_BUILD_PATH_MAPPING_DEFAULT})
if (ENABLE_BUILD_PATH_MAPPING)
set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.")
set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.")
endif()
endif ()
if (${CMAKE_VERSION} VERSION_LESS "3.12.4")
# CMake < 3.12 doesn't support setting 20 as a C++ standard version.

View File

@ -197,7 +197,6 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter(color_enabled);
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel);
logger.warning("Logging " + console_log_level_string + " to console");
log->setLevel(console_log_level);
split->addChannel(log, "console");
}

View File

@ -25,13 +25,21 @@ read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}"
env
cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" ..
if [ "coverity" == "$COMBINED_OUTPUT" ]
then
wget --post-data "token=$COV_TOKEN&project=ClickHouse%2FClickHouse" -qO- https://scan.coverity.com/download/linux64 | tar xz -C /opt/cov-analysis --strip-components 1
export PATH=$PATH:/opt/cov-analysis/bin
cov-configure --config ./coverity.config --template --comptype clangcc --compiler "$CC"
SCAN_WRAPPER="cov-build --config ./coverity.config --dir cov-int"
fi
cache_status
# clear cache stats
ccache --zero-stats ||:
# No quotes because I want it to expand to nothing if empty.
# shellcheck disable=SC2086
ninja $NINJA_FLAGS clickhouse-bundle
# shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty.
$SCAN_WRAPPER ninja $NINJA_FLAGS clickhouse-bundle
cache_status
@ -91,6 +99,12 @@ then
mv "$COMBINED_OUTPUT.tgz" /output
fi
if [ "coverity" == "$COMBINED_OUTPUT" ]
then
tar -cv -I pigz -f "coverity-scan.tgz" cov-int
mv "coverity-scan.tgz" /output
fi
# Also build fuzzers if any sanitizer specified
# if [ -n "$SANITIZER" ]
# then

View File

@ -86,6 +86,7 @@ def parse_env_variables(
additional_pkgs,
with_coverage,
with_binaries,
coverity_scan,
):
DARWIN_SUFFIX = "-darwin"
DARWIN_ARM_SUFFIX = "-darwin-aarch64"
@ -176,6 +177,9 @@ def parse_env_variables(
if package_type == "performance":
result.append("COMBINED_OUTPUT=performance")
cmake_flags.append("-DENABLE_TESTS=0")
elif package_type == "coverity":
result.append("COMBINED_OUTPUT=coverity")
result.append("COV_TOKEN={}".format(cov_token))
elif split_binary:
result.append("COMBINED_OUTPUT=shared_build")
@ -262,9 +266,8 @@ if __name__ == "__main__":
# and configs to be used for performance test.
parser.add_argument(
"--package-type",
choices=("deb", "binary", "performance"),
choices=["deb", "binary", "performance", "coverity"],
required=True,
help="a build type",
)
parser.add_argument(
"--clickhouse-repo-path",
@ -325,12 +328,13 @@ if __name__ == "__main__":
parser.add_argument(
"--docker-image-version", default="latest", help="docker image tag to use"
)
parser.add_argument("--cov_token", default="")
args = parser.parse_args()
if not os.path.isabs(args.output_dir):
args.output_dir = os.path.abspath(os.path.join(os.getcwd(), args.output_dir))
image_type = "binary" if args.package_type == "performance" else args.package_type
image_type = "binary" if args.package_type in ("performance", "coverity") else args.package_type
image_name = "clickhouse/binary-builder"
if not os.path.isabs(args.clickhouse_repo_path):
@ -372,6 +376,7 @@ if __name__ == "__main__":
args.additional_pkgs,
args.with_coverage,
args.with_binaries,
args.cov_token,
)
run_docker_image_with_env(

View File

@ -226,7 +226,6 @@ quit
--receive_data_timeout_ms=10000 \
--stacktrace \
--query-fuzzer-runs=1000 \
--testmode \
--queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \
$NEW_TESTS_OPT \
> >(tail -n 100000 > fuzzer.log) \

View File

@ -1,8 +1,10 @@
# docker build -t clickhouse/mysql-js-client .
# MySQL JavaScript client docker container
FROM node:8
FROM node:16.14.2
WORKDIR /usr/app
RUN npm install mysql
COPY ./test.js test.js
COPY ./test.js ./test.js

View File

@ -348,13 +348,13 @@ then
rm -f /test_output/tmp
# OOM
zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \
&& echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv
# Logical errors
echo "Check for Logical errors in server log:"
zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /test_output/bc_check_logical_errors.txt \
zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log* > /test_output/bc_check_logical_errors.txt \
&& echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv
@ -362,13 +362,13 @@ then
[ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt
# Crash
zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log* > /dev/null \
&& echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
echo "Check for Fatal message in server log:"
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log > /test_output/bc_check_fatal_messages.txt \
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log* > /test_output/bc_check_fatal_messages.txt \
&& echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv

View File

@ -43,7 +43,7 @@ toc_title: Adopters
| <a href="https://city-mobil.ru" class="favicon">Citymobil</a> | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) |
| <a href="https://cloudflare.com" class="favicon">Cloudflare</a> | CDN | Traffic analysis | 36 servers | — | [Blog post, May 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Blog post, March 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) |
| <a href="https://corporate.comcast.com/" class="favicon">Comcast</a> | Media | CDN Traffic Analysis | — | — | [ApacheCon 2019 Talk](https://www.youtube.com/watch?v=e9TZ6gFDjNg) |
| <a href="https://contentsquare.com" class="favicon">ContentSquare</a> | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) |
| <a href="https://contentsquare.com" class="favicon">Contentsquare</a> | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) |
| <a href="https://coru.net/" class="favicon">Corunet</a> | Analytics | Main product | — | — | [Slides in English, April 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) |
| <a href="https://www.creditx.com" class="favicon">CraiditX 氪信</a> | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) |
| <a href="https://crazypanda.ru/en/" class="favicon">Crazypanda</a> | Games | | — | — | Live session on ClickHouse meetup |

View File

@ -27,7 +27,7 @@ toc_title: "\u30A2\u30C0\u30D7\u30BF\u30FC"
| <a href="http://cisco.com/" class="favicon">Cisco</a> | ネットワーク | トラフィック分析 | — | — | [ライトニングトーク2019](https://youtu.be/-hI1vDR2oPY?t=5057) |
| <a href="https://www.citadelsecurities.com/" class="favicon">Citadel Securities</a> | 金融 | — | — | — | [2019年の貢献](https://github.com/ClickHouse/ClickHouse/pull/4774) |
| <a href="https://city-mobil.ru" class="favicon">シティモービル</a> | タクシー | 分析 | — | — | [ロシア語でのブログ投稿,月2020](https://habr.com/en/company/citymobil/blog/490660/) |
| <a href="https://contentsquare.com" class="favicon">ContentSquare</a> | ウェブ分析 | 主な製品 | — | — | [フランス語でのブログ投稿,November2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) |
| <a href="https://contentsquare.com" class="favicon">Contentsquare</a> | ウェブ分析 | 主な製品 | — | — | [フランス語でのブログ投稿,November2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) |
| <a href="https://cloudflare.com" class="favicon">Cloudflare</a> | CDN | トラフィック分析 | 36台のサーバー | — | [ブログ投稿,月2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [ブログ投稿,月2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) |
| <a href="https://coru.net/" class="favicon">コルネット</a> | 分析 | 主な製品 | — | — | [2019年英語スライド](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) |
| <a href="https://www.creditx.com" class="favicon">CraiditX 氪信</a> | ファイナンスAI | 分析 | — | — | [2019年のスライド](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) |

View File

@ -163,10 +163,24 @@ void Client::initialize(Poco::Util::Application & self)
configReadClient(config(), home_path);
/** getenv is thread-safe in Linux glibc and in all sane libc implementations.
* But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer.
*
* man getenv:
*
* As typically implemented, getenv() returns a pointer to a string within the environment list.
* The caller must take care not to modify this string, since that would change the environment of
* the process.
*
* The implementation of getenv() is not required to be reentrant. The string pointed to by the return value of getenv()
* may be statically allocated, and can be modified by a subsequent call to getenv(), putenv(3), setenv(3), or unsetenv(3).
*/
const char * env_user = getenv("CLICKHOUSE_USER");
const char * env_password = getenv("CLICKHOUSE_PASSWORD");
if (env_user)
config().setString("user", env_user);
const char * env_password = getenv("CLICKHOUSE_PASSWORD");
if (env_password)
config().setString("password", env_password);
@ -810,7 +824,7 @@ void Client::addOptions(OptionsDescription & options_description)
("quota_key", po::value<std::string>(), "A string to differentiate quotas when the user have keyed quotas configured on server")
("max_client_network_bandwidth", po::value<int>(), "the maximum speed of data exchange over the network for the client in bytes per second.")
("compression", po::value<bool>(), "enable or disable compression")
("compression", po::value<bool>(), "enable or disable compression (enabled by default for remote communication and disabled for localhost communication).")
("query-fuzzer-runs", po::value<int>()->default_value(0), "After executing every SELECT query, do random mutations in it and run again specified number of times. This is used for testing to discover unexpected corner cases.")
("interleave-queries-file", po::value<std::vector<std::string>>()->multitoken(),
@ -1005,6 +1019,7 @@ void Client::processConfig()
global_context->setCurrentQueryId(query_id);
}
print_stack_trace = config().getBool("stacktrace", false);
logging_initialized = true;
if (config().has("multiquery"))
is_multiquery = true;

View File

@ -434,6 +434,14 @@ catch (...)
return getCurrentExceptionCode();
}
void LocalServer::updateLoggerLevel(const String & logs_level)
{
if (!logging_initialized)
return;
config().setString("logger.level", logs_level);
updateLevels(config(), logger());
}
void LocalServer::processConfig()
{
@ -460,30 +468,31 @@ void LocalServer::processConfig()
auto logging = (config().has("logger.console")
|| config().has("logger.level")
|| config().has("log-level")
|| config().has("send_logs_level")
|| config().has("logger.log"));
auto file_logging = config().has("server_logs_file");
if (is_interactive && logging && !file_logging)
throw Exception("For interactive mode logging is allowed only with --server_logs_file option",
ErrorCodes::BAD_ARGUMENTS);
auto level = config().getString("log-level", "trace");
if (file_logging)
if (config().has("server_logs_file"))
{
auto level = Poco::Logger::parseLevel(config().getString("log-level", "trace"));
Poco::Logger::root().setLevel(level);
auto poco_logs_level = Poco::Logger::parseLevel(level);
Poco::Logger::root().setLevel(poco_logs_level);
Poco::Logger::root().setChannel(Poco::AutoPtr<Poco::SimpleFileChannel>(new Poco::SimpleFileChannel(server_logs_file)));
logging_initialized = true;
}
else if (logging)
else if (logging || is_interactive)
{
// force enable logging
config().setString("logger", "logger");
// sensitive data rules are not used here
auto log_level_default = is_interactive && !logging ? "none" : level;
config().setString("logger.level", config().getString("log-level", config().getString("send_logs_level", log_level_default)));
buildLoggers(config(), logger(), "clickhouse-local");
logging_initialized = true;
}
else
{
Poco::Logger::root().setLevel("none");
Poco::Logger::root().setChannel(Poco::AutoPtr<Poco::NullChannel>(new Poco::NullChannel()));
logging_initialized = false;
}
shared_context = Context::createShared();
@ -713,6 +722,8 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp
config().setString("logger.log", options["logger.log"].as<std::string>());
if (options.count("logger.level"))
config().setString("logger.level", options["logger.level"].as<std::string>());
if (options.count("send_logs_level"))
config().setString("send_logs_level", options["send_logs_level"].as<std::string>());
}
}

View File

@ -46,6 +46,8 @@ protected:
void processConfig() override;
void updateLoggerLevel(const String & logs_level) override;
private:
/** Composes CREATE subquery based on passed arguments (--structure --file --table and --input-format)
* This query will be executed first, before queries passed through --query argument

View File

@ -49,6 +49,18 @@ if (COMPILER_GCC)
add_definitions ("-fno-tree-loop-distribute-patterns")
endif ()
# ClickHouse developers may use platform-dependent code under some macro (e.g. `#ifdef ENABLE_MULTITARGET`).
# If turned ON, this option defines such macro.
# See `src/Common/TargetSpecific.h`
option(ENABLE_MULTITARGET_CODE "Enable platform-dependent code" ON)
if (ENABLE_MULTITARGET_CODE)
add_definitions(-DENABLE_MULTITARGET_CODE=1)
else()
add_definitions(-DENABLE_MULTITARGET_CODE=0)
endif()
add_subdirectory (Access)
add_subdirectory (Backups)
add_subdirectory (Columns)

View File

@ -1298,6 +1298,13 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
}
}
if (const auto * set_query = parsed_query->as<ASTSetQuery>())
{
const auto * logs_level_field = set_query->changes.tryGet(std::string_view{"send_logs_level"});
if (logs_level_field)
updateLoggerLevel(logs_level_field->safeGet<String>());
}
processed_rows = 0;
written_first_block = false;
progress_indication.resetProgress();
@ -1494,24 +1501,19 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
bool ClientBase::executeMultiQuery(const String & all_queries_text)
{
// It makes sense not to base any control flow on this, so that it is
// the same in tests and in normal usage. The only difference is that in
// normal mode we ignore the test hints.
const bool test_mode = config().has("testmode");
if (test_mode)
{
/// disable logs if expects errors
TestHint test_hint(test_mode, all_queries_text);
if (test_hint.clientError() || test_hint.serverError())
processTextAsSingleQuery("SET send_logs_level = 'fatal'");
}
bool echo_query = echo_queries;
/// Test tags are started with "--" so they are interpreted as comments anyway.
/// But if the echo is enabled we have to remove the test tags from `all_queries_text`
/// because we don't want test tags to be echoed.
size_t test_tags_length = test_mode ? getTestTagsLength(all_queries_text) : 0;
{
/// disable logs if expects errors
TestHint test_hint(all_queries_text);
if (test_hint.clientError() || test_hint.serverError())
processTextAsSingleQuery("SET send_logs_level = 'fatal'");
}
size_t test_tags_length = getTestTagsLength(all_queries_text);
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
@ -1548,7 +1550,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
// Try to find test hint for syntax error. We don't know where
// the query ends because we failed to parse it, so we consume
// the entire line.
TestHint hint(test_mode, String(this_query_begin, this_query_end - this_query_begin));
TestHint hint(String(this_query_begin, this_query_end - this_query_begin));
if (hint.serverError())
{
// Syntax errors are considered as client errors
@ -1586,7 +1588,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
// Look for the hint in the text of query + insert data + trailing
// comments, e.g. insert into t format CSV 'a' -- { serverError 123 }.
// Use the updated query boundaries we just calculated.
TestHint test_hint(test_mode, full_query);
TestHint test_hint(full_query);
// Echo all queries if asked; makes for a more readable reference file.
echo_query = test_hint.echoQueries().value_or(echo_query);
@ -2187,8 +2189,6 @@ void ClientBase::init(int argc, char ** argv)
("suggestion_limit", po::value<int>()->default_value(10000),
"Suggestion limit for how many databases, tables and columns to fetch.")
("testmode,T", "enable test hints in comments")
("format,f", po::value<std::string>(), "default output format")
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
("highlight", po::value<bool>()->default_value(true), "enable or disable basic syntax highlight in interactive command line")
@ -2294,8 +2294,6 @@ void ClientBase::init(int argc, char ** argv)
config().setBool("interactive", true);
if (options.count("pager"))
config().setString("pager", options["pager"].as<std::string>());
if (options.count("testmode"))
config().setBool("testmode", true);
if (options.count("log-level"))
Poco::Logger::root().setLevel(options["log-level"].as<std::string>());

View File

@ -95,6 +95,7 @@ protected:
std::optional<ProgramOptionsDescription> hosts_and_ports_description;
};
virtual void updateLoggerLevel(const String &) {}
virtual void printHelpMessage(const OptionsDescription & options_description) = 0;
virtual void addOptions(OptionsDescription & options_description) = 0;
virtual void processOptions(const OptionsDescription & options_description,
@ -265,6 +266,8 @@ protected:
bool allow_repeated_settings = false;
bool cancelled = false;
bool logging_initialized = false;
};
}

View File

@ -32,12 +32,9 @@ int parseErrorCode(DB::ReadBufferFromString & in)
namespace DB
{
TestHint::TestHint(bool enabled_, const String & query_)
TestHint::TestHint(const String & query_)
: query(query_)
{
if (!enabled_)
return;
// Don't parse error hints in leading comments, because it feels weird.
// Leading 'echo' hint is OK.
bool is_leading_hint = true;

View File

@ -7,7 +7,7 @@
namespace DB
{
/// Checks expected server and client error codes in --testmode.
/// Checks expected server and client error codes.
///
/// The following comment hints are supported:
///
@ -25,12 +25,12 @@ namespace DB
///
/// Examples:
///
/// - echo 'select / -- { clientError 62 }' | clickhouse-client --testmode -nm
/// - echo 'select / -- { clientError 62 }' | clickhouse-client -nm
///
// Here the client parses the query but it is incorrect, so it expects
/// SYNTAX_ERROR (62).
///
/// - echo 'select foo -- { serverError 47 }' | clickhouse-client --testmode -nm
/// - echo 'select foo -- { serverError 47 }' | clickhouse-client -nm
///
/// But here the query is correct, but there is no such column "foo", so it
/// is UNKNOWN_IDENTIFIER server error.
@ -43,7 +43,7 @@ namespace DB
class TestHint
{
public:
TestHint(bool enabled_, const String & query_);
TestHint(const String & query_);
int serverError() const { return server_error; }
int clientError() const { return client_error; }

View File

@ -81,6 +81,14 @@
M(ActiveSyncDrainedConnections, "Number of active connections drained synchronously.") \
M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \
M(PendingAsyncInsert, "Number of asynchronous inserts that are waiting for flush.") \
M(KafkaConsumers, "Number of active Kafka consumers") \
M(KafkaConsumersWithAssignment, "Number of active Kafka consumers which have some partitions assigned.") \
M(KafkaProducers, "Number of active Kafka producer created") \
M(KafkaLibrdkafkaThreads, "Number of active librdkafka threads") \
M(KafkaBackgroundReads, "Number of background reads currently working (populating materialized views from Kafka)") \
M(KafkaConsumersInUse, "Number of consumers which are currently used by direct or background reads") \
M(KafkaWrites, "Number of currently running inserts to Kafka") \
M(KafkaAssignedPartitions, "Number of partitions Kafka tables currently assigned to") \
namespace CurrentMetrics
{

View File

@ -112,6 +112,8 @@
M(CompileExpressionsMicroseconds, "Total time spent for compilation of expressions to LLVM code.") \
M(CompileExpressionsBytes, "Number of bytes used for expressions compilation.") \
\
M(ExecuteShellCommand, "Number of shell command executions.") \
\
M(ExternalSortWritePart, "") \
M(ExternalSortMerge, "") \
M(ExternalAggregationWritePart, "") \
@ -295,6 +297,25 @@
M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \
M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \
\
M(KafkaRebalanceRevocations, "Number of partition revocations (the first stage of consumer group rebalance)") \
M(KafkaRebalanceAssignments, "Number of partition assignments (the final stage of consumer group rebalance)") \
M(KafkaRebalanceErrors, "Number of failed consumer group rebalances") \
M(KafkaMessagesPolled, "Number of Kafka messages polled from librdkafka to ClickHouse") \
M(KafkaMessagesRead, "Number of Kafka messages already processed by ClickHouse") \
M(KafkaMessagesFailed, "Number of Kafka messages ClickHouse failed to parse") \
M(KafkaRowsRead, "Number of rows parsed from Kafka messages") \
M(KafkaRowsRejected, "Number of parsed rows which were later rejected (due to rebalances / errors or similar reasons). Those rows will be consumed again after the rebalance.") \
M(KafkaDirectReads, "Number of direct selects from Kafka tables since server start") \
M(KafkaBackgroundReads, "Number of background reads populating materialized views from Kafka since server start") \
M(KafkaCommits, "Number of successful commits of consumed offsets to Kafka (normally should be the same as KafkaBackgroundReads)") \
M(KafkaCommitFailures, "Number of failed commits of consumed offsets to Kafka (usually is a sign of some data duplication)") \
M(KafkaConsumerErrors, "Number of errors reported by librdkafka during polls") \
M(KafkaWrites, "Number of writes (inserts) to Kafka tables ") \
M(KafkaRowsWritten, "Number of rows inserted into Kafka tables") \
M(KafkaProducerFlushes, "Number of explicit flushes to Kafka producer") \
M(KafkaMessagesProduced, "Number of messages produced to Kafka") \
M(KafkaProducerErrors, "Number of errors during producing the messages to Kafka") \
\
M(ScalarSubqueriesGlobalCacheHit, "Number of times a read from a scalar subquery was done using the global cache") \
M(ScalarSubqueriesLocalCacheHit, "Number of times a read from a scalar subquery was done using the local cache") \
M(ScalarSubqueriesCacheMiss, "Number of times a read from a scalar subquery was not cached and had to be calculated completely")

View File

@ -29,6 +29,11 @@ namespace
};
}
namespace ProfileEvents
{
extern const Event ExecuteShellCommand;
}
namespace DB
{
@ -158,6 +163,7 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(
const Config & config)
{
logCommand(filename, argv);
ProfileEvents::increment(ProfileEvents::ExecuteShellCommand);
#if !defined(USE_MUSL)
/** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs,

View File

@ -1,4 +1,4 @@
#include <Functions/TargetSpecific.h>
#include <Common/TargetSpecific.h>
#include <Common/CpuId.h>

View File

@ -701,24 +701,34 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String &
}
}
void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, const String & keep_child_node)
bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probably_flat, const String & keep_child_node)
{
Strings children;
if (tryGetChildren(path, children) != Coordination::Error::ZOK)
return;
return false;
bool removed_as_expected = true;
while (!children.empty())
{
Coordination::Requests ops;
Strings batch;
ops.reserve(MULTI_BATCH_SIZE);
batch.reserve(MULTI_BATCH_SIZE);
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{
String child_path = fs::path(path) / children.back();
tryRemoveChildrenRecursive(child_path);
/// Will try to avoid recursive getChildren calls if child_path probably has no children.
/// It may be extremely slow when path contain a lot of leaf children.
if (!probably_flat)
tryRemoveChildrenRecursive(child_path);
if (likely(keep_child_node.empty() || keep_child_node != children.back()))
{
batch.push_back(child_path);
ops.emplace_back(zkutil::makeRemoveRequest(child_path, -1));
}
children.pop_back();
}
@ -726,10 +736,39 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, const Strin
/// this means someone is concurrently removing these children and we will have
/// to remove them one by one.
Coordination::Responses responses;
if (tryMulti(ops, responses) != Coordination::Error::ZOK)
for (const std::string & child : batch)
tryRemove(child);
if (tryMulti(ops, responses) == Coordination::Error::ZOK)
continue;
removed_as_expected = false;
std::vector<zkutil::ZooKeeper::FutureRemove> futures;
futures.reserve(batch.size());
for (const std::string & child : batch)
futures.push_back(asyncTryRemoveNoThrow(child, -1));
for (size_t i = 0; i < batch.size(); ++i)
{
auto res = futures[i].get();
if (res.error == Coordination::Error::ZOK)
continue;
if (res.error == Coordination::Error::ZNONODE)
continue;
if (res.error == Coordination::Error::ZNOTEMPTY)
{
if (probably_flat)
{
/// It actually has children, let's remove them
tryRemoveChildrenRecursive(batch[i]);
tryRemove(batch[i]);
}
continue;
}
throw KeeperException(res.error, batch[i]);
}
}
return removed_as_expected;
}
void ZooKeeper::removeRecursive(const std::string & path)

View File

@ -225,7 +225,10 @@ public:
/// If keep_child_node is not empty, this method will not remove path/keep_child_node (but will remove its subtree).
/// It can be useful to keep some child node as a flag which indicates that path is currently removing.
void removeChildrenRecursive(const std::string & path, const String & keep_child_node = {});
void tryRemoveChildrenRecursive(const std::string & path, const String & keep_child_node = {});
/// If probably_flat is true, this method will optimistically try to remove children non-recursive
/// and will fall back to recursive removal if it gets ZNOTEMPTY for some child.
/// Returns true if no kind of fallback happened.
bool tryRemoveChildrenRecursive(const std::string & path, bool probably_flat = false, const String & keep_child_node = {});
/// Remove all children nodes (non recursive).
void removeChildren(const std::string & path);

View File

@ -36,7 +36,7 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type)
element = recursiveRemoveLowCardinality(element);
if (tuple_type->haveExplicitNames())
return std::make_shared<DataTypeTuple>(elements, tuple_type->getElementNames(), tuple_type->serializeNames());
return std::make_shared<DataTypeTuple>(elements, tuple_type->getElementNames());
else
return std::make_shared<DataTypeTuple>(elements);
}

View File

@ -64,8 +64,8 @@ static std::optional<Exception> checkTupleNames(const Strings & names)
return {};
}
DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, bool serialize_names_)
: elems(elems_), names(names_), have_explicit_names(true), serialize_names(serialize_names_)
DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_)
: elems(elems_), names(names_), have_explicit_names(true)
{
size_t size = elems.size();
if (names.size() != size)
@ -75,11 +75,6 @@ DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, b
throw std::move(*exception);
}
bool DataTypeTuple::canBeCreatedWithNames(const Strings & names)
{
return checkTupleNames(names) == std::nullopt;
}
std::string DataTypeTuple::doGetName() const
{
size_t size = elems.size();
@ -91,7 +86,7 @@ std::string DataTypeTuple::doGetName() const
if (i != 0)
s << ", ";
if (have_explicit_names && serialize_names)
if (have_explicit_names)
s << backQuoteIfNeed(names[i]) << ' ';
s << elems[i]->getName();
@ -206,7 +201,7 @@ bool DataTypeTuple::equals(const IDataType & rhs) const
return false;
for (size_t i = 0; i < size; ++i)
if (!elems[i]->equals(*rhs_tuple.elems[i]))
if (!elems[i]->equals(*rhs_tuple.elems[i]) || names[i] != rhs_tuple.names[i])
return false;
return true;
@ -265,31 +260,29 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const
SerializationPtr DataTypeTuple::doGetDefaultSerialization() const
{
SerializationTuple::ElementSerializations serializations(elems.size());
bool use_explicit_names = have_explicit_names && serialize_names;
for (size_t i = 0; i < elems.size(); ++i)
{
String elem_name = use_explicit_names ? names[i] : toString(i + 1);
String elem_name = have_explicit_names ? names[i] : toString(i + 1);
auto serialization = elems[i]->getDefaultSerialization();
serializations[i] = std::make_shared<SerializationNamed>(serialization, elem_name);
}
return std::make_shared<SerializationTuple>(std::move(serializations), use_explicit_names);
return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
}
SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info) const
{
SerializationTuple::ElementSerializations serializations(elems.size());
const auto & info_tuple = assert_cast<const SerializationInfoTuple &>(info);
bool use_explicit_names = have_explicit_names && serialize_names;
for (size_t i = 0; i < elems.size(); ++i)
{
String elem_name = use_explicit_names ? names[i] : toString(i + 1);
String elem_name = have_explicit_names ? names[i] : toString(i + 1);
auto serialization = elems[i]->getSerialization(*info_tuple.getElementInfo(i));
serializations[i] = std::make_shared<SerializationNamed>(serialization, elem_name);
}
return std::make_shared<SerializationTuple>(std::move(serializations), use_explicit_names);
return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
}
MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const

View File

@ -22,14 +22,11 @@ private:
DataTypes elems;
Strings names;
bool have_explicit_names;
bool serialize_names = true;
public:
static constexpr bool is_parametric = true;
explicit DataTypeTuple(const DataTypes & elems);
DataTypeTuple(const DataTypes & elems, const Strings & names, bool serialize_names_ = true);
static bool canBeCreatedWithNames(const Strings & names);
DataTypeTuple(const DataTypes & elems, const Strings & names);
TypeIndex getTypeId() const override { return TypeIndex::Tuple; }
std::string doGetName() const override;
@ -66,7 +63,6 @@ public:
String getNameByPosition(size_t i) const;
bool haveExplicitNames() const { return have_explicit_names; }
bool serializeNames() const { return serialize_names; }
};
}

View File

@ -674,7 +674,6 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
LOG_INFO(log, "Marked recovered {} as finished", entry_name);
}
}
current_zookeeper->set(replica_path + "/log_ptr", toString(max_log_ptr));
}
std::map<String, String> DatabaseReplicated::tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr)

View File

@ -66,9 +66,17 @@ void DatabaseReplicatedDDLWorker::initializeReplication()
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
logs_to_keep = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/logs_to_keep"));
if (our_log_ptr == 0 || our_log_ptr + logs_to_keep < max_log_ptr)
{
database->recoverLostReplica(zookeeper, our_log_ptr, max_log_ptr);
zookeeper->set(database->replica_path + "/log_ptr", toString(max_log_ptr));
initializeLogPointer(DDLTaskBase::getLogEntryName(max_log_ptr));
}
else
last_skipped_entry_name.emplace(DDLTaskBase::getLogEntryName(our_log_ptr));
{
String log_entry_name = DDLTaskBase::getLogEntryName(our_log_ptr);
last_skipped_entry_name.emplace(log_entry_name);
initializeLogPointer(log_entry_name);
}
}
String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry)
@ -140,10 +148,10 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
/// but it requires more complex logic around /try node.
auto zookeeper = getAndSetZooKeeper();
UInt32 our_log_ptr = parse<UInt32>(zookeeper->get(database->replica_path + "/log_ptr"));
UInt32 our_log_ptr = getLogPointer();
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
assert(our_log_ptr <= max_log_ptr);
if (database->db_settings.max_replication_lag_to_enqueue < max_log_ptr - our_log_ptr)
if (our_log_ptr + database->db_settings.max_replication_lag_to_enqueue < max_log_ptr)
throw Exception(ErrorCodes::NOT_A_LEADER, "Cannot enqueue query on this replica, "
"because it has replication lag of {} queries. Try other replica.", max_log_ptr - our_log_ptr);
@ -203,7 +211,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
}
}
UInt32 our_log_ptr = parse<UInt32>(zookeeper->get(fs::path(database->replica_path) / "log_ptr"));
UInt32 our_log_ptr = getLogPointer();
UInt32 entry_num = DatabaseReplicatedTask::getLogEntryNumber(entry_name);
if (entry_num <= our_log_ptr)
@ -308,4 +316,18 @@ bool DatabaseReplicatedDDLWorker::canRemoveQueueEntry(const String & entry_name,
return entry_number + logs_to_keep < max_log_ptr;
}
void DatabaseReplicatedDDLWorker::initializeLogPointer(const String & processed_entry_name)
{
updateMaxDDLEntryID(processed_entry_name);
assert(max_id.load() == parse<UInt32>(getAndSetZooKeeper()->get(fs::path(database->replica_path) / "log_ptr")));
}
UInt32 DatabaseReplicatedDDLWorker::getLogPointer() const
{
/// NOTE it may not be equal to the log_ptr in zk:
/// - max_id can be equal to log_ptr - 1 due to race condition (when it's updated in zk, but not updated in memory yet)
/// - max_id can be greater than log_ptr, because log_ptr is not updated for failed and dummy entries
return max_id.load();
}
}

View File

@ -32,9 +32,11 @@ public:
static String enqueueQueryImpl(const ZooKeeperPtr & zookeeper, DDLLogEntry & entry,
DatabaseReplicated * const database, bool committed = false); /// NOLINT
UInt32 getLogPointer() const;
private:
bool initializeMainThread() override;
void initializeReplication();
void initializeLogPointer(const String & processed_entry_name);
DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) override;
bool canRemoveQueueEntry(const String & entry_name, const Coordination::Stat & stat) override;

View File

@ -334,15 +334,17 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getImplementationBuffer(File
read_buffer_for_file_segment->seek(file_offset_of_buffer_end, SEEK_SET);
}
auto impl_range = read_buffer_for_file_segment->getRemainingReadRange();
auto download_offset = file_segment->getDownloadOffset();
if (download_offset != static_cast<size_t>(read_buffer_for_file_segment->getPosition()))
{
auto impl_range = read_buffer_for_file_segment->getRemainingReadRange();
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Buffer's offsets mismatch; cached buffer offset: {}, download_offset: {}, position: {}, implementation buffer offset: {}, "
"implementation buffer reading until: {}, file segment info: {}",
file_offset_of_buffer_end, download_offset, read_buffer_for_file_segment->getPosition(),
impl_range.left, *impl_range.right, file_segment->getInfoForLog());
}
break;
}
@ -802,12 +804,14 @@ std::optional<size_t> CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset()
String CachedReadBufferFromRemoteFS::getInfoForLog()
{
auto implementation_buffer_read_range_str =
implementation_buffer ?
std::to_string(implementation_buffer->getRemainingReadRange().left)
+ '-'
+ (implementation_buffer->getRemainingReadRange().right ? std::to_string(*implementation_buffer->getRemainingReadRange().right) : "None")
: "None";
String implementation_buffer_read_range_str;
if (implementation_buffer)
{
auto read_range = implementation_buffer->getRemainingReadRange();
implementation_buffer_read_range_str = std::to_string(read_range.left) + '-' + (read_range.right ? std::to_string(*read_range.right) : "None");
}
else
implementation_buffer_read_range_str = "None";
auto current_file_segment_info = current_file_segment_it == file_segments_holder->file_segments.end() ? "None" : (*current_file_segment_it)->getInfoForLog();

View File

@ -96,17 +96,6 @@ if (TARGET ch_contrib::rapidjson)
target_link_libraries(clickhouse_functions PRIVATE ch_contrib::rapidjson)
endif()
# ClickHouse developers may use platform-dependent code under some macro (e.g. `#ifdef ENABLE_MULTITARGET`).
# If turned ON, this option defines such macro.
# See `src/Functions/TargetSpecific.h`
option(ENABLE_MULTITARGET_CODE "Enable platform-dependent code" ON)
if (ENABLE_MULTITARGET_CODE)
add_definitions(-DENABLE_MULTITARGET_CODE=1)
else()
add_definitions(-DENABLE_MULTITARGET_CODE=0)
endif()
add_subdirectory(GatherUtils)
target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_gatherutils)

View File

@ -1,12 +1,12 @@
#pragma once
#include <base/map.h>
#include <Common/TargetSpecific.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/GatherUtils/GatherUtils.h>
#include <Functions/GatherUtils/Sources.h>
#include <Functions/IFunction.h>
#include <Functions/PerformanceAdaptors.h>
#include <Functions/TargetSpecific.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/getLeastSupertype.h>

View File

@ -2958,8 +2958,7 @@ private:
/// For named tuples allow conversions for tuples with
/// different sets of elements. If element exists in @to_type
/// and doesn't exist in @to_type it will be filled by default values.
if (from_type->haveExplicitNames() && from_type->serializeNames()
&& to_type->haveExplicitNames() && to_type->serializeNames())
if (from_type->haveExplicitNames() && to_type->haveExplicitNames())
{
const auto & from_names = from_type->getElementNames();
std::unordered_map<String, size_t> from_positions;

View File

@ -38,8 +38,8 @@
#include <Columns/ColumnTuple.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/TargetSpecific.h>
#include <Functions/PerformanceAdaptors.h>
#include <Common/TargetSpecific.h>
#include <base/range.h>
#include <base/bit_cast.h>

View File

@ -1,9 +1,9 @@
#pragma once
#include <Common/TargetSpecific.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Functions/IFunction.h>
#include <Functions/TargetSpecific.h>
#include <Functions/PerformanceAdaptors.h>
#include <IO/WriteHelpers.h>

View File

@ -7,6 +7,8 @@
#include <Core/AccurateComparison.h>
#include <base/range.h>
#include "GatherUtils.h"
#include "sliceEqualElements.h"
#include "sliceHasImplAnyAll.h"
namespace DB::ErrorCodes
@ -461,39 +463,19 @@ void NO_INLINE conditional(SourceA && src_a, SourceB && src_b, Sink && sink, con
}
/// Methods to check if first array has elements from second array, overloaded for various combinations of types.
template <
ArraySearchType search_type,
typename FirstSliceType,
typename SecondSliceType,
bool (*isEqual)(const FirstSliceType &, const SecondSliceType &, size_t, size_t)>
bool sliceHasImplAnyAll(const FirstSliceType & first, const SecondSliceType & second, const UInt8 * first_null_map, const UInt8 * second_null_map)
template <typename T>
bool insliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
size_t first_ind [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
for (size_t i = 0; i < second.size; ++i)
{
bool has = false;
for (size_t j = 0; j < first.size && !has; ++j)
{
const bool is_first_null = has_first_null_map && first_null_map[j];
const bool is_second_null = has_second_null_map && second_null_map[i];
if (is_first_null && is_second_null)
has = true;
if (!is_first_null && !is_second_null && isEqual(first, second, j, i))
has = true;
}
if (has && search_type == ArraySearchType::Any)
return true;
if (!has && search_type == ArraySearchType::All)
return false;
}
return search_type == ArraySearchType::All;
if constexpr (is_decimal<T>)
return accurate::equalsOp(first.data[first_ind].value, first.data[second_ind].value);
else
return accurate::equalsOp(first.data[first_ind], first.data[second_ind]);
}
inline ALWAYS_INLINE bool insliceEqualElements(const GenericArraySlice & first, size_t first_ind, size_t second_ind)
{
return first.elements->compareAt(first_ind + first.begin, second_ind + first.begin, *first.elements, -1) == 0;
}
template <
@ -620,55 +602,6 @@ bool sliceHasImpl(const FirstSliceType & first, const SecondSliceType & second,
return sliceHasImplAnyAll<search_type, FirstSliceType, SecondSliceType, isEqual>(first, second, first_null_map, second_null_map);
}
template <typename T, typename U>
bool sliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
const NumericArraySlice<U> & second [[maybe_unused]],
size_t first_ind [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
/// TODO: Decimal scale
if constexpr (is_decimal<T> && is_decimal<U>)
return accurate::equalsOp(first.data[first_ind].value, second.data[second_ind].value);
else if constexpr (is_decimal<T> || is_decimal<U>)
return false;
else
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
}
template <typename T>
bool sliceEqualElements(const NumericArraySlice<T> &, const GenericArraySlice &, size_t, size_t)
{
return false;
}
template <typename U>
bool sliceEqualElements(const GenericArraySlice &, const NumericArraySlice<U> &, size_t, size_t)
{
return false;
}
inline ALWAYS_INLINE bool sliceEqualElements(const GenericArraySlice & first, const GenericArraySlice & second, size_t first_ind, size_t second_ind)
{
return first.elements->compareAt(first_ind + first.begin, second_ind + second.begin, *second.elements, -1) == 0;
}
template <typename T>
bool insliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
size_t first_ind [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
if constexpr (is_decimal<T>)
return accurate::equalsOp(first.data[first_ind].value, first.data[second_ind].value);
else
return accurate::equalsOp(first.data[first_ind], first.data[second_ind]);
}
inline ALWAYS_INLINE bool insliceEqualElements(const GenericArraySlice & first, size_t first_ind, size_t second_ind)
{
return first.elements->compareAt(first_ind + first.begin, second_ind + first.begin, *first.elements, -1) == 0;
}
template <ArraySearchType search_type, typename T, typename U>
bool sliceHas(const NumericArraySlice<T> & first, const NumericArraySlice<U> & second)
{
@ -854,4 +787,3 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
}
}

View File

@ -1,4 +1,5 @@
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
add_headers_and_sources(clickhouse_functions_gatherutils .)
add_library(clickhouse_functions_gatherutils ${clickhouse_functions_gatherutils_sources} ${clickhouse_functions_gatherutils_headers})
target_link_libraries(clickhouse_functions_gatherutils PRIVATE dbms)
@ -14,3 +15,5 @@ endif()
if (STRIP_DEBUG_SYMBOLS_FUNCTIONS)
target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0")
endif()
set_target_properties(clickhouse_functions_gatherutils PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}")

View File

@ -0,0 +1,41 @@
#pragma once
#include <Core/AccurateComparison.h>
#include "Slices.h"
namespace DB::GatherUtils
{
template <typename T, typename U>
bool sliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
const NumericArraySlice<U> & second [[maybe_unused]],
size_t first_ind [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
/// TODO: Decimal scale
if constexpr (is_decimal<T> && is_decimal<U>)
return accurate::equalsOp(first.data[first_ind].value, second.data[second_ind].value);
else if constexpr (is_decimal<T> || is_decimal<U>)
return false;
else
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
}
template <typename T>
bool sliceEqualElements(const NumericArraySlice<T> &, const GenericArraySlice &, size_t, size_t)
{
return false;
}
template <typename U>
bool sliceEqualElements(const GenericArraySlice &, const NumericArraySlice<U> &, size_t, size_t)
{
return false;
}
inline ALWAYS_INLINE bool sliceEqualElements(const GenericArraySlice & first, const GenericArraySlice & second, size_t first_ind, size_t second_ind)
{
return first.elements->compareAt(first_ind + first.begin, second_ind + second.begin, *second.elements, -1) == 0;
}
}

View File

@ -0,0 +1,943 @@
#pragma once
#include "GatherUtils.h"
#include "Slices.h"
#include "sliceEqualElements.h"
#if defined(__SSE4_2__)
#include <emmintrin.h>
#include <smmintrin.h>
#include <nmmintrin.h>
#endif
#if defined(__AVX2__)
#include <immintrin.h>
#endif
#include <Common/TargetSpecific.h>
namespace DB::GatherUtils
{
inline ALWAYS_INLINE bool hasNull(const UInt8 * null_map, size_t null_map_size)
{
if (null_map == nullptr)
return false;
for (size_t i = 0; i < null_map_size; ++i)
{
if (null_map[i])
return true;
}
return false;
}
template<class T>
inline ALWAYS_INLINE bool hasAllIntegralLoopRemainder(
size_t j, const NumericArraySlice<T> & first, const NumericArraySlice<T> & second, const UInt8 * first_null_map, const UInt8 * second_null_map)
{
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
for (; j < second.size; ++j)
{
// skip null elements since both have at least one - assuming it was checked earlier that at least one element in 'first' is null
if (has_second_null_map && second_null_map[j])
continue;
bool found = false;
for (size_t i = 0; i < first.size; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
if (first.data[i] == second.data[j])
{
found = true;
break;
}
}
if (!found)
return false;
}
return true;
}
#if defined(__AVX2__)
DECLARE_AVX2_SPECIFIC_CODE (
// AVX2 Int64, UInt64 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int64> || std::is_same_v<IntType, UInt64>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr Int64 full = -1, none = 0;
const __m256i ones = _mm256_set1_epi64x(full);
const __m256i zeros = _mm256_setzero_si256();
if (second.size > 3 && first.size > 3)
{
for (; j < second.size - 3 && has_mask; j += 4)
{
has_mask = 0;
const __m256i second_data = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(second.data + j));
// bits of the bitmask are set to one if considered as null in the corresponding null map, 0 otherwise;
__m256i bitmask = has_second_null_map ?
_mm256_set_epi64x(
(second_null_map[j + 3])? full : none,
(second_null_map[j + 2])? full : none,
(second_null_map[j + 1])? full : none,
(second_null_map[j]) ? full : none)
: zeros;
size_t i = 0;
for (; i < first.size - 3 && !has_mask; has_mask = _mm256_testc_si256(bitmask, ones), i += 4)
{
const __m256i first_data = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(first.data + i));
const __m256i first_nm_mask = has_first_null_map?
_mm256_set_m128i(
_mm_cvtepi8_epi64(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i + 2))),
_mm_cvtepi8_epi64(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i))))
: zeros;
bitmask =
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
first_nm_mask,
_mm256_cmpeq_epi64(second_data, first_data)),
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(5,4,3,2,1,0,7,6)),
_mm256_cmpeq_epi64(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(5,4,3,2,1,0,7,6))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(3,2,1,0,7,6,5,4)),
_mm256_cmpeq_epi64(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(3,2,1,0,7,6,5,4)))),
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(1,0,7,6,5,4,3,2)),
_mm256_cmpeq_epi64(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(1,0,7,6,5,4,3,2)))))),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m256i v_i = _mm256_set1_epi64x(first.data[i]);
bitmask = _mm256_or_si256(bitmask, _mm256_cmpeq_epi64(second_data, v_i));
has_mask = _mm256_testc_si256(bitmask, ones);
}
}
}
}
if (!has_mask && second.size > 3)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
// AVX2 Int32, UInt32 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int32> || std::is_same_v<IntType, UInt32>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr int full = -1, none = 0;
const __m256i ones = _mm256_set1_epi32(full);
const __m256i zeros = _mm256_setzero_si256();
if (second.size > 7 && first.size > 7)
{
for (; j < second.size - 7 && has_mask; j += 8)
{
has_mask = 0;
const __m256i second_data = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(second.data + j));
// bits of the bitmask are set to one if considered as null in the corresponding null map, 0 otherwise;
__m256i bitmask = has_second_null_map ?
_mm256_set_epi32(
(second_null_map[j + 7]) ? full : none,
(second_null_map[j + 6]) ? full : none,
(second_null_map[j + 5]) ? full : none,
(second_null_map[j + 4]) ? full : none,
(second_null_map[j + 3]) ? full : none,
(second_null_map[j + 2]) ? full : none,
(second_null_map[j + 1]) ? full : none,
(second_null_map[j]) ? full : none)
: zeros;
size_t i = 0;
for (; i < first.size - 7 && !has_mask; has_mask = _mm256_testc_si256(bitmask, ones), i += 8)
{
const __m256i first_data = _mm256_loadu_si256(reinterpret_cast<const __m256i *>(first.data + i));
// Create a mask to avoid to compare null elements
// set_m128i takes two arguments: (high segment, low segment) that are two __m128i convert from 8bits to 32bits to match with next operations
const __m256i first_nm_mask = has_first_null_map?
_mm256_set_m128i(
_mm_cvtepi8_epi32(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i + 4))),
_mm_cvtepi8_epi32(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i))))
: zeros;
bitmask =
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
first_nm_mask,
_mm256_cmpeq_epi32(second_data, first_data)),
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(6,5,4,3,2,1,0,7)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(6,5,4,3,2,1,0,7))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(5,4,3,2,1,0,7,6)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(5,4,3,2,1,0,7,6)))),
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(4,3,2,1,0,7,6,5)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(4,3,2,1,0,7,6,5)))))
),
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(3,2,1,0,7,6,5,4)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(3,2,1,0,7,6,5,4)))),
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(2,1,0,7,6,5,4,3)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(2,1,0,7,6,5,4,3))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(1,0,7,6,5,4,3,2)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(1,0,7,6,5,4,3,2)))),
_mm256_andnot_si256(
_mm256_permutevar8x32_epi32(first_nm_mask, _mm256_set_epi32(0,7,6,5,4,3,2,1)),
_mm256_cmpeq_epi32(second_data, _mm256_permutevar8x32_epi32(first_data, _mm256_set_epi32(0,7,6,5,4,3,2,1))))))),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m256i v_i = _mm256_set1_epi32(first.data[i]);
bitmask = _mm256_or_si256(bitmask, _mm256_cmpeq_epi32(second_data, v_i));
has_mask = _mm256_testc_si256(bitmask, ones);
}
}
}
}
if (!has_mask && second.size > 7)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
// AVX2 Int16, UInt16 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int16> || std::is_same_v<IntType, UInt16>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr int16_t full = -1, none = 0;
const __m256i ones = _mm256_set1_epi16(full);
const __m256i zeros = _mm256_setzero_si256();
if (second.size > 15 && first.size > 15)
{
for (; j < second.size - 15 && has_mask; j += 16)
{
has_mask = 0;
const __m256i second_data = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(second.data + j));
__m256i bitmask = has_second_null_map ?
_mm256_set_epi16(
(second_null_map[j + 15]) ? full : none, (second_null_map[j + 14]) ? full : none,
(second_null_map[j + 13]) ? full : none, (second_null_map[j + 12]) ? full : none,
(second_null_map[j + 11]) ? full : none, (second_null_map[j + 10]) ? full : none,
(second_null_map[j + 9]) ? full : none, (second_null_map[j + 8])? full : none,
(second_null_map[j + 7]) ? full : none, (second_null_map[j + 6])? full : none,
(second_null_map[j + 5]) ? full : none, (second_null_map[j + 4])? full : none,
(second_null_map[j + 3]) ? full : none, (second_null_map[j + 2])? full : none,
(second_null_map[j + 1]) ? full : none, (second_null_map[j]) ? full : none)
: zeros;
size_t i = 0;
for (; i < first.size - 15 && !has_mask; has_mask = _mm256_testc_si256(bitmask, ones), i += 16)
{
const __m256i first_data = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(first.data + i));
const __m256i first_nm_mask = has_first_null_map?
_mm256_set_m128i(
_mm_cvtepi8_epi16(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i + 8))),
_mm_cvtepi8_epi16(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i))))
: zeros;
bitmask =
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
first_nm_mask,
_mm256_cmpeq_epi16(second_data, first_data)),
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28)))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26)))))
),
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24)))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20)))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(first_nm_mask, _mm256_set_epi8(17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(first_data, _mm256_set_epi8(17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18))))))
),
_mm256_or_si256(
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_permute2x128_si256(first_nm_mask, first_nm_mask,1),
_mm256_cmpeq_epi16(second_data, _mm256_permute2x128_si256(first_data, first_data, 1))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(13,12,11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(11,10,9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12)))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(9,8,7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10)))))
),
_mm256_or_si256(
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data ,first_data, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6))))),
_mm256_or_si256(
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data ,first_data ,1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)))),
_mm256_andnot_si256(
_mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2)),
_mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2))))))
)
),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m256i v_i = _mm256_set1_epi16(first.data[i]);
bitmask = _mm256_or_si256(bitmask, _mm256_cmpeq_epi16(second_data, v_i));
has_mask = _mm256_testc_si256(bitmask, ones);
}
}
}
}
if (!has_mask && second.size > 15)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
)
#endif
#if defined(__SSE4_2__)
DECLARE_SSE42_SPECIFIC_CODE (
// SSE4.2 Int64, UInt64 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int64> || std::is_same_v<IntType, UInt64>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt64(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr Int64 full = -1, none = 0;
const __m128i zeros = _mm_setzero_si128();
if (second.size > 1 && first.size > 1)
{
for (; j < second.size - 1 && has_mask; j += 2)
{
has_mask = 0;
const __m128i second_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(second.data + j));
__m128i bitmask = has_second_null_map ?
_mm_set_epi64x(
(second_null_map[j + 1]) ? full : none,
(second_null_map[j]) ? full : none)
: zeros;
size_t i = 0;
for (; i < first.size - 1 && !has_mask; has_mask = _mm_test_all_ones(bitmask), i += 2)
{
const __m128i first_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(first.data + i));
const __m128i first_nm_mask = has_first_null_map ?
_mm_cvtepi8_epi64(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i)))
: zeros;
bitmask =
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
first_nm_mask,
_mm_cmpeq_epi64(second_data, first_data)),
_mm_andnot_si128(
_mm_shuffle_epi32(first_nm_mask, _MM_SHUFFLE(1,0,3,2)),
_mm_cmpeq_epi64(second_data, _mm_shuffle_epi32(first_data, _MM_SHUFFLE(1,0,3,2))))),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m128i v_i = _mm_set1_epi64x(first.data[i]);
bitmask = _mm_or_si128(bitmask, _mm_cmpeq_epi64(second_data, v_i));
has_mask = _mm_test_all_ones(bitmask);
}
}
}
}
if (!has_mask && second.size > 1)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
// SSE4.2 Int32, UInt32 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int32> || std::is_same_v<IntType, UInt32>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt32(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr int full = -1, none = 0;
const __m128i zeros = _mm_setzero_si128();
if (second.size > 3 && first.size > 3)
{
for (; j < second.size - 3 && has_mask; j += 4)
{
has_mask = 0;
const __m128i second_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(second.data + j));
__m128i bitmask = has_second_null_map ?
_mm_set_epi32(
(second_null_map[j + 3]) ? full : none,
(second_null_map[j + 2]) ? full : none,
(second_null_map[j + 1]) ? full : none,
(second_null_map[j]) ? full : none)
: zeros;
size_t i = 0;
for (; i < first.size - 3 && !has_mask; has_mask = _mm_test_all_ones(bitmask), i += 4)
{
const __m128i first_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(first.data + i));
const __m128i first_nm_mask = has_first_null_map ?
_mm_cvtepi8_epi32(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i)))
: zeros;
bitmask =
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
first_nm_mask,
_mm_cmpeq_epi32(second_data, first_data)),
_mm_andnot_si128(
_mm_shuffle_epi32(first_nm_mask, _MM_SHUFFLE(2,1,0,3)),
_mm_cmpeq_epi32(second_data, _mm_shuffle_epi32(first_data, _MM_SHUFFLE(2,1,0,3))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi32(first_nm_mask, _MM_SHUFFLE(1,0,3,2)),
_mm_cmpeq_epi32(second_data, _mm_shuffle_epi32(first_data, _MM_SHUFFLE(1,0,3,2)))),
_mm_andnot_si128(
_mm_shuffle_epi32(first_nm_mask, _MM_SHUFFLE(0,3,2,1)),
_mm_cmpeq_epi32(second_data, _mm_shuffle_epi32(first_data, _MM_SHUFFLE(0,3,2,1)))))
),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m128i r_i = _mm_set1_epi32(first.data[i]);
bitmask = _mm_or_si128(bitmask, _mm_cmpeq_epi32(second_data, r_i));
has_mask = _mm_test_all_ones(bitmask);
}
}
}
}
if (!has_mask && second.size > 3)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
// SSE4.2 Int16, UInt16 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int16> || std::is_same_v<IntType, UInt16>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt16(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr int16_t full = -1, none = 0;
const __m128i zeros = _mm_setzero_si128();
if (second.size > 6 && first.size > 6)
{
for (; j < second.size - 7 && has_mask; j += 8)
{
has_mask = 0;
const __m128i second_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(second.data + j));
__m128i bitmask = has_second_null_map ?
_mm_set_epi16(
(second_null_map[j + 7]) ? full : none, (second_null_map[j + 6]) ? full : none,
(second_null_map[j + 5]) ? full : none, (second_null_map[j + 4]) ? full : none,
(second_null_map[j + 3]) ? full : none, (second_null_map[j + 2]) ? full : none,
(second_null_map[j + 1]) ? full : none, (second_null_map[j]) ? full: none)
: zeros;
size_t i = 0;
for (; i < first.size-7 && !has_mask; has_mask = _mm_test_all_ones(bitmask), i += 8)
{
const __m128i first_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(first.data + i));
const __m128i first_nm_mask = has_first_null_map ?
_mm_cvtepi8_epi16(_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i)))
: zeros;
bitmask =
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
first_nm_mask,
_mm_cmpeq_epi16(second_data, first_data)),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(13,12,11,10,9,8,7,6,5,4,3,2,1,0,15,14)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(13,12,11,10,9,8,7,6,5,4,3,2,1,0,15,14))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(11,10,9,8,7,6,5,4,3,2,1,0,15,14,13,12)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(11,10,9,8,7,6,5,4,3,2,1,0,15,14,13,12)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(9,8,7,6,5,4,3,2,1,0,15,14,13,12,11,10)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(9,8,7,6,5,4,3,2,1,0,15,14,13,12,11,10)))))
),
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(7,6,5,4,3,2,1,0,15,14,13,12,11,10,9,8)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(7,6,5,4,3,2,1,0,15,14,13,12,11,10,9,8)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(5,4,3,2,1,0,15,14,13,12,11,10,9,8,7,6)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(5,4,3,2,1,0,15,14,13,12,11,10,9,8,7,6))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(3,2,1,0,15,14,13,12,11,10,9,8,7,6,5,4)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(3,2,1,0,15,14,13,12,11,10,9,8,7,6,5,4)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(1,0,15,14,13,12,11,10,9,8,7,6,5,4,3,2)),
_mm_cmpeq_epi16(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(1,0,15,14,13,12,11,10,9,8,7,6,5,4,3,2))))))
),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m128i v_i = _mm_set1_epi16(first.data[i]);
bitmask = _mm_or_si128(bitmask, _mm_cmpeq_epi16(second_data, v_i));
has_mask = _mm_test_all_ones(bitmask);
}
}
}
}
if (!has_mask && second.size > 6)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
// Int8/UInt8 version is faster with SSE than with AVX2
// SSE2 Int8, UInt8 specialization
template<typename IntType>
requires (std::is_same_v<IntType, Int8> || std::is_same_v<IntType, UInt8>)
inline ALWAYS_INLINE bool sliceHasImplAnyAllImplInt8(
const NumericArraySlice<IntType> & first,
const NumericArraySlice<IntType> & second,
const UInt8 * first_null_map,
const UInt8 * second_null_map)
{
if (second.size == 0)
return true;
if (!hasNull(first_null_map, first.size) && hasNull(second_null_map, second.size))
return false;
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
size_t j = 0;
int has_mask = 1;
static constexpr int8_t full = -1, none = 0;
const __m128i zeros = _mm_setzero_si128();
if (second.size > 15 && first.size > 15)
{
for (; j < second.size - 15 && has_mask; j += 16)
{
has_mask = 0;
const __m128i second_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(second.data + j));
__m128i bitmask = has_second_null_map ?
_mm_set_epi8(
(second_null_map[j + 15]) ? full : none, (second_null_map[j + 14]) ? full : none,
(second_null_map[j + 13]) ? full : none, (second_null_map[j + 12]) ? full : none,
(second_null_map[j + 11]) ? full : none, (second_null_map[j + 10]) ? full : none,
(second_null_map[j + 9]) ? full : none, (second_null_map[j + 8]) ? full : none,
(second_null_map[j + 7]) ? full : none, (second_null_map[j + 6]) ? full : none,
(second_null_map[j + 5]) ? full : none, (second_null_map[j + 4]) ? full : none,
(second_null_map[j + 3]) ? full : none, (second_null_map[j + 2]) ? full : none,
(second_null_map[j + 1]) ? full : none, (second_null_map[j]) ? full : none)
: zeros;
size_t i = 0;
for (; i < first.size - 15 && !has_mask; has_mask = _mm_test_all_ones(bitmask), i += 16)
{
const __m128i first_data = _mm_loadu_si128(reinterpret_cast<const __m128i *>(first.data + i));
const __m128i first_nm_mask = has_first_null_map ?
_mm_loadu_si128(reinterpret_cast<const __m128i *>(first_null_map + i))
: zeros;
bitmask =
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
first_nm_mask,
_mm_cmpeq_epi8(second_data, first_data)),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,15)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(14,13,12,11,10,9,8,7,6,5,4,3,2,1,0,15))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(13,12,11,10,9,8,7,6,5,4,3,2,1,0,15,14)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(13,12,11,10,9,8,7,6,5,4,3,2,1,0,15,14)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(12,11,10,9,8,7,6,5,4,3,2,1,0,15,14,13)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(12,11,10,9,8,7,6,5,4,3,2,1,0,15,14,13)))))
),
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(11,10,9,8,7,6,5,4,3,2,1,0,15,14,13,12)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(11,10,9,8,7,6,5,4,3,2,1,0,15,14,13,12)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(10,9,8,7,6,5,4,3,2,1,0,15,14,13,12,11)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(10,9,8,7,6,5,4,3,2,1,0,15,14,13,12,11))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(9,8,7,6,5,4,3,2,1,0,15,14,13,12,11,10)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(9,8,7,6,5,4,3,2,1,0,15,14,13,12,11,10)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(8,7,6,5,4,3,2,1,0,15,14,13,12,11,10,9)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(8,7,6,5,4,3,2,1,0,15,14,13,12,11,10,9))))))),
_mm_or_si128(
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(7,6,5,4,3,2,1,0,15,14,13,12,11,10,9,8)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(7,6,5,4,3,2,1,0,15,14,13,12,11,10,9,8)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(6,5,4,3,2,1,0,15,14,13,12,11,10,9,8,7)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(6,5,4,3,2,1,0,15,14,13,12,11,10,9,8,7))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(5,4,3,2,1,0,15,14,13,12,11,10,9,8,7,6)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(5,4,3,2,1,0,15,14,13,12,11,10,9,8,7,6)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(4,3,2,1,0,15,14,13,12,11,10,9,8,7,6,5)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(4,3,2,1,0,15,14,13,12,11,10,9,8,7,6,5)))))),
_mm_or_si128(
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(3,2,1,0,15,14,13,12,11,10,9,8,7,6,5,4)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(3,2,1,0,15,14,13,12,11,10,9,8,7,6,5,4)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(2,1,0,15,14,13,12,11,10,9,8,7,6,5,4,3)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(2,1,0,15,14,13,12,11,10,9,8,7,6,5,4,3))))),
_mm_or_si128(
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(1,0,15,14,13,12,11,10,9,8,7,6,5,4,3,2)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(1,0,15,14,13,12,11,10,9,8,7,6,5,4,3,2)))),
_mm_andnot_si128(
_mm_shuffle_epi8(first_nm_mask, _mm_set_epi8(0,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1)),
_mm_cmpeq_epi8(second_data, _mm_shuffle_epi8(first_data, _mm_set_epi8(0,15,14,13,12,11,10,9,8,7,6,5,4,3,2,1)))))))),
bitmask);
}
if (i < first.size)
{
for (; i < first.size && !has_mask; ++i)
{
if (has_first_null_map && first_null_map[i])
continue;
__m128i v_i = _mm_set1_epi8(first.data[i]);
bitmask = _mm_or_si128(bitmask, _mm_cmpeq_epi8(second_data, v_i));
has_mask = _mm_test_all_ones(bitmask);
}
}
}
}
if (!has_mask && second.size > 15)
return false;
return hasAllIntegralLoopRemainder(j, first, second, first_null_map, second_null_map);
}
)
#endif
template <
ArraySearchType search_type,
typename FirstSliceType,
typename SecondSliceType,
bool (*isEqual)(const FirstSliceType &, const SecondSliceType &, size_t, size_t)>
bool sliceHasImplAnyAllGenericImpl(const FirstSliceType & first, const SecondSliceType & second, const UInt8 * first_null_map, const UInt8 * second_null_map)
{
const bool has_first_null_map = first_null_map != nullptr;
const bool has_second_null_map = second_null_map != nullptr;
const bool has_second_null = hasNull(second_null_map, second.size);
if (has_second_null)
{
const bool has_first_null = hasNull(first_null_map, first.size);
if (has_first_null && search_type == ArraySearchType::Any)
return true;
if (!has_first_null && search_type == ArraySearchType::All)
return false;
}
for (size_t i = 0; i < second.size; ++i)
{
if (has_second_null_map && second_null_map[i])
continue;
bool has = false;
for (size_t j = 0; j < first.size && !has; ++j)
{
if (has_first_null_map && first_null_map[j])
continue;
if (isEqual(first, second, j, i))
{
has = true;
break;
}
}
if (has && search_type == ArraySearchType::Any)
return true;
if (!has && search_type == ArraySearchType::All)
return false;
}
return search_type == ArraySearchType::All;
}
/// Methods to check if first array has elements from second array, overloaded for various combinations of types.
template <
ArraySearchType search_type,
typename FirstSliceType,
typename SecondSliceType,
bool (*isEqual)(const FirstSliceType &, const SecondSliceType &, size_t, size_t)>
inline ALWAYS_INLINE bool sliceHasImplAnyAll(const FirstSliceType & first, const SecondSliceType & second, const UInt8 * first_null_map, const UInt8 * second_null_map)
{
#if USE_MULTITARGET_CODE
if constexpr (search_type == ArraySearchType::All && std::is_same_v<FirstSliceType, SecondSliceType>)
{
#if defined(__AVX2__)
if (isArchSupported(TargetArch::AVX2))
{
if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int16>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt16>>)
{
return GatherUtils::TargetSpecific::AVX2::sliceHasImplAnyAllImplInt16(first, second, first_null_map, second_null_map);
}
else if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int32>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt32>>)
{
return GatherUtils::TargetSpecific::AVX2::sliceHasImplAnyAllImplInt32(first, second, first_null_map, second_null_map);
}
else if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int64>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt64>>)
{
return GatherUtils::TargetSpecific::AVX2::sliceHasImplAnyAllImplInt64(first, second, first_null_map, second_null_map);
}
}
#endif
if (isArchSupported(TargetArch::SSE42))
{
if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int8>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt8>>)
{
return TargetSpecific::SSE42::sliceHasImplAnyAllImplInt8(first, second, first_null_map, second_null_map);
}
else if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int16>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt16>>)
{
return GatherUtils::TargetSpecific::SSE42::sliceHasImplAnyAllImplInt16(first, second, first_null_map, second_null_map);
}
else if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int32>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt32>>)
{
return GatherUtils::TargetSpecific::SSE42::sliceHasImplAnyAllImplInt32(first, second, first_null_map, second_null_map);
}
else if constexpr (std::is_same_v<FirstSliceType, NumericArraySlice<Int64>> || std::is_same_v<FirstSliceType, NumericArraySlice<UInt64>>)
{
return GatherUtils::TargetSpecific::SSE42::sliceHasImplAnyAllImplInt64(first, second, first_null_map, second_null_map);
}
}
}
#endif
return sliceHasImplAnyAllGenericImpl<search_type, FirstSliceType, SecondSliceType, isEqual>(first, second, first_null_map, second_null_map);
}
}

View File

@ -1,8 +1,8 @@
#pragma once
#include <Functions/TargetSpecific.h>
#include <Functions/IFunction.h>
#include <Common/TargetSpecific.h>
#include <Common/Stopwatch.h>
#include <Interpreters/Context.h>

View File

@ -6,8 +6,8 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <Functions/TargetSpecific.h>
#include <Functions/PerformanceAdaptors.h>
#include <Common/TargetSpecific.h>
#include <base/range.h>
#include <cmath>

View File

@ -0,0 +1,139 @@
#include <random>
#include <gtest/gtest.h>
#include <Functions/GatherUtils/Algorithms.h>
using namespace DB::GatherUtils;
auto uni_int_dist(int min, int max)
{
std::random_device rd;
std::mt19937 mt(rd());
std::uniform_int_distribution<> dist(min, max);
return std::make_pair(dist, mt);
}
template<class T>
void arrayInit(T* elements_to_have, size_t nb_elements_to_have, T* array_elements, size_t array_size, bool all_elements_present)
{
for (size_t i = 0; i < array_size; ++i)
{
array_elements[i] = i;
}
auto [dist, gen] = uni_int_dist(0, array_size - 1);
for (size_t i = 0; i < nb_elements_to_have; ++i)
{
elements_to_have[i] = array_elements[dist(gen)];
}
if (!all_elements_present)
{
/// make one element to be searched for missing from the target array
elements_to_have[nb_elements_to_have - 1] = array_size + 1;
}
}
void nullMapInit(UInt8 * null_map, size_t null_map_size, size_t nb_null_elements)
{
/// -2 to keep the last element of the array non-null
auto [dist, gen] = uni_int_dist(0, null_map_size - 2);
for (size_t i = 0; i < null_map_size; ++i)
{
null_map[i] = 0;
}
for (size_t i = 0; i < null_map_size - 1 && i < nb_null_elements; ++i)
{
null_map[dist(gen)] = 1;
}
}
template<class T>
bool testHasAll(size_t nb_elements_to_have, size_t array_size, bool with_null_maps, bool all_elements_present)
{
auto array_elements = std::make_unique<T[]>(array_size);
auto elements_to_have = std::make_unique<T[]>(nb_elements_to_have);
std::unique_ptr<UInt8[]> first_nm = nullptr, second_nm = nullptr;
if (with_null_maps)
{
first_nm = std::make_unique<UInt8[]>(array_size);
second_nm = std::make_unique<UInt8[]>(nb_elements_to_have);
/// add a null to elements to have, but not to the target array, making the answer negative
nullMapInit(first_nm.get(), array_size, 0);
nullMapInit(second_nm.get(), nb_elements_to_have, 1);
}
arrayInit(elements_to_have.get(), nb_elements_to_have, array_elements.get(), array_size, all_elements_present);
NumericArraySlice<T> first = {array_elements.get(), array_size};
NumericArraySlice<T> second = {elements_to_have.get(), nb_elements_to_have};
/// check whether all elements of the second array are also elements of the first array, overloaded for various combinations of types.
return sliceHasImplAnyAll<ArraySearchType::All, NumericArraySlice<T>, NumericArraySlice<T>, sliceEqualElements<T,T> >(
first, second, first_nm.get(), second_nm.get());
}
TEST(HasAll, integer)
{
bool test1 = testHasAll<int>(4, 100, false, true);
bool test2 = testHasAll<int>(4, 100, false, false);
bool test3 = testHasAll<int>(100, 4096, false, true);
bool test4 = testHasAll<int>(100, 4096, false, false);
ASSERT_EQ(test1, true);
ASSERT_EQ(test2, false);
ASSERT_EQ(test3, true);
ASSERT_EQ(test4, false);
}
TEST(HasAll, int64)
{
bool test1 = testHasAll<int64_t>(2, 100, false, true);
bool test2 = testHasAll<int64_t>(2, 100, false, false);
bool test3 = testHasAll<int64_t>(100, 4096, false, true);
bool test4 = testHasAll<int64_t>(100, 4096, false, false);
ASSERT_EQ(test1, true);
ASSERT_EQ(test2, false);
ASSERT_EQ(test3, true);
ASSERT_EQ(test4, false);
}
TEST(HasAll, int16)
{
bool test1 = testHasAll<int16_t>(2, 100, false, true);
bool test2 = testHasAll<int16_t>(2, 100, false, false);
bool test3 = testHasAll<int16_t>(100, 4096, false, true);
bool test4 = testHasAll<int16_t>(100, 4096, false, false);
ASSERT_EQ(test1, true);
ASSERT_EQ(test2, false);
ASSERT_EQ(test3, true);
ASSERT_EQ(test4, false);
}
TEST(HasAll, int8)
{
bool test1 = testHasAll<int8_t>(2, 100, false, true);
bool test2 = testHasAll<int8_t>(2, 100, false, false);
bool test3 = testHasAll<int8_t>(50, 125, false, true);
bool test4 = testHasAll<int8_t>(50, 125, false, false);
ASSERT_EQ(test1, true);
ASSERT_EQ(test2, false);
ASSERT_EQ(test3, true);
ASSERT_EQ(test4, false);
}
TEST(HasAllSingleNullElement, all)
{
bool test1 = testHasAll<int>(4, 100, true, true);
bool test2 = testHasAll<int64_t>(4, 100, true, true);
bool test3 = testHasAll<int16_t>(4, 100, true, true);
bool test4 = testHasAll<int8_t>(4, 100, true, true);
ASSERT_EQ(test1, false);
ASSERT_EQ(test2, false);
ASSERT_EQ(test3, false);
ASSERT_EQ(test4, false);
}

View File

@ -54,29 +54,12 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypes types;
Strings names;
for (const auto & argument : arguments)
{
types.emplace_back(argument.type);
names.emplace_back(argument.name);
}
/// Create named tuple if possible. We don't print tuple element names
/// because they are bad anyway -- aliases are not used, e.g. tuple(1 a)
/// will have element name '1' and not 'a'. If we ever change this, and
/// add the ability to access tuple elements by name, like tuple(1 a).a,
/// we should probably enable printing for better discoverability.
if (DataTypeTuple::canBeCreatedWithNames(names))
return std::make_shared<DataTypeTuple>(types, names, false /*print names*/);
return std::make_shared<DataTypeTuple>(types);
return std::make_shared<DataTypeTuple>(arguments);
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override

View File

@ -240,7 +240,7 @@ void ReadBufferFromS3::setReadUntilPosition(size_t position)
SeekableReadBuffer::Range ReadBufferFromS3::getRemainingReadRange() const
{
return Range{.left = static_cast<size_t>(offset), .right = read_until_position ? std::optional{read_until_position - 1} : std::nullopt};
return Range{ .left = static_cast<size_t>(offset), .right = read_until_position ? std::optional{read_until_position - 1} : std::nullopt };
}
std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize()

View File

@ -33,8 +33,11 @@ private:
String key;
UInt64 max_single_read_retries;
off_t offset = 0;
off_t read_until_position = 0;
/// These variables are atomic because they can be used for `logging only`
/// (where it is not important to get consistent result)
/// from separate thread other than the one which uses the buffer for s3 reading.
std::atomic<off_t> offset = 0;
std::atomic<off_t> read_until_position = 0;
Aws::S3::Model::GetObjectResult read_result;
std::unique_ptr<ReadBuffer> impl;

View File

@ -851,6 +851,8 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons
/// YYYY-MM-DD hh:mm:ss
static constexpr auto DateTimeStringInputSize = 19;
///YYYY-MM-DD
static constexpr auto DateStringInputSize = 10;
bool optimistic_path_for_date_time_input = s + DateTimeStringInputSize <= buf.buffer().end();
if (optimistic_path_for_date_time_input)
@ -861,16 +863,27 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons
UInt8 month = (s[5] - '0') * 10 + (s[6] - '0');
UInt8 day = (s[8] - '0') * 10 + (s[9] - '0');
UInt8 hour = (s[11] - '0') * 10 + (s[12] - '0');
UInt8 minute = (s[14] - '0') * 10 + (s[15] - '0');
UInt8 second = (s[17] - '0') * 10 + (s[18] - '0');
UInt8 hour = 0;
UInt8 minute = 0;
UInt8 second = 0;
///simply determine whether it is YYYY-MM-DD hh:mm:ss or YYYY-MM-DD by the content of the tenth character in an optimistic scenario
bool dt_long = (s[10] == ' ' || s[10] == 'T');
if (dt_long)
{
hour = (s[11] - '0') * 10 + (s[12] - '0');
minute = (s[14] - '0') * 10 + (s[15] - '0');
second = (s[17] - '0') * 10 + (s[18] - '0');
}
if (unlikely(year == 0))
datetime = 0;
else
datetime = date_lut.makeDateTime(year, month, day, hour, minute, second);
buf.position() += DateTimeStringInputSize;
if (dt_long)
buf.position() += DateTimeStringInputSize;
else
buf.position() += DateStringInputSize;
return ReturnType(true);
}
else

View File

@ -1527,6 +1527,21 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet &
return res;
}
ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameSet & sort_columns) const
{
std::unordered_set<const Node *> split_nodes;
for (const auto & sort_column : sort_columns)
if (const auto * node = tryFindInIndex(sort_column))
split_nodes.insert(node);
else
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Sorting column {} wasn't found in the ActionsDAG's index. DAG:\n{}", sort_column, dumpDAG());
auto res = split(split_nodes);
res.second->project_input = project_input;
return res;
}
ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & column_name) const
{
const auto * node = tryFindInIndex(column_name);

View File

@ -274,6 +274,10 @@ public:
/// Index of initial actions must contain column_name.
SplitResult splitActionsForFilter(const std::string & column_name) const;
/// Splits actions into two parts. The first part contains all the calculations required to calculate sort_columns.
/// The second contains the rest.
SplitResult splitActionsBySortingDescription(const NameSet & sort_columns) const;
/// Create actions which may calculate part of filter using only available_inputs.
/// If nothing may be calculated, returns nullptr.
/// Otherwise, return actions which inputs are from available_inputs.

View File

@ -632,8 +632,6 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper)
task.was_executed = true;
}
updateMaxDDLEntryID(task.entry_name);
/// Step 3: Create node in finished/ status dir and write execution status.
/// FIXME: if server fails right here, the task will be executed twice. We need WAL here.
/// NOTE: If ZooKeeper connection is lost here, we will try again to write query status.
@ -650,6 +648,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper)
active_node->setAlreadyRemoved();
task.completely_processed = true;
updateMaxDDLEntryID(task.entry_name);
}
@ -876,7 +875,7 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper)
/// We recursively delete all nodes except node_path/finished to prevent staled hosts from
/// creating node_path/active node (see createStatusDirs(...))
zookeeper->tryRemoveChildrenRecursive(node_path, "finished");
zookeeper->tryRemoveChildrenRecursive(node_path, /* probably_flat */ false, "finished");
/// And then we remove node_path and node_path/finished in a single transaction
Coordination::Requests ops;

View File

@ -136,7 +136,7 @@ std::string ExternalDictionariesLoader::resolveDictionaryNameFromDatabaseCatalog
if (qualified_name->database.empty())
{
/// Ether database name is not specified and we should use current one
/// Either database name is not specified and we should use current one
/// or it's an XML dictionary.
bool is_xml_dictionary = has(name);
if (is_xml_dictionary)

View File

@ -57,6 +57,10 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
/// Do not start user defined script during query analysis. Because user script startup could be heavy.
if (input_rows_count == 0)
return result_type->createColumn();
auto coordinator = executable_function->getCoordinator();
const auto & coordinator_configuration = coordinator->getConfiguration();
const auto & configuration = executable_function->getConfiguration();

View File

@ -44,16 +44,20 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &);
/// May split FilterStep and push down only part of it.
size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes);
/// Move ExpressionStep after SortingStep if possible.
/// May split ExpressionStep and lift up only a part of it.
size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes);
inline const auto & getOptimizations()
{
static const std::array<Optimization, 5> optimizations =
{{
static const std::array<Optimization, 6> optimizations = {{
{tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::optimize_plan},
{tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::optimize_plan},
{trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::optimize_plan},
{tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::optimize_plan},
{tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down},
}};
{tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan},
}};
return optimizations;
}

View File

@ -0,0 +1,77 @@
#include <Interpreters/ActionsDAG.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/SortingStep.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
namespace
{
const DB::DataStream & getChildOutputStream(DB::QueryPlan::Node & node)
{
if (node.children.size() != 1)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Node \"{}\" is expected to have only one child.", node.step->getName());
return node.children.front()->step->getOutputStream();
}
}
namespace DB::QueryPlanOptimizations
{
size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes)
{
if (parent_node->children.size() != 1)
return 0;
QueryPlan::Node * child_node = parent_node->children.front();
auto & parent_step = parent_node->step;
auto & child_step = child_node->step;
auto * sorting_step = typeid_cast<SortingStep *>(parent_step.get());
auto * expression_step = typeid_cast<ExpressionStep *>(child_step.get());
if (!sorting_step || !expression_step)
return 0;
NameSet sort_columns;
for (const auto & col : sorting_step->getSortDescription())
sort_columns.insert(col.column_name);
auto [needed_for_sorting, unneeded_for_sorting] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns);
// No calculations can be postponed.
if (unneeded_for_sorting->trivial())
return 0;
// Sorting (parent_node) -> Expression (child_node)
auto & node_with_needed = nodes.emplace_back();
std::swap(node_with_needed.children, child_node->children);
child_node->children = {&node_with_needed};
node_with_needed.step = std::make_unique<ExpressionStep>(getChildOutputStream(node_with_needed), std::move(needed_for_sorting));
node_with_needed.step->setStepDescription(child_step->getStepDescription());
// Sorting (parent_node) -> so far the origin Expression (child_node) -> NeededCalculations (node_with_needed)
std::swap(parent_step, child_step);
// so far the origin Expression (parent_node) -> Sorting (child_node) -> NeededCalculations (node_with_needed)
sorting_step->updateInputStream(getChildOutputStream(*child_node));
auto input_header = sorting_step->getInputStreams().at(0).header;
sorting_step->updateOutputStream(std::move(input_header));
auto description = parent_step->getStepDescription();
parent_step = std::make_unique<DB::ExpressionStep>(child_step->getOutputStream(), std::move(unneeded_for_sorting));
parent_step->setStepDescription(description + " [lifted up part]");
// UneededCalculations (parent_node) -> Sorting (child_node) -> NeededCalculations (node_with_needed)
return 3;
}
}

View File

@ -1,11 +1,12 @@
#include <stdexcept>
#include <IO/Operators.h>
#include <Processors/Merges/MergingSortedTransform.h>
#include <Processors/QueryPlan/SortingStep.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/Transforms/FinishSortingTransform.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MergeSortingTransform.h>
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Transforms/FinishSortingTransform.h>
#include <Processors/Merges/MergingSortedTransform.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <IO/Operators.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Common/JSONBuilder.h>
namespace DB
@ -88,6 +89,18 @@ SortingStep::SortingStep(
output_stream->sort_mode = DataStream::SortMode::Stream;
}
void SortingStep::updateInputStream(DataStream input_stream)
{
input_streams.clear();
input_streams.emplace_back(std::move(input_stream));
}
void SortingStep::updateOutputStream(Block result_header)
{
output_stream = createOutputStream(input_streams.at(0), std::move(result_header), getDataStreamTraits());
updateDistinctColumns(output_stream->header, output_stream->distinct_columns);
}
void SortingStep::updateLimit(size_t limit_)
{
if (limit_ && (limit == 0 || limit_ < limit))

View File

@ -49,6 +49,11 @@ public:
/// Add limit or change it to lower value.
void updateLimit(size_t limit_);
void updateInputStream(DataStream input_stream);
void updateOutputStream(Block result_header);
SortDescription getSortDescription() const { return result_description; }
private:
enum class Type

View File

@ -94,6 +94,8 @@ bool RemoteReadBuffer::nextImpl()
return status;
}
//file_buffer::pos should increase correspondingly when RemoteReadBuffer is consumed, otherwise start_offset will be incorrect.
local_file_holder->file_buffer->position() = local_file_holder->file_buffer->buffer().begin() + BufferBase::offset();
auto start_offset = local_file_holder->file_buffer->getPosition();
auto end_offset = start_offset + local_file_holder->file_buffer->internalBuffer().size();
local_file_holder->file_cache_controller->value().waitMoreData(start_offset, end_offset);

View File

@ -723,6 +723,7 @@ bool StorageFileLog::streamToViews()
size_t rows = 0;
{
block_io.pipeline.complete(std::move(input));
block_io.pipeline.setNumThreads(max_streams_number);
block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); });
CompletedPipelineExecutor executor(block_io.pipeline);
executor.execute();

View File

@ -6,6 +6,16 @@
#include <base/logger_useful.h>
#include <Interpreters/Context.h>
#include <Common/ProfileEvents.h>
namespace ProfileEvents
{
extern const Event KafkaMessagesRead;
extern const Event KafkaMessagesFailed;
extern const Event KafkaRowsRead;
extern const Event KafkaRowsRejected;
}
namespace DB
{
namespace ErrorCodes
@ -85,6 +95,8 @@ Chunk KafkaSource::generateImpl()
auto on_error = [&](const MutableColumns & result_columns, Exception & e)
{
ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed);
if (put_error_to_stream)
{
exception_message = e.message();
@ -117,7 +129,11 @@ Chunk KafkaSource::generateImpl()
size_t new_rows = 0;
exception_message.reset();
if (buffer->poll())
{
// poll provide one message at a time to the input_format
ProfileEvents::increment(ProfileEvents::KafkaMessagesRead);
new_rows = executor.execute();
}
if (new_rows)
{
@ -128,6 +144,8 @@ Chunk KafkaSource::generateImpl()
if (buffer->isStalled())
throw Exception("Polled messages became unusable", ErrorCodes::LOGICAL_ERROR);
ProfileEvents::increment(ProfileEvents::KafkaRowsRead, new_rows);
buffer->storeLastReadMessageOffset();
auto topic = buffer->currentTopic();
@ -212,8 +230,18 @@ Chunk KafkaSource::generateImpl()
}
}
if (buffer->polledDataUnusable() || total_rows == 0)
if (total_rows == 0)
{
return {};
}
else if (buffer->polledDataUnusable())
{
// the rows were counted already before by KafkaRowsRead,
// so let's count the rows we ignore separately
// (they will be retried after the rebalance)
ProfileEvents::increment(ProfileEvents::KafkaRowsRejected, total_rows);
return {};
}
/// MATERIALIZED columns can be added here, but I think
// they are not needed here:

View File

@ -10,6 +10,26 @@
#include <boost/algorithm/string/join.hpp>
#include <algorithm>
#include <Common/CurrentMetrics.h>
#include <Common/ProfileEvents.h>
namespace CurrentMetrics
{
extern const Metric KafkaAssignedPartitions;
extern const Metric KafkaConsumersWithAssignment;
}
namespace ProfileEvents
{
extern const Event KafkaRebalanceRevocations;
extern const Event KafkaRebalanceAssignments;
extern const Event KafkaRebalanceErrors;
extern const Event KafkaMessagesPolled;
extern const Event KafkaCommitFailures;
extern const Event KafkaCommits;
extern const Event KafkaConsumerErrors;
}
namespace DB
{
@ -45,6 +65,9 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer(
// called (synchronously, during poll) when we enter the consumer group
consumer->set_assignment_callback([this](const cppkafka::TopicPartitionList & topic_partitions)
{
CurrentMetrics::add(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size());
ProfileEvents::increment(ProfileEvents::KafkaRebalanceAssignments);
if (topic_partitions.empty())
{
LOG_INFO(log, "Got empty assignment: Not enough partitions in the topic for all consumers?");
@ -52,6 +75,7 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer(
else
{
LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions);
CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1);
}
assignment = topic_partitions;
@ -60,10 +84,18 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer(
// called (synchronously, during poll) when we leave the consumer group
consumer->set_revocation_callback([this](const cppkafka::TopicPartitionList & topic_partitions)
{
CurrentMetrics::sub(CurrentMetrics::KafkaAssignedPartitions, topic_partitions.size());
ProfileEvents::increment(ProfileEvents::KafkaRebalanceRevocations);
// Rebalance is happening now, and now we have a chance to finish the work
// with topics/partitions we were working with before rebalance
LOG_TRACE(log, "Rebalance initiated. Revoking partitions: {}", topic_partitions);
if (!topic_partitions.empty())
{
CurrentMetrics::sub(CurrentMetrics::KafkaConsumersWithAssignment, 1);
}
// we can not flush data to target from that point (it is pulled, not pushed)
// so the best we can now it to
// 1) repeat last commit in sync mode (async could be still in queue, we need to be sure is is properly committed before rebalance)
@ -91,6 +123,7 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer(
consumer->set_rebalance_error_callback([this](cppkafka::Error err)
{
LOG_ERROR(log, "Rebalance error: {}", err);
ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors);
});
}
@ -229,8 +262,14 @@ void ReadBufferFromKafkaConsumer::commit()
if (!committed)
{
// TODO: insert atomicity / transactions is needed here (possibility to rollback, on 2 phase commits)
ProfileEvents::increment(ProfileEvents::KafkaCommitFailures);
throw Exception("All commit attempts failed. Last block was already written to target table(s), but was not committed to Kafka.", ErrorCodes::CANNOT_COMMIT_OFFSET);
}
else
{
ProfileEvents::increment(ProfileEvents::KafkaCommits);
}
}
else
{
@ -423,6 +462,8 @@ bool ReadBufferFromKafkaConsumer::poll()
return false;
}
ProfileEvents::increment(ProfileEvents::KafkaMessagesPolled, messages.size());
stalled_status = NOT_STALLED;
allowed = true;
return true;
@ -436,6 +477,7 @@ size_t ReadBufferFromKafkaConsumer::filterMessageErrors()
{
if (auto error = message.get_error())
{
ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors);
LOG_ERROR(log, "Consumer error: {}", error);
return true;
}

View File

@ -5,6 +5,12 @@
#include <IO/ReadBuffer.h>
#include <cppkafka/cppkafka.h>
#include <Common/CurrentMetrics.h>
namespace CurrentMetrics
{
extern const Metric KafkaConsumers;
}
namespace Poco
{
@ -67,6 +73,7 @@ public:
private:
using Messages = std::vector<cppkafka::Message>;
CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers};
enum StalledStatus
{

View File

@ -41,6 +41,26 @@
#include <Common/typeid_cast.h>
#include <Common/CurrentMetrics.h>
#include <Common/ProfileEvents.h>
namespace CurrentMetrics
{
extern const Metric KafkaLibrdkafkaThreads;
extern const Metric KafkaBackgroundReads;
extern const Metric KafkaConsumersInUse;
extern const Metric KafkaWrites;
}
namespace ProfileEvents
{
extern const Event KafkaDirectReads;
extern const Event KafkaBackgroundReads;
extern const Event KafkaWrites;
}
namespace DB
{
@ -58,6 +78,7 @@ struct StorageKafkaInterceptors
static rd_kafka_resp_err_t rdKafkaOnThreadStart(rd_kafka_t *, rd_kafka_thread_type_t thread_type, const char *, void * ctx)
{
StorageKafka * self = reinterpret_cast<StorageKafka *>(ctx);
CurrentMetrics::add(CurrentMetrics::KafkaLibrdkafkaThreads, 1);
const auto & storage_id = self->getStorageID();
const auto & table = storage_id.getTableName();
@ -89,6 +110,7 @@ struct StorageKafkaInterceptors
static rd_kafka_resp_err_t rdKafkaOnThreadExit(rd_kafka_t *, rd_kafka_thread_type_t, const char *, void * ctx)
{
StorageKafka * self = reinterpret_cast<StorageKafka *>(ctx);
CurrentMetrics::sub(CurrentMetrics::KafkaLibrdkafkaThreads, 1);
std::lock_guard lock(self->thread_statuses_mutex);
const auto it = std::find_if(self->thread_statuses.begin(), self->thread_statuses.end(), [](const auto & thread_status_ptr)
@ -279,6 +301,8 @@ Pipe StorageKafka::read(
if (mv_attached)
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageKafka with attached materialized views");
ProfileEvents::increment(ProfileEvents::KafkaDirectReads);
/// Always use all consumers at once, otherwise SELECT may not read messages from all partitions.
Pipes pipes;
pipes.reserve(num_created_consumers);
@ -304,6 +328,9 @@ SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr &
auto modified_context = Context::createCopy(local_context);
modified_context->applySettingsChanges(settings_adjustments);
CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaWrites};
ProfileEvents::increment(ProfileEvents::KafkaWrites);
if (topics.size() > 1)
throw Exception("Can't write to Kafka table with multiple topics!", ErrorCodes::NOT_IMPLEMENTED);
return std::make_shared<KafkaSink>(*this, metadata_snapshot, modified_context);
@ -358,6 +385,7 @@ void StorageKafka::pushReadBuffer(ConsumerBufferPtr buffer)
std::lock_guard lock(mutex);
buffers.push_back(buffer);
semaphore.set();
CurrentMetrics::sub(CurrentMetrics::KafkaConsumersInUse, 1);
}
@ -382,6 +410,7 @@ ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout)
std::lock_guard lock(mutex);
auto buffer = buffers.back();
buffers.pop_back();
CurrentMetrics::add(CurrentMetrics::KafkaConsumersInUse, 1);
return buffer;
}
@ -615,6 +644,9 @@ bool StorageKafka::streamToViews()
if (!table)
throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR);
CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads};
ProfileEvents::increment(ProfileEvents::KafkaBackgroundReads);
auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr());
// Create an INSERT query for streaming data

View File

@ -3,6 +3,16 @@
#include "Columns/ColumnString.h"
#include "Columns/ColumnsNumber.h"
#include <Common/ProfileEvents.h>
namespace ProfileEvents
{
extern const Event KafkaRowsWritten;
extern const Event KafkaProducerFlushes;
extern const Event KafkaMessagesProduced;
extern const Event KafkaProducerErrors;
}
namespace DB
{
WriteBufferToKafkaProducer::WriteBufferToKafkaProducer(
@ -53,6 +63,8 @@ WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer()
void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t current_row)
{
ProfileEvents::increment(ProfileEvents::KafkaRowsWritten);
if (++rows % max_rows == 0)
{
const std::string & last_chunk = chunks.back();
@ -103,8 +115,10 @@ void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t curren
producer->poll(timeout);
continue;
}
ProfileEvents::increment(ProfileEvents::KafkaProducerErrors);
throw;
}
ProfileEvents::increment(ProfileEvents::KafkaMessagesProduced);
break;
}
@ -126,9 +140,12 @@ void WriteBufferToKafkaProducer::flush()
{
if (e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT)
continue;
ProfileEvents::increment(ProfileEvents::KafkaProducerErrors);
throw;
}
ProfileEvents::increment(ProfileEvents::KafkaProducerFlushes);
break;
}
}

View File

@ -7,6 +7,14 @@
#include <list>
#include <Common/CurrentMetrics.h>
namespace CurrentMetrics
{
extern const Metric KafkaProducers;
}
namespace DB
{
class Block;
@ -32,6 +40,7 @@ private:
void nextImpl() override;
void addChunk();
void reinitializeChunks();
CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaProducers};
ProducerPtr producer;
const std::string topic;

View File

@ -2962,7 +2962,8 @@ void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part)
{
auto lock = lockParts();
LOG_TRACE(log, "Trying to immediately remove part {}", part->getNameWithState());
auto part_name_with_state = part->getNameWithState();
LOG_TRACE(log, "Trying to immediately remove part {}", part_name_with_state);
if (part->getState() != DataPartState::Temporary)
{
@ -2973,7 +2974,16 @@ void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part)
part.reset();
if (!((*it)->getState() == DataPartState::Outdated && it->unique()))
{
if ((*it)->getState() != DataPartState::Outdated)
LOG_WARNING(log, "Cannot immediately remove part {} because it's not in Outdated state "
"usage counter {}", part_name_with_state, it->use_count());
if (!it->unique())
LOG_WARNING(log, "Cannot immediately remove part {} because someone using it right now "
"usage counter {}", part_name_with_state, it->use_count());
return;
}
modifyPartState(it, DataPartState::Deleting);
@ -4641,6 +4651,7 @@ static void selectBestProjection(
Block MergeTreeData::getMinMaxCountProjectionBlock(
const StorageMetadataPtr & metadata_snapshot,
const Names & required_columns,
bool has_filter,
const SelectQueryInfo & query_info,
const DataPartsVector & parts,
DataPartsVector & normal_parts,
@ -4655,13 +4666,12 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
auto block = metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty();
bool need_primary_key_max_column = false;
const auto & primary_key_max_column_name = metadata_snapshot->minmax_count_projection->primary_key_max_column_name;
NameSet required_columns_set(required_columns.begin(), required_columns.end());
if (!primary_key_max_column_name.empty())
{
need_primary_key_max_column = std::any_of(
required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; });
}
need_primary_key_max_column = required_columns_set.contains(primary_key_max_column_name);
auto partition_minmax_count_columns = block.mutateColumns();
auto partition_minmax_count_column_names = block.getNames();
auto insert = [](ColumnAggregateFunction & column, const Field & value)
{
auto func = column.getAggregateFunction();
@ -4670,51 +4680,76 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
size_t align_of_state = func->alignOfData();
auto * place = arena.alignedAlloc(size_of_state, align_of_state);
func->create(place);
auto value_column = func->getReturnType()->createColumnConst(1, value)->convertToFullColumnIfConst();
const auto * value_column_ptr = value_column.get();
func->add(place, &value_column_ptr, 0, &arena);
if (const AggregateFunctionCount * agg_count = typeid_cast<const AggregateFunctionCount *>(func.get()))
agg_count->set(place, value.get<UInt64>());
else
{
auto value_column = func->getReturnType()->createColumnConst(1, value)->convertToFullColumnIfConst();
const auto * value_column_ptr = value_column.get();
func->add(place, &value_column_ptr, 0, &arena);
}
column.insertFrom(place);
};
ASTPtr expression_ast;
Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */);
if (virtual_columns_block.rows() == 0)
return {};
Block virtual_columns_block;
auto virtual_block = getSampleBlockWithVirtualColumns();
bool has_virtual_column = std::any_of(required_columns.begin(), required_columns.end(), [&](const auto & name) { return virtual_block.has(name); });
if (has_virtual_column || has_filter)
{
virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */);
if (virtual_columns_block.rows() == 0)
return {};
}
size_t rows = parts.size();
ColumnPtr part_name_column;
std::optional<PartitionPruner> partition_pruner;
std::optional<KeyCondition> minmax_idx_condition;
DataTypes minmax_columns_types;
if (metadata_snapshot->hasPartitionKey())
if (has_filter)
{
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_columns_names = getMinMaxColumnsNames(partition_key);
minmax_columns_types = getMinMaxColumnsTypes(partition_key);
if (metadata_snapshot->hasPartitionKey())
{
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_columns_names = getMinMaxColumnsNames(partition_key);
minmax_columns_types = getMinMaxColumnsTypes(partition_key);
minmax_idx_condition.emplace(
query_info,
query_context,
minmax_columns_names,
getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(query_context)));
partition_pruner.emplace(metadata_snapshot, query_info, query_context, false /* strict */);
minmax_idx_condition.emplace(
query_info,
query_context,
minmax_columns_names,
getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(query_context)));
partition_pruner.emplace(metadata_snapshot, query_info, query_context, false /* strict */);
}
// Generate valid expressions for filtering
ASTPtr expression_ast;
VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, query_context, virtual_columns_block, expression_ast);
if (expression_ast)
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, query_context, expression_ast);
rows = virtual_columns_block.rows();
part_name_column = virtual_columns_block.getByName("_part").column;
}
// Generate valid expressions for filtering
VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, query_context, virtual_columns_block, expression_ast);
if (expression_ast)
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, query_context, expression_ast);
size_t rows = virtual_columns_block.rows();
const ColumnString & part_name_column = typeid_cast<const ColumnString &>(*virtual_columns_block.getByName("_part").column);
size_t part_idx = 0;
auto filter_column = ColumnUInt8::create();
auto & filter_column_data = filter_column->getData();
for (size_t row = 0; row < rows; ++row)
DataPartsVector real_parts;
real_parts.reserve(rows);
for (size_t row = 0, part_idx = 0; row < rows; ++row, ++part_idx)
{
while (parts[part_idx]->name != part_name_column.getDataAt(row))
++part_idx;
if (part_name_column)
{
while (parts[part_idx]->name != part_name_column->getDataAt(row))
++part_idx;
}
const auto & part = parts[part_idx];
if (part->isEmpty())
continue;
if (!part->minmax_idx->initialized)
throw Exception("Found a non-empty part with uninitialized minmax_idx. It's a bug", ErrorCodes::LOGICAL_ERROR);
@ -4743,48 +4778,12 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
continue;
}
real_parts.push_back(part);
filter_column_data.back() = 1;
size_t pos = 0;
for (size_t i : metadata_snapshot->minmax_count_projection->partition_value_indices)
{
if (i >= part->partition.value.size())
throw Exception("Partition value index is out of boundary. It's a bug", ErrorCodes::LOGICAL_ERROR);
partition_minmax_count_columns[pos++]->insert(part->partition.value[i]);
}
size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size();
for (size_t i = 0; i < minmax_idx_size; ++i)
{
auto & min_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos++]);
auto & max_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos++]);
const auto & range = part->minmax_idx->hyperrectangle[i];
insert(min_column, range.left);
insert(max_column, range.right);
}
if (!primary_key_max_column_name.empty())
{
const auto & primary_key_column = *part->index[0];
auto & min_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos++]);
auto & max_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos++]);
insert(min_column, primary_key_column[0]);
insert(max_column, primary_key_column[primary_key_column.size() - 1]);
}
{
auto & column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns.back());
auto func = column.getAggregateFunction();
Arena & arena = column.createOrGetArena();
size_t size_of_state = func->sizeOfData();
size_t align_of_state = func->alignOfData();
auto * place = arena.alignedAlloc(size_of_state, align_of_state);
func->create(place);
const AggregateFunctionCount & agg_count = assert_cast<const AggregateFunctionCount &>(*func);
agg_count.set(place, part->rows_count);
column.insertFrom(place);
}
}
block.setColumns(std::move(partition_minmax_count_columns));
if (real_parts.empty())
return {};
FilterDescription filter(*filter_column);
for (size_t i = 0; i < virtual_columns_block.columns(); ++i)
@ -4793,8 +4792,78 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
column = column->filter(*filter.data, -1);
}
if (block.rows() == 0)
return {};
size_t pos = 0;
for (size_t i : metadata_snapshot->minmax_count_projection->partition_value_indices)
{
if (required_columns_set.contains(partition_minmax_count_column_names[pos]))
for (const auto & part : real_parts)
partition_minmax_count_columns[pos]->insert(part->partition.value[i]);
++pos;
}
size_t minmax_idx_size = real_parts.front()->minmax_idx->hyperrectangle.size();
for (size_t i = 0; i < minmax_idx_size; ++i)
{
if (required_columns_set.contains(partition_minmax_count_column_names[pos]))
{
for (const auto & part : real_parts)
{
const auto & range = part->minmax_idx->hyperrectangle[i];
auto & min_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos]);
insert(min_column, range.left);
}
}
++pos;
if (required_columns_set.contains(partition_minmax_count_column_names[pos]))
{
for (const auto & part : real_parts)
{
const auto & range = part->minmax_idx->hyperrectangle[i];
auto & max_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos]);
insert(max_column, range.right);
}
}
++pos;
}
if (!primary_key_max_column_name.empty())
{
if (required_columns_set.contains(partition_minmax_count_column_names[pos]))
{
for (const auto & part : real_parts)
{
const auto & primary_key_column = *part->index[0];
auto & min_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos]);
insert(min_column, primary_key_column[0]);
}
}
++pos;
if (required_columns_set.contains(partition_minmax_count_column_names[pos]))
{
for (const auto & part : real_parts)
{
const auto & primary_key_column = *part->index[0];
auto & max_column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns[pos]);
insert(max_column, primary_key_column[primary_key_column.size() - 1]);
}
}
++pos;
}
bool has_count
= std::any_of(required_columns.begin(), required_columns.end(), [&](const auto & name) { return startsWith(name, "count"); });
if (has_count)
{
for (const auto & part : real_parts)
{
auto & column = assert_cast<ColumnAggregateFunction &>(*partition_minmax_count_columns.back());
insert(column, part->rows_count);
}
}
block.setColumns(std::move(partition_minmax_count_columns));
Block res;
for (const auto & name : required_columns)
@ -4803,6 +4872,11 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
res.insert(virtual_columns_block.getByName(name));
else if (block.has(name))
res.insert(block.getByName(name));
else if (startsWith(name, "count")) // special case to match count(...) variants
{
const auto & column = block.getByName("count()");
res.insert({column.column, column.type, name});
}
else
throw Exception(
ErrorCodes::LOGICAL_ERROR,
@ -4974,7 +5048,7 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
};
auto virtual_block = getSampleBlockWithVirtualColumns();
auto add_projection_candidate = [&](const ProjectionDescription & projection)
auto add_projection_candidate = [&](const ProjectionDescription & projection, bool minmax_count_projection = false)
{
ProjectionCandidate candidate{};
candidate.desc = &projection;
@ -5001,22 +5075,30 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
if (projection.type == ProjectionDescription::Type::Aggregate && analysis_result.need_aggregate && can_use_aggregate_projection)
{
bool match = true;
Block aggregates;
// Let's first check if all aggregates are provided by current projection
for (const auto & aggregate : select.getQueryAnalyzer()->aggregates())
{
const auto * column = sample_block.findByName(aggregate.column_name);
if (column)
aggregates.insert(*column);
else
if (const auto * column = sample_block.findByName(aggregate.column_name))
{
match = false;
break;
aggregates.insert(*column);
continue;
}
}
if (!match)
// We can treat every count_not_null_column as count() when selecting minmax_count_projection
if (minmax_count_projection && dynamic_cast<const AggregateFunctionCount *>(aggregate.function.get()))
{
const auto * count_column = sample_block.findByName("count()");
if (!count_column)
throw Exception(
ErrorCodes::LOGICAL_ERROR, "`count()` column is missing when minmax_count_projection == true. It is a bug");
aggregates.insert({count_column->column, count_column->type, aggregate.column_name});
continue;
}
// No match
return;
}
// Check if all aggregation keys can be either provided by some action, or by current
// projection directly. Reshape the `before_aggregation` action DAG so that it only
@ -5036,9 +5118,20 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
candidate.before_aggregation->reorderAggregationKeysForProjection(key_name_pos_map);
candidate.before_aggregation->addAggregatesViaProjection(aggregates);
// minmax_count_projections only have aggregation actions
if (minmax_count_projection)
candidate.required_columns = {required_columns.begin(), required_columns.end()};
if (rewrite_before_where(candidate, projection, required_columns, sample_block_for_keys, aggregates))
{
candidate.required_columns = {required_columns.begin(), required_columns.end()};
if (minmax_count_projection)
{
candidate.before_where = nullptr;
candidate.prewhere_info = nullptr;
}
else
candidate.required_columns = {required_columns.begin(), required_columns.end()};
for (const auto & aggregate : aggregates)
candidate.required_columns.push_back(aggregate.name);
candidates.push_back(std::move(candidate));
@ -5069,11 +5162,11 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
ProjectionCandidate * selected_candidate = nullptr;
size_t min_sum_marks = std::numeric_limits<size_t>::max();
if (metadata_snapshot->minmax_count_projection)
add_projection_candidate(*metadata_snapshot->minmax_count_projection);
std::optional<ProjectionCandidate> minmax_conut_projection_candidate;
add_projection_candidate(*metadata_snapshot->minmax_count_projection, true);
std::optional<ProjectionCandidate> minmax_count_projection_candidate;
if (!candidates.empty())
{
minmax_conut_projection_candidate.emplace(std::move(candidates.front()));
minmax_count_projection_candidate.emplace(std::move(candidates.front()));
candidates.clear();
}
MergeTreeDataSelectExecutor reader(*this);
@ -5086,21 +5179,22 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
auto parts = getDataPartsVector();
// If minmax_count_projection is a valid candidate, check its completeness.
if (minmax_conut_projection_candidate)
if (minmax_count_projection_candidate)
{
DataPartsVector normal_parts;
query_info.minmax_count_projection_block = getMinMaxCountProjectionBlock(
metadata_snapshot,
minmax_conut_projection_candidate->required_columns,
minmax_count_projection_candidate->required_columns,
analysis_result.prewhere_info || analysis_result.before_where,
query_info,
parts,
normal_parts,
max_added_blocks.get(),
query_context);
if (query_info.minmax_count_projection_block && minmax_conut_projection_candidate->prewhere_info)
if (query_info.minmax_count_projection_block && minmax_count_projection_candidate->prewhere_info)
{
const auto & prewhere_info = minmax_conut_projection_candidate->prewhere_info;
const auto & prewhere_info = minmax_count_projection_candidate->prewhere_info;
if (prewhere_info->alias_actions)
ExpressionActions(prewhere_info->alias_actions, actions_settings).execute(query_info.minmax_count_projection_block);
@ -5119,7 +5213,7 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
if (normal_parts.empty())
{
selected_candidate = &*minmax_conut_projection_candidate;
selected_candidate = &*minmax_count_projection_candidate;
selected_candidate->complete = true;
min_sum_marks = query_info.minmax_count_projection_block.rows();
}
@ -5143,7 +5237,7 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
if (!normal_result_ptr->error())
{
selected_candidate = &*minmax_conut_projection_candidate;
selected_candidate = &*minmax_count_projection_candidate;
selected_candidate->merge_tree_normal_select_result_ptr = normal_result_ptr;
min_sum_marks = query_info.minmax_count_projection_block.rows() + normal_result_ptr->marks();
}

View File

@ -383,6 +383,8 @@ public:
/// Build a block of minmax and count values of a MergeTree table. These values are extracted
/// from minmax_indices, the first expression of primary key, and part rows.
///
/// has_filter - if query has no filter, bypass partition pruning completely
///
/// query_info - used to filter unneeded parts
///
/// parts - part set to filter
@ -393,6 +395,7 @@ public:
Block getMinMaxCountProjectionBlock(
const StorageMetadataPtr & metadata_snapshot,
const Names & required_columns,
bool has_filter,
const SelectQueryInfo & query_info,
const DataPartsVector & parts,
DataPartsVector & normal_parts,

View File

@ -319,19 +319,22 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
}
bool skip_sanity_checks = false;
if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
/// It does not make sense for CREATE query
if (attach)
{
skip_sanity_checks = true;
current_zookeeper->remove(replica_path + "/flags/force_restore_data");
if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
{
skip_sanity_checks = true;
current_zookeeper->remove(replica_path + "/flags/force_restore_data");
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag {}/flags/force_restore_data).", replica_path);
}
else if (has_force_restore_data_flag)
{
skip_sanity_checks = true;
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag {}/flags/force_restore_data).", replica_path);
}
else if (has_force_restore_data_flag)
{
skip_sanity_checks = true;
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
}
}
loadDataParts(skip_sanity_checks);
@ -569,35 +572,31 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
{
auto zookeeper = getZooKeeper();
/// Working with quorum.
zookeeper->createIfNotExists(zookeeper_path + "/quorum", String());
zookeeper->createIfNotExists(zookeeper_path + "/quorum/parallel", String());
zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", String());
zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", String());
/// Tracking lag of replicas.
zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", String());
zookeeper->createIfNotExists(replica_path + "/max_processed_insert_time", String());
/// Mutations
zookeeper->createIfNotExists(zookeeper_path + "/mutations", String());
zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
std::vector<zkutil::ZooKeeper::FutureCreate> futures;
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/parallel", String(), zkutil::CreateMode::Persistent));
/// Nodes for remote fs zero-copy replication
const auto settings = getSettings();
if (settings->allow_remote_fs_zero_copy_replication)
{
zookeeper->createIfNotExists(zookeeper_path + "/zero_copy_s3", String());
zookeeper->createIfNotExists(zookeeper_path + "/zero_copy_s3/shared", String());
zookeeper->createIfNotExists(zookeeper_path + "/zero_copy_hdfs", String());
zookeeper->createIfNotExists(zookeeper_path + "/zero_copy_hdfs/shared", String());
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/zero_copy_s3", String(), zkutil::CreateMode::Persistent));
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/zero_copy_s3/shared", String(), zkutil::CreateMode::Persistent));
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/zero_copy_hdfs", String(), zkutil::CreateMode::Persistent));
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/zero_copy_hdfs/shared", String(), zkutil::CreateMode::Persistent));
}
/// Part movement.
zookeeper->createIfNotExists(zookeeper_path + "/part_moves_shard", String());
zookeeper->createIfNotExists(zookeeper_path + "/pinned_part_uuids", getPinnedPartUUIDs()->toString());
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/part_moves_shard", String(), zkutil::CreateMode::Persistent));
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/pinned_part_uuids", getPinnedPartUUIDs()->toString(), zkutil::CreateMode::Persistent));
/// For ALTER PARTITION with multi-leaders
zookeeper->createIfNotExists(zookeeper_path + "/alter_partition_version", String());
futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/alter_partition_version", String(), zkutil::CreateMode::Persistent));
for (auto & future : futures)
{
auto res = future.get();
if (res.error != Coordination::Error::ZOK && res.error != Coordination::Error::ZNODEEXISTS)
throw Coordination::Exception(fmt::format("Failed to create new nodes at {}", zookeeper_path), res.error);
}
}
@ -672,6 +671,16 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name,
zkutil::CreateMode::Persistent));
/// The following 4 nodes were added in version 1.1.xxx, so we create them here, not in createNewZooKeeperNodes()
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/quorum", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/quorum/last_part", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/quorum/failed_parts", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/mutations", "",
zkutil::CreateMode::Persistent));
/// And create first replica atomically. See also "createReplica" method that is used to create not the first replicas.
ops.emplace_back(zkutil::makeCreateRequest(replica_path, "",
@ -695,6 +704,14 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version),
zkutil::CreateMode::Persistent));
/// The following 3 nodes were added in version 1.1.xxx, so we create them here, not in createNewZooKeeperNodes()
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/min_unprocessed_insert_time", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/max_processed_insert_time", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "",
zkutil::CreateMode::Persistent));
Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code == Coordination::Error::ZNODEEXISTS)
@ -760,6 +777,14 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version),
zkutil::CreateMode::Persistent));
/// The following 3 nodes were added in version 1.1.xxx, so we create them here, not in createNewZooKeeperNodes()
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/min_unprocessed_insert_time", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/max_processed_insert_time", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "",
zkutil::CreateMode::Persistent));
/// Check version of /replicas to see if there are any replicas created at the same moment of time.
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, replicas_stat.version));
@ -803,13 +828,14 @@ void StorageReplicatedMergeTree::drop()
throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
shutdown();
dropReplica(zookeeper, zookeeper_path, replica_name, log);
dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings());
}
dropAllData();
}
void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger)
void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica,
Poco::Logger * logger, MergeTreeSettingsPtr table_settings)
{
if (zookeeper->expired())
throw Exception("Table was not dropped because ZooKeeper session has expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED);
@ -841,18 +867,43 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con
/// "The local set of parts of table X doesn't look like the set of parts in ZooKeeper"
///
{
Strings children = zookeeper->getChildren(remote_replica_path);
/// Remove metadata first
[[maybe_unused]] auto code = zookeeper->tryRemove(fs::path(remote_replica_path) / "metadata");
assert(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE);
if (std::find(children.begin(), children.end(), "metadata") != children.end())
zookeeper->remove(fs::path(remote_replica_path) / "metadata");
for (const auto & child : children)
/// Then try to remove paths that are known to be flat (all children are leafs)
Strings flat_nodes = {"flags", "queue"};
if (table_settings && table_settings->use_minimalistic_part_header_in_zookeeper)
flat_nodes.emplace_back("parts");
for (const auto & node : flat_nodes)
{
if (child != "metadata")
zookeeper->removeRecursive(fs::path(remote_replica_path) / child);
bool removed_quickly = zookeeper->tryRemoveChildrenRecursive(fs::path(remote_replica_path) / node, /* probably flat */ true);
if (!removed_quickly)
LOG_WARNING(logger, "Failed to quickly remove node '{}' and its children, fell back to recursive removal (replica: {})",
node, remote_replica_path);
}
zookeeper->remove(remote_replica_path);
/// Then try to remove nodes that are known to have no children (and should always exist)
Coordination::Requests ops;
for (const auto & node : flat_nodes)
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/" + node, -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/columns", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/host", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/is_lost", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/log_pointer", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/max_processed_insert_time", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/min_unprocessed_insert_time", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/metadata_version", -1));
ops.emplace_back(zkutil::makeRemoveRequest(remote_replica_path + "/mutation_pointer", -1));
Coordination::Responses res;
code = zookeeper->tryMulti(ops, res);
if (code != Coordination::Error::ZOK)
LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (replica: {}). Will remove recursively.",
Coordination::errorMessage(code), remote_replica_path);
/// And finally remove everything else recursively
zookeeper->tryRemoveRecursive(remote_replica_path);
}
/// It may left some garbage if replica_path subtree are concurrently modified
@ -912,17 +963,47 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper
const String & zookeeper_path, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger)
{
bool completely_removed = false;
/// NOTE /block_numbers/ actually is not flat, because /block_numbers/<partition_id>/ may have ephemeral children,
/// but we assume that all ephemeral block locks are already removed when table is being dropped.
static constexpr std::array flat_nodes = {"block_numbers", "blocks", "leader_election", "log", "mutations", "pinned_part_uuids"};
/// First try to remove paths that are known to be flat
for (const auto * node : flat_nodes)
{
bool removed_quickly = zookeeper->tryRemoveChildrenRecursive(fs::path(zookeeper_path) / node, /* probably flat */ true);
if (!removed_quickly)
LOG_WARNING(logger, "Failed to quickly remove node '{}' and its children, fell back to recursive removal (table: {})",
node, zookeeper_path);
}
/// Then try to remove nodes that are known to have no children (and should always exist)
Coordination::Requests ops;
for (const auto * node : flat_nodes)
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/" + node, -1));
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/alter_partition_version", -1));
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/columns", -1));
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/metadata", -1));
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/table_shared_id", -1));
Coordination::Responses res;
auto code = zookeeper->tryMulti(ops, res);
if (code != Coordination::Error::ZOK)
LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (table: {}). Will remove recursively.",
Coordination::errorMessage(code), zookeeper_path);
Strings children;
Coordination::Error code = zookeeper->tryGetChildren(zookeeper_path, children);
code = zookeeper->tryGetChildren(zookeeper_path, children);
if (code == Coordination::Error::ZNONODE)
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of replicated table. It's a bug");
for (const auto & child : children)
{
if (child != "dropped")
zookeeper->tryRemoveRecursive(fs::path(zookeeper_path) / child);
}
Coordination::Requests ops;
ops.clear();
Coordination::Responses responses;
ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1));
ops.emplace_back(zkutil::makeRemoveRequest(fs::path(zookeeper_path) / "dropped", -1));
@ -3322,7 +3403,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
if (!broken_part_info.contains(part->info))
continue;
/// Broken part itself ether already moved to detached or does not exist.
/// Broken part itself either already moved to detached or does not exist.
assert(broken_part_info != part->info);
part->makeCloneInDetached("covered-by-broken", getInMemoryMetadataPtr());
}
@ -4328,7 +4409,7 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalBytes(const Settings & se
void StorageReplicatedMergeTree::assertNotReadonly() const
{
if (is_readonly)
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (zookeeper path: {})", zookeeper_path);
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {})", replica_path);
}

View File

@ -214,7 +214,8 @@ public:
/** Remove a specific replica from zookeeper.
*/
static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger);
static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica,
Poco::Logger * logger, MergeTreeSettingsPtr table_settings = nullptr);
/// Removes table from ZooKeeper after the last replica was dropped
static bool removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path,

View File

@ -224,14 +224,12 @@ namespace
}
Chunk chunk;
std::lock_guard lock(reader_mutex);
if (reader->pull(chunk))
return chunk;
{
std::lock_guard lock(reader_mutex);
pipeline->reset();
reader.reset();
}
pipeline->reset();
reader.reset();
}
}

View File

@ -55,6 +55,7 @@ def get_packager_cmd(
image_version: str,
ccache_path: str,
official: bool,
cov_token: str,
) -> str:
package_type = build_config["package_type"]
comp = build_config["compiler"]
@ -87,6 +88,8 @@ def get_packager_cmd(
if official:
cmd += " --official"
if cov_token:
cmd += " --cov-token={}".format(cov_token)
return cmd
@ -203,6 +206,9 @@ def main():
build_check_name = sys.argv[1]
build_name = sys.argv[2]
cov_token = ""
if len(sys.argv) > 3:
cov_token = sys.argv[3]
build_config = get_build_config(build_check_name, build_name)
@ -296,7 +302,8 @@ def main():
version.string,
image_version,
ccache_path,
official=official_flag,
official_flag,
cov_token,
)
logging.info("Going to run packager with %s", packager_cmd)

View File

@ -120,6 +120,7 @@ if __name__ == "__main__":
os.makedirs(temp_path)
build_check_name = sys.argv[1]
reports_length = int(sys.argv[2]) if len(sys.argv) > 2 else 0
gh = Github(get_best_robot_token())
pr_info = PRInfo()
@ -138,7 +139,7 @@ if __name__ == "__main__":
logging.info("Found build report json %s", f)
build_name = get_build_name_from_file_name(f)
if build_name in reports_order:
with open(os.path.join(root, f), "r") as file_handler:
with open(os.path.join(root, f), "rb") as file_handler:
build_report = json.load(file_handler)
build_reports_map[build_name] = build_report
else:
@ -148,7 +149,8 @@ if __name__ == "__main__":
build_name,
)
some_builds_are_missing = len(build_reports_map) < len(reports_order)
reports_length = reports_length or len(reports_order)
some_builds_are_missing = len(build_reports_map) < reports_length
if some_builds_are_missing:
logging.info(
@ -188,7 +190,7 @@ if __name__ == "__main__":
branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master"
branch_name = "master"
if pr_info.number != 0:
branch_name = "PR #{}".format(pr_info.number)
branch_name = f"PR #{pr_info.number}"
branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/pull/{pr_info.number}"
commit_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commit/{pr_info.sha}"
task_url = GITHUB_RUN_URL
@ -204,8 +206,8 @@ if __name__ == "__main__":
)
report_path = os.path.join(temp_path, "report.html")
with open(report_path, "w") as f:
f.write(report)
with open(report_path, "w", encoding="utf-8") as fd:
fd.write(report)
logging.info("Going to upload prepared report")
context_name_for_path = build_check_name.lower().replace(" ", "_")
@ -235,11 +237,11 @@ if __name__ == "__main__":
addition = ""
if some_builds_are_missing:
addition = "(some builds are missing)"
addition = f"({len(build_reports_map)} < {reports_length})"
description = f"{ok_builds}/{total_builds} builds are OK {addition}"
print("::notice ::Report url: {}".format(url))
print(f"::notice ::Report url: {url}")
commit = get_commit(gh, pr_info.sha)
commit.create_status(

View File

@ -28,6 +28,16 @@ CI_CONFIG = {
"tidy": "disable",
"with_coverage": False,
},
"coverity": {
"compiler": "clang-13",
"build_type": "",
"sanitizer": "",
"package_type": "coverity",
"bundled": "bundled",
"splitted": "unsplitted",
"tidy": "disable",
"with_coverage": False,
},
"binary_gcc": {
"compiler": "gcc-11",
"build_type": "",
@ -190,6 +200,7 @@ CI_CONFIG = {
"ClickHouse build check (actions)": [
"package_release",
"performance",
"coverity",
"package_aarch64",
"package_asan",
"package_ubsan",

View File

@ -720,7 +720,7 @@ class TestCase:
pattern = '{test} > {stdout} 2> {stderr}'
if self.ext == '.sql':
pattern = "{client} --send_logs_level={logs_level} --testmode --multiquery {options} < " + pattern
pattern = "{client} --send_logs_level={logs_level} --multiquery {options} < " + pattern
command = pattern.format(**params)

View File

@ -55,7 +55,7 @@ class Client:
command = self.command[:]
if stdin is None:
command += ["--multiquery", "--testmode"]
command += ["--multiquery"]
stdin = sql
else:
command += ["--query", sql]

View File

@ -48,6 +48,8 @@ import docker
from .client import Client
from .hdfs_api import HDFSApi
from .config_cluster import *
HELPERS_DIR = p.dirname(__file__)
CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..")
LOCAL_DOCKER_COMPOSE_DIR = p.join(
@ -1664,8 +1666,8 @@ class ClickHouseCluster:
while time.time() - start < timeout:
try:
conn = pymysql.connect(
user="root",
password="clickhouse",
user=mysql_user,
password=mysql_pass,
host=self.mysql_ip,
port=self.mysql_port,
)
@ -1686,8 +1688,8 @@ class ClickHouseCluster:
while time.time() - start < timeout:
try:
conn = pymysql.connect(
user="root",
password="clickhouse",
user=mysql8_user,
password=mysql8_pass,
host=self.mysql8_ip,
port=self.mysql8_port,
)
@ -1711,8 +1713,8 @@ class ClickHouseCluster:
try:
for ip in [self.mysql2_ip, self.mysql3_ip, self.mysql4_ip]:
conn = pymysql.connect(
user="root",
password="clickhouse",
user=mysql_user,
password=mysql_pass,
host=ip,
port=self.mysql_port,
)
@ -1735,9 +1737,9 @@ class ClickHouseCluster:
self.postgres_conn = psycopg2.connect(
host=self.postgres_ip,
port=self.postgres_port,
database="postgres",
user="postgres",
password="mysecretpassword",
database=pg_db,
user=pg_user,
password=pg_pass,
)
self.postgres_conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
self.postgres_conn.autocommit = True
@ -1759,9 +1761,9 @@ class ClickHouseCluster:
self.postgres2_conn = psycopg2.connect(
host=self.postgres2_ip,
port=self.postgres_port,
database="postgres",
user="postgres",
password="mysecretpassword",
database=pg_db,
user=pg_user,
password=pg_pass,
)
self.postgres2_conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
self.postgres2_conn.autocommit = True
@ -1775,9 +1777,9 @@ class ClickHouseCluster:
self.postgres3_conn = psycopg2.connect(
host=self.postgres3_ip,
port=self.postgres_port,
database="postgres",
user="postgres",
password="mysecretpassword",
database=pg_db,
user=pg_user,
password=pg_pass,
)
self.postgres3_conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
self.postgres3_conn.autocommit = True
@ -1791,9 +1793,9 @@ class ClickHouseCluster:
self.postgres4_conn = psycopg2.connect(
host=self.postgres4_ip,
port=self.postgres_port,
database="postgres",
user="postgres",
password="mysecretpassword",
database=pg_db,
user=pg_user,
password=pg_pass,
)
self.postgres4_conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
self.postgres4_conn.autocommit = True
@ -1945,7 +1947,7 @@ class ClickHouseCluster:
def wait_mongo_to_start(self, timeout=30, secure=False):
connection_str = "mongodb://{user}:{password}@{host}:{port}".format(
host="localhost", port=self.mongo_port, user="root", password="clickhouse"
host="localhost", port=self.mongo_port, user=mongo_user, password=mongo_pass
)
if secure:
connection_str += "/?tls=true&tlsAllowInvalidCertificates=true"
@ -1969,8 +1971,8 @@ class ClickHouseCluster:
)
minio_client = Minio(
f"{self.minio_ip}:{self.minio_port}",
access_key="minio",
secret_key="minio123",
access_key=minio_access_key,
secret_key=minio_secret_key,
secure=secure,
http_client=urllib3.PoolManager(cert_reqs="CERT_NONE"),
) # disable SSL check as we test ClickHouse and not Python library
@ -3488,16 +3490,16 @@ class ClickHouseInstance:
"MySQL": {
"DSN": "mysql_odbc",
"Driver": "/usr/lib/x86_64-linux-gnu/odbc/libmyodbc.so",
"Database": "clickhouse",
"Uid": "root",
"Pwd": "clickhouse",
"Database": odbc_mysql_db,
"Uid": odbc_mysql_uid,
"Pwd": odbc_mysql_pass,
"Server": self.cluster.mysql_host,
},
"PostgreSQL": {
"DSN": "postgresql_odbc",
"Database": "postgres",
"UserName": "postgres",
"Password": "mysecretpassword",
"Database": odbc_psql_db,
"UserName": odbc_psql_user,
"Password": odbc_psql_pass,
"Port": str(self.cluster.postgres_port),
"Servername": self.cluster.postgres_host,
"Protocol": "9.3",

View File

@ -0,0 +1,31 @@
# MYSQL CREDENTIALS
mysql_user = "root"
mysql_pass = "clickhouse"
# MYSQL8 CREDENTIALS
mysql8_user = "root"
mysql8_pass = "clickhouse"
# POSTGRES CREDENTIALS
pg_user = "postgres"
pg_pass = "mysecretpassword"
pg_db = "postgres"
# MINIO CREDENTIALS
minio_access_key = "minio"
minio_secret_key = "minio123"
# MONGODB CREDENTIALS
mongo_user = "root"
mongo_pass = "clickhouse"
# ODBC CREDENTIALS
odbc_mysql_uid = "root"
odbc_mysql_pass = "clickhouse"
odbc_mysql_db = "clickhouse"
odbc_psql_db = "postgres"
odbc_psql_user = "postgres"
odbc_psql_pass = "mysecretpassword"

View File

@ -0,0 +1,4 @@
<test>
<query>SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number LIMIT 5</query>
<query>SELECT sipHash64(number) FROM numbers(1e8) ORDER BY number + 1 LIMIT 5</query>
</test>

View File

@ -0,0 +1,53 @@
<test>
<substitutions>
<substitution>
<name>array_type</name>
<values>
<value>Int8</value>
<value>Int16</value>
<value>Int32</value>
<value>Int64</value>
</values>
</substitution>
</substitutions>
<create_query>
CREATE TABLE test_table_small_{array_type}
(
`set` Array({array_type}),
`subset` Array ({array_type})
)
ENGINE = MergeTree ORDER BY set;
</create_query>
<create_query>
CREATE TABLE test_table_medium_{array_type}
(
`set` Array({array_type}),
`subset` Array ({array_type})
)
ENGINE = MergeTree ORDER BY set;
</create_query>
<create_query>
CREATE TABLE test_table_large_{array_type}
(
`set` Array({array_type}),
`subset` Array ({array_type})
)
ENGINE = MergeTree ORDER BY set;
</create_query>
<fill_query>INSERT INTO test_table_small_{array_type} SELECT groupArraySample(5000)(rand64()) AS set, groupArraySample(500)(rand64()) AS subset FROM numbers(10000000) GROUP BY number % 5000;</fill_query>
<fill_query>INSERT INTO test_table_medium_{array_type} SELECT groupArraySample(50000)(rand64()) AS set, groupArraySample(5000)(rand64()) AS subset FROM numbers(25000000) GROUP BY number % 50000;</fill_query>
<fill_query>INSERT INTO test_table_large_{array_type} SELECT groupArraySample(500000)(rand64()) AS set, groupArraySample(500000)(rand64()) AS subset FROM numbers(50000000) GROUP BY number % 500000;</fill_query>
<query>SELECT hasAll(set, subset) FROM test_table_small_{array_type} FORMAT Null</query>
<query>SELECT hasAll(set, subset) FROM test_table_medium_{array_type} FORMAT Null</query>
<query>SELECT hasAll(set, subset) FROM test_table_large_{array_type} FORMAT Null</query>
<drop_query>DROP TABLE IF EXISTS test_table_small_{array_type}</drop_query>
<drop_query>DROP TABLE IF EXISTS test_table_medium_{array_type}</drop_query>
<drop_query>DROP TABLE IF EXISTS test_table_large_{array_type}</drop_query>
</test>

View File

@ -43,7 +43,7 @@ $CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_no_length_delimiter_protobuf
rm "$BINARY_FILE_PATH"
# The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter.
$CLICKHOUSE_CLIENT --multiquery --testmode > /dev/null <<EOF
$CLICKHOUSE_CLIENT --multiquery > /dev/null <<EOF
SELECT * FROM no_length_delimiter_protobuf_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_no_length_delimiter:Message'; -- { clientError 546 }
EOF

View File

@ -13,5 +13,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# ${CURDIR}/00921_datetime64_compatibility.python
python3 "${CURDIR}"/00921_datetime64_compatibility_long.python \
| ${CLICKHOUSE_CLIENT} --ignore-error -T -nm --calculate_text_stack_trace 0 --log-level 'error' 2>&1 \
| ${CLICKHOUSE_CLIENT} --ignore-error -nm --calculate_text_stack_trace 0 --log-level 'error' 2>&1 \
| grep -v -e 'Received exception .*$' -e '^(query: ' | sed 's/^\(Code: [0-9]\+\).*$/\1/g'

View File

@ -88,9 +88,9 @@ from numbers(100000); -- { serverError 241; }" > /dev/null 2>&1
# fails
echo "Should throw 1"
execute_insert --testmode
execute_insert
echo "Should throw 2"
execute_insert --testmode --min_insert_block_size_rows=1 --min_insert_block_size_rows_for_materialized_views=$((1<<20))
execute_insert --min_insert_block_size_rows=1 --min_insert_block_size_rows_for_materialized_views=$((1<<20))
# passes
echo "Should pass 1"

View File

@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT -n --query="
LIFETIME(MIN 1000 MAX 2000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d'));"
$CLICKHOUSE_CLIENT --testmode -nq "SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }"
$CLICKHOUSE_CLIENT -nq "SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }"
$CLICKHOUSE_CLIENT -n --query="
SELECT 'TEST_SMALL';
@ -65,7 +65,7 @@ $CLICKHOUSE_CLIENT -n --query="
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('10', toInt32(-20)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('10', toInt32(-20)));"
$CLICKHOUSE_CLIENT --testmode -nq "SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }"
$CLICKHOUSE_CLIENT -nq "SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }"
$CLICKHOUSE_CLIENT -n --query="DROP DICTIONARY 01280_db.ssd_dict;
DROP TABLE IF EXISTS 01280_db.keys_table;

View File

@ -35,10 +35,11 @@ Expression (Projection)
ReadFromMergeTree (default.test_table)
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree (default.test_table)
Expression (Before ORDER BY [lifted up part])
Sorting
Expression (Before ORDER BY)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromMergeTree (default.test_table)
optimize_aggregation_in_order
Expression ((Projection + Before ORDER BY))
Aggregating

View File

@ -925,10 +925,11 @@ Expression ((Projection + Before ORDER BY))
Window (Window step for window \'ORDER BY o ASC, number ASC\')
Sorting (Sorting for window \'ORDER BY o ASC, number ASC\')
Window (Window step for window \'ORDER BY number ASC\')
Sorting (Sorting for window \'ORDER BY number ASC\')
Expression ((Before window functions + (Projection + Before ORDER BY)))
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemNumbers)
Expression ((Before window functions + (Projection + Before ORDER BY)) [lifted up part])
Sorting (Sorting for window \'ORDER BY number ASC\')
Expression ((Before window functions + (Projection + Before ORDER BY)))
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemNumbers)
-- A test case for the sort comparator found by fuzzer.
SELECT
max(number) OVER (ORDER BY number DESC NULLS FIRST),

View File

@ -10,8 +10,8 @@ set max_block_size=40960;
-- MergeSortingTransform: Re-merging intermediate ORDER BY data (20 blocks with 819200 rows) to save memory consumption
-- MergeSortingTransform: Memory usage is lowered from 186.25 MiB to 95.00 MiB
-- MergeSortingTransform: Re-merging is not useful (memory usage was not lowered by remerge_sort_lowered_memory_bytes_ratio=2.0)
select number k, repeat(toString(number), 11) v1, repeat(toString(number), 12) v2 from numbers(3e6) order by k limit 400e3 format Null; -- { serverError 241 }
select number k, repeat(toString(number), 11) v1, repeat(toString(number), 12) v2 from numbers(3e6) order by k limit 400e3 settings remerge_sort_lowered_memory_bytes_ratio=2. format Null; -- { serverError 241 }
select number k, repeat(toString(number), 11) v1, repeat(toString(number), 12) v2 from numbers(3e6) order by v1, v2 limit 400e3 format Null; -- { serverError 241 }
select number k, repeat(toString(number), 11) v1, repeat(toString(number), 12) v2 from numbers(3e6) order by v1, v2 limit 400e3 settings remerge_sort_lowered_memory_bytes_ratio=2. format Null; -- { serverError 241 }
-- remerge_sort_lowered_memory_bytes_ratio 1.9 is good (need at least 1.91/0.98=1.94)
-- MergeSortingTransform: Re-merging intermediate ORDER BY data (20 blocks with 819200 rows) to save memory consumption

View File

@ -142,3 +142,12 @@ Filter
Filter
2 3
2 3
> function calculation should be done after sorting and limit (if possible)
> Expression should be divided into two subexpressions and only one of them should be moved after Sorting
Expression (Before ORDER BY [lifted up part])
FUNCTION sipHash64
Sorting
Expression (Before ORDER BY)
FUNCTION plus
> this query should be executed without throwing an exception
0

View File

@ -196,3 +196,12 @@ $CLICKHOUSE_CLIENT -q "
select a, b from (
select number + 1 as a, number + 2 as b from numbers(2) union all select number + 1 as b, number + 2 as a from numbers(2)
) where a != 1 settings enable_optimize_predicate_expression = 0"
echo "> function calculation should be done after sorting and limit (if possible)"
echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting"
$CLICKHOUSE_CLIENT -q "
explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" |
sed 's/^ *//g' | grep -o "^ *\(Expression (Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)"
echo "> this query should be executed without throwing an exception"
$CLICKHOUSE_CLIENT -q "
select throwIf(number = 5) from (select * from numbers(10)) order by number limit 1"

File diff suppressed because one or more lines are too long

View File

@ -9,6 +9,7 @@
1 9999
3
2021-10-25 10:00:00 2021-10-27 10:00:00 3
2021-10-25 10:00:00 2021-10-27 10:00:00 3
1
1
1
@ -17,3 +18,5 @@
0
2021-10-24 10:00:00
0
1000
1000

View File

@ -50,6 +50,8 @@ drop table if exists d;
create table d (dt DateTime, j int) engine MergeTree partition by (toDate(dt), ceiling(j), toDate(dt), CEILING(j)) order by tuple();
insert into d values ('2021-10-24 10:00:00', 10), ('2021-10-25 10:00:00', 10), ('2021-10-26 10:00:00', 10), ('2021-10-27 10:00:00', 10);
select min(dt), max(dt), count() from d where toDate(dt) >= '2021-10-25';
-- fuzz crash
select min(dt), max(dt), count(toDate(dt) >= '2021-10-25') from d where toDate(dt) >= '2021-10-25';
select count() from d group by toDate(dt);
-- fuzz crash
@ -59,3 +61,15 @@ SELECT min(dt) FROM d PREWHERE ((0.9998999834060669 AND 1023) AND 255) <= ceil(j
SELECT count('') AND NULL FROM d PREWHERE ceil(j) <= NULL;
drop table d;
-- count variant optimization
drop table if exists test;
create table test (id Int64, d Int64, projection dummy(select * order by id)) engine MergeTree order by id;
insert into test select number, number from numbers(1e3);
select count(if(d=4, d, 1)) from test settings force_optimize_projection = 1;
select count(d/3) from test settings force_optimize_projection = 1;
select count(if(d=4, Null, 1)) from test settings force_optimize_projection = 1; -- { serverError 584 }
drop table test;

View File

@ -0,0 +1 @@
Tuple(foo Int8, k1 Int8, k2 Int8)

View File

@ -0,0 +1,16 @@
-- Tags: no-fasttest
DROP TABLE IF EXISTS t_json;
SET allow_experimental_object_type = 1;
CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id;
INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}};
INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}};
OPTIMIZE TABLE t_json FINAL;
SELECT any(toTypeName(obj)) from t_json;
DROP TABLE IF EXISTS t_json;

View File

@ -1,4 +1,4 @@
-- Tags: long
-- Tags: long, no-backward-compatibility-check:22.3.2.1
DROP TABLE IF EXISTS t_json_parallel;
SET allow_experimental_object_type = 1, max_insert_threads = 20, max_threads = 20;

View File

@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --testmode -n -q 'select 1 -- { clientError FOOBAR }' |& grep -o 'No error code with name:.*'
$CLICKHOUSE_CLIENT -n -q 'select 1 -- { clientError FOOBAR }' |& grep -o 'No error code with name:.*'

Some files were not shown because too many files have changed in this diff Show More