Compilable code

This commit is contained in:
alesapin 2020-05-25 20:22:20 +03:00
commit c3a6571036
352 changed files with 4219 additions and 2622 deletions

View File

@ -9,7 +9,7 @@ Changelog category (leave one):
- Build/Testing/Packaging Improvement
- Documentation (changelog entry is not required)
- Other
- Non-significant (changelog entry is not required)
- Not for changelog (changelog entry is not required)
Changelog entry (a user-readable short description of the changes that goes to CHANGELOG.md):

3
.gitmodules vendored
View File

@ -157,3 +157,6 @@
[submodule "contrib/openldap"]
path = contrib/openldap
url = https://github.com/openldap/openldap.git
[submodule "contrib/fmtlib"]
path = contrib/fmtlib
url = https://github.com/fmtlib/fmt.git

View File

@ -79,6 +79,7 @@ target_link_libraries (common
Poco::Util
Poco::Foundation
replxx
fmt
PRIVATE
cctz

View File

@ -2,16 +2,14 @@
/// Macros for convenient usage of Poco logger.
#include <sstream>
#include <fmt/format.h>
#include <fmt/ostream.h>
#include <Poco/Logger.h>
#include <Poco/Message.h>
#include <Poco/Version.h>
#include <Common/CurrentThread.h>
#ifndef QUERY_PREVIEW_LENGTH
#define QUERY_PREVIEW_LENGTH 160
#endif
/// TODO Remove this.
using Poco::Logger;
using Poco::Message;
using DB::LogsLevel;
@ -19,21 +17,20 @@ using DB::CurrentThread;
/// Logs a message to a specified logger with that level.
#define LOG_SIMPLE(logger, message, priority, PRIORITY) do \
#define LOG_IMPL(logger, priority, PRIORITY, ...) do \
{ \
const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \
(CurrentThread::getGroup()->client_logs_level >= (priority)); \
if ((logger)->is((PRIORITY)) || is_clients_log) \
{ \
std::stringstream oss_internal_rare; \
oss_internal_rare << message; \
std::string formatted_message = fmt::format(__VA_ARGS__); \
if (auto channel = (logger)->getChannel()) \
{ \
std::string file_function; \
file_function += __FILE__; \
file_function += "; "; \
file_function += __PRETTY_FUNCTION__; \
Message poco_message((logger)->name(), oss_internal_rare.str(), \
Message poco_message((logger)->name(), formatted_message, \
(PRIORITY), file_function.c_str(), __LINE__); \
channel->log(poco_message); \
} \
@ -41,10 +38,9 @@ using DB::CurrentThread;
} while (false)
#define LOG_TRACE(logger, message) LOG_SIMPLE(logger, message, LogsLevel::trace, Message::PRIO_TRACE)
#define LOG_DEBUG(logger, message) LOG_SIMPLE(logger, message, LogsLevel::debug, Message::PRIO_DEBUG)
#define LOG_INFO(logger, message) LOG_SIMPLE(logger, message, LogsLevel::information, Message::PRIO_INFORMATION)
#define LOG_WARNING(logger, message) LOG_SIMPLE(logger, message, LogsLevel::warning, Message::PRIO_WARNING)
#define LOG_ERROR(logger, message) LOG_SIMPLE(logger, message, LogsLevel::error, Message::PRIO_ERROR)
#define LOG_FATAL(logger, message) LOG_SIMPLE(logger, message, LogsLevel::error, Message::PRIO_FATAL)
#define LOG_TRACE(logger, ...) LOG_IMPL(logger, LogsLevel::trace, Message::PRIO_TRACE, __VA_ARGS__)
#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, LogsLevel::debug, Message::PRIO_DEBUG, __VA_ARGS__)
#define LOG_INFO(logger, ...) LOG_IMPL(logger, LogsLevel::information, Message::PRIO_INFORMATION, __VA_ARGS__)
#define LOG_WARNING(logger, ...) LOG_IMPL(logger, LogsLevel::warning, Message::PRIO_WARNING, __VA_ARGS__)
#define LOG_ERROR(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_ERROR, __VA_ARGS__)
#define LOG_FATAL(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_FATAL, __VA_ARGS__)

View File

@ -24,6 +24,7 @@ PEERDIR(
contrib/libs/cxxsupp/libcxx-filesystem
contrib/libs/poco/Net
contrib/libs/poco/Util
contrib/libs/fmt
contrib/restricted/boost
contrib/restricted/cityhash-1.0.2
)

View File

@ -180,7 +180,7 @@ public:
// levels and more info, but for completeness we log all signals
// here at trace level.
// Don't use strsignal here, because it's not thread-safe.
LOG_TRACE(log, "Received signal " << sig);
LOG_TRACE(log, "Received signal {}", sig);
if (sig == Signals::StopThread)
{
@ -236,7 +236,7 @@ private:
void onTerminate(const std::string & message, UInt32 thread_num) const
{
LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message);
LOG_FATAL(log, "(version {}{}) (from thread {}) {}", VERSION_STRING, VERSION_OFFICIAL, thread_num, message);
}
void onFault(
@ -257,9 +257,9 @@ private:
message << " (no query)";
else
message << " (query_id: " << query_id << ")";
message << " Received signal " << strsignal(sig) << " (" << sig << ")" << ".";
message << " Received signal " << strsignal(sig) << " (" << sig << ").";
LOG_FATAL(log, message.rdbuf());
LOG_FATAL(log, message.str());
}
LOG_FATAL(log, signalToErrorMessage(sig, info, context));
@ -274,7 +274,7 @@ private:
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
bare_stacktrace << ' ' << stack_trace.getFrames()[i];
LOG_FATAL(log, bare_stacktrace.rdbuf());
LOG_FATAL(log, bare_stacktrace.str());
}
/// Write symbolized stack trace line by line for better grep-ability.
@ -302,7 +302,7 @@ static void sanitizerDeathCallback()
message << " (query_id: " << query_id << ")";
message << " Sanitizer trap.";
LOG_FATAL(log, message.rdbuf());
LOG_FATAL(log, message.str());
}
/// Just in case print our own stack trace. In case when llvm-symbolizer does not work.
@ -314,7 +314,7 @@ static void sanitizerDeathCallback()
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
bare_stacktrace << ' ' << stack_trace.getFrames()[i];
LOG_FATAL(log, bare_stacktrace.rdbuf());
LOG_FATAL(log, bare_stacktrace.str());
}
/// Write symbolized stack trace line by line for better grep-ability.
@ -379,7 +379,7 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path
}
catch (...)
{
LOG_WARNING(logger, __PRETTY_FUNCTION__ << ": when creating " << path << ", " << DB::getCurrentExceptionMessage(true));
LOG_WARNING(logger, "{}: when creating {}, {}", __PRETTY_FUNCTION__, path, DB::getCurrentExceptionMessage(true));
}
return false;
}
@ -498,11 +498,10 @@ void debugIncreaseOOMScore()
}
catch (const Poco::Exception & e)
{
LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '" +
e.displayText() + "'.");
LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText());
return;
}
LOG_INFO(&Logger::root(), "Set OOM score adjustment to " + new_score);
LOG_INFO(&Logger::root(), "Set OOM score adjustment to {}", new_score);
}
#else
void debugIncreaseOOMScore() {}
@ -734,7 +733,7 @@ void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn)
task_failed = true;
Poco::AutoPtr<Poco::TaskFailedNotification> fn(_tfn);
Logger *lg = &(logger());
LOG_ERROR(lg, "Task '" << fn->task()->name() << "' failed. Daemon is shutting down. Reason - " << fn->reason().displayText());
LOG_ERROR(lg, "Task '{}' failed. Daemon is shutting down. Reason - {}", fn->task()->name(), fn->reason().displayText());
ServerApplication::terminate();
}
@ -850,7 +849,7 @@ void BaseDaemon::handleSignal(int signal_id)
void BaseDaemon::onInterruptSignals(int signal_id)
{
is_cancelled = true;
LOG_INFO(&logger(), "Received termination signal (" << strsignal(signal_id) << ")");
LOG_INFO(&logger(), "Received termination signal ({})", strsignal(signal_id));
if (sigint_signals_counter >= 2)
{

View File

@ -52,8 +52,7 @@ private:
}
catch (const Poco::Exception & e)
{
LOG_WARNING(&Poco::Util::Application::instance().logger(),
"Fail to write to Graphite " << host << ":" << port << ". e.what() = " << e.what() << ", e.message() = " << e.message());
LOG_WARNING(&Poco::Util::Application::instance().logger(), "Fail to write to Graphite {}:{}. e.what() = {}, e.message() = {}", host, port, e.what(), e.message());
}
}

View File

@ -4,6 +4,7 @@
#include <ctime>
#include <string>
#include <iomanip>
#include <sstream>
namespace ext

View File

@ -162,4 +162,10 @@ elseif (COMPILER_GCC)
add_cxx_compile_options(-Wunused)
# Warn if vector operation is not implemented via SIMD capabilities of the architecture
add_cxx_compile_options(-Wvector-operation-performance)
# XXX: gcc10 stuck with this option while compiling GatherUtils code
# (anyway there are builds with clang, that will warn)
if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 10)
add_cxx_compile_options(-Wno-sequence-point)
endif()
endif ()

View File

@ -317,3 +317,5 @@ endif()
if (USE_FASTOPS)
add_subdirectory (fastops-cmake)
endif()
add_subdirectory (fmtlib-cmake)

1
contrib/fmtlib vendored Submodule

@ -0,0 +1 @@
Subproject commit 297c3b2ed551a4989826fc8c4780bf533e964bd9

View File

@ -0,0 +1,20 @@
set (SRCS
../fmtlib/src/format.cc
../fmtlib/src/os.cc
../fmtlib/include/fmt/chrono.h
../fmtlib/include/fmt/color.h
../fmtlib/include/fmt/compile.h
../fmtlib/include/fmt/core.h
../fmtlib/include/fmt/format.h
../fmtlib/include/fmt/format-inl.h
../fmtlib/include/fmt/locale.h
../fmtlib/include/fmt/os.h
../fmtlib/include/fmt/ostream.h
../fmtlib/include/fmt/posix.h
../fmtlib/include/fmt/printf.h
../fmtlib/include/fmt/ranges.h
)
add_library(fmt ${SRCS})
target_include_directories(fmt SYSTEM PUBLIC ../fmtlib/include)

View File

@ -9,6 +9,17 @@ if (ENABLE_JEMALLOC)
option (USE_INTERNAL_JEMALLOC "Use internal jemalloc library" ${NOT_UNBUNDLED})
if (USE_INTERNAL_JEMALLOC)
# ThreadPool select job randomly, and there can be some threads that had been
# performed some memory heavy task before and will be inactive for some time,
# but until it will became active again, the memory will not be freed since by
# default each thread has it's own arena, but there should be not more then
# 4*CPU arenas (see opt.nareans description).
#
# By enabling percpu_arena number of arenas limited to number of CPUs and hence
# this problem should go away.
set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu" CACHE STRING "Change default configuration string of JEMalloc" )
message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}")
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc")
set (SRCS
@ -52,11 +63,19 @@ if (ENABLE_JEMALLOC)
add_library(jemalloc ${SRCS})
target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include)
target_include_directories(jemalloc SYSTEM PUBLIC include)
set(JEMALLOC_INCLUDE)
if (ARCH_AMD64)
target_include_directories(jemalloc SYSTEM PUBLIC include_linux_x86_64)
set(JEMALLOC_INCLUDE_PREFIX include_linux_x86_64)
elseif (ARCH_ARM)
target_include_directories(jemalloc SYSTEM PUBLIC include_linux_aarch64)
set(JEMALLOC_INCLUDE_PREFIX include_linux_aarch64)
endif ()
target_include_directories(jemalloc SYSTEM PUBLIC
${JEMALLOC_INCLUDE_PREFIX})
configure_file(${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h.in
${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h)
target_include_directories(jemalloc SYSTEM PRIVATE
${CMAKE_CURRENT_BINARY_DIR}/${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal)
target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE)

View File

@ -5,3 +5,4 @@ Added #define GNU_SOURCE
Added JEMALLOC_OVERRIDE___POSIX_MEMALIGN because why not.
Removed JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF because it's non standard.
Removed JEMALLOC_PURGE_MADVISE_FREE because it's available only from Linux 4.5.
Added JEMALLOC_CONFIG_MALLOC_CONF substitution

View File

@ -369,7 +369,7 @@
/* #undef JEMALLOC_EXPORT */
/* config.malloc_conf options string. */
#define JEMALLOC_CONFIG_MALLOC_CONF ""
#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@"
/* If defined, jemalloc takes the malloc/free/etc. symbol names. */
#define JEMALLOC_IS_MALLOC 1

View File

@ -5,3 +5,4 @@ Added #define GNU_SOURCE
Added JEMALLOC_OVERRIDE___POSIX_MEMALIGN because why not.
Removed JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF because it's non standard.
Removed JEMALLOC_PURGE_MADVISE_FREE because it's available only from Linux 4.5.
Added JEMALLOC_CONFIG_MALLOC_CONF substitution

View File

@ -360,7 +360,7 @@
/* #undef JEMALLOC_EXPORT */
/* config.malloc_conf options string. */
#define JEMALLOC_CONFIG_MALLOC_CONF ""
#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@"
/* If defined, jemalloc takes the malloc/free/etc. symbol names. */
#define JEMALLOC_IS_MALLOC 1

11
debian/rules vendored
View File

@ -24,6 +24,10 @@ DEB_BUILD_OPTIONS+=parallel=$(THREADS_COUNT)
ifndef ENABLE_TESTS
CMAKE_FLAGS += -DENABLE_TESTS=0
else
# To export binaries and from deb build we do not strip them. No need to run tests in deb build as we run them in CI
DEB_BUILD_OPTIONS+= nocheck
DEB_BUILD_OPTIONS+= nostrip
endif
ifndef MAKE_TARGET
@ -88,14 +92,19 @@ override_dh_auto_build:
$(MAKE) $(THREADS_COUNT) -C $(BUILDDIR) $(MAKE_TARGET)
override_dh_auto_test:
ifeq (,$(filter nocheck,$(DEB_BUILD_OPTIONS)))
cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V -E with_server
endif
override_dh_clean:
rm -rf debian/copyright debian/clickhouse-client.docs debian/clickhouse-common-static.docs
dh_clean -X contrib
dh_clean # -X contrib
override_dh_strip:
#https://www.debian.org/doc/debian-policy/ch-source.html#debian-rules-and-deb-build-options
ifeq (,$(filter nostrip,$(DEB_BUILD_OPTIONS)))
dh_strip -pclickhouse-common-static --dbg-package=clickhouse-common-static-dbg
endif
override_dh_install:
# Making docs

View File

@ -10,5 +10,16 @@ mv *.changes /output
mv *.buildinfo /output
mv /*.rpm /output ||: # if exists
mv /*.tgz /output ||: # if exists
if [ -n "$BINARY_OUTPUT" ] && { [ "$BINARY_OUTPUT" = "programs" ] || [ "$BINARY_OUTPUT" = "tests" ] ;}
then
echo Place $BINARY_OUTPUT to output
mkdir /output/binary ||: # if exists
mv /build/obj-*/programs/clickhouse* /output/binary
if [ "$BINARY_OUTPUT" = "tests" ]
then
mv /build/obj-*/src/unit_tests_dbms /output/binary
fi
fi
ccache --show-stats ||:
ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||:

View File

@ -54,7 +54,7 @@ def run_docker_image_with_env(image_name, output, env_variables, ch_root, ccache
subprocess.check_call(cmd, shell=True)
def parse_env_variables(build_type, compiler, sanitizer, package_type, image_type, cache, distcc_hosts, unbundled, split_binary, clang_tidy, version, author, official, alien_pkgs, with_coverage):
def parse_env_variables(build_type, compiler, sanitizer, package_type, image_type, cache, distcc_hosts, unbundled, split_binary, clang_tidy, version, author, official, alien_pkgs, with_coverage, with_binaries):
CLANG_PREFIX = "clang"
DARWIN_SUFFIX = "-darwin"
ARM_SUFFIX = "-aarch64"
@ -131,6 +131,14 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ
if alien_pkgs:
result.append("ALIEN_PKGS='" + ' '.join(['--' + pkg for pkg in alien_pkgs]) + "'")
if with_binaries == "programs":
result.append('BINARY_OUTPUT=programs')
elif with_binaries == "tests":
result.append('ENABLE_TESTS=1')
result.append('BINARY_OUTPUT=tests')
cmake_flags.append('-DENABLE_TESTS=1')
cmake_flags.append('-DUSE_GTEST=1')
if unbundled:
# TODO: fix build with ENABLE_RDKAFKA
cmake_flags.append('-DUNBUNDLED=1 -DENABLE_MYSQL=0 -DENABLE_ODBC=0 -DENABLE_REPLXX=0 -DENABLE_RDKAFKA=0')
@ -179,6 +187,7 @@ if __name__ == "__main__":
parser.add_argument("--official", action="store_true")
parser.add_argument("--alien-pkgs", nargs='+', default=[])
parser.add_argument("--with-coverage", action="store_true")
parser.add_argument("--with-binaries", choices=("programs", "tests", ""), default="")
args = parser.parse_args()
if not os.path.isabs(args.output_dir):
@ -195,6 +204,12 @@ if __name__ == "__main__":
if args.alien_pkgs and not image_type == "deb":
raise Exception("Can add alien packages only in deb build")
if args.with_binaries != "" and not image_type == "deb":
raise Exception("Can add additional binaries only in deb build")
if args.with_binaries != "" and image_type == "deb":
logging.info("Should place {} to output".format(args.with_binaries))
dockerfile = os.path.join(ch_root, "docker/packager", image_type, "Dockerfile")
if image_type != "freebsd" and not check_image_exists_locally(image_name) or args.force_build_image:
if not pull_image(image_name) or args.force_build_image:
@ -202,6 +217,6 @@ if __name__ == "__main__":
env_prepared = parse_env_variables(
args.build_type, args.compiler, args.sanitizer, args.package_type, image_type,
args.cache, args.distcc_hosts, args.unbundled, args.split_binary, args.clang_tidy,
args.version, args.author, args.official, args.alien_pkgs, args.with_coverage)
args.version, args.author, args.official, args.alien_pkgs, args.with_coverage, args.with_binaries)
run_docker_image_with_env(image_name, args.output_dir, env_prepared, ch_root, args.ccache_dir)
logging.info("Output placed into {}".format(args.output_dir))

View File

@ -8,3 +8,4 @@ services:
MONGO_INITDB_ROOT_PASSWORD: clickhouse
ports:
- 27018:27017
command: --profile=2 --verbose

View File

@ -76,6 +76,14 @@ directory.
More complex setup is possible, but inconvenient and requires some scripting.
See `manual-run.sh` for inspiration.
#### Compare two published releases
Use `compare-releases.sh`. It will download and extract static + dbg + test
packages for both releases, and then call the main comparison script
`compare.sh`, starting from `configure` stage.
```
compare-releaseses.sh 19.16.19.85 20.4.2.9
```
#### Statistical considerations
Generating randomization distribution for medians is tricky. Suppose we have N

View File

@ -0,0 +1,82 @@
#!/bin/bash
set -ex
set -o pipefail
trap "exit" INT TERM
trap 'kill $(jobs -pr) ||:' EXIT
left_version=${1}
right_version=${2}
if [ "$left_version" == "" ] || [ "$right_version" == "" ]
then
>&2 echo Usage: $(basename "$0") left_version right_version
exit 1
fi
script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
repo_dir=${repo_dir:-$(readlink -f "$script_dir/../../..")}
function download_package() # (version, path)
{
version="$1"
path="$2"
cd "$path"
wget -nv -nd -nc "https://repo.clickhouse.tech/deb/stable/main/clickhouse-common-static-dbg_${version}_amd64.deb" ||:
wget -nv -nd -nc "https://repo.clickhouse.tech/deb/stable/main/clickhouse-common-static_${version}_amd64.deb" ||:
wget -nv -nd -nc "https://repo.clickhouse.tech/deb/stable/main/clickhouse-test_${version}_all.deb" ||:
mkdir tmp ||:
for x in *.deb; do dpkg-deb -x "$x" tmp ; done
mv tmp/usr/bin/clickhouse ./clickhouse
mkdir .debug
mv tmp/usr/lib/debug/usr/bin/clickhouse .debug/clickhouse
mv tmp/usr/share/clickhouse-test/performance .
ln -s clickhouse clickhouse-local
ln -s clickhouse clickhouse-client
ln -s clickhouse clickhouse-server
rm -rf tmp
}
function download
{
rm -r left right db0 ||:
mkdir left right db0 ||:
"$script_dir/download.sh" ||: &
download_package "$left_version" left &
download_package "$right_version" right &
wait
rm -rf {right,left}/tmp
}
function configure
{
# Configs
cp -av "$script_dir/config" right
cp -av "$script_dir/config" left
cp -av "$repo_dir"/programs/server/config* right/config
cp -av "$repo_dir"/programs/server/user* right/config
cp -av "$repo_dir"/programs/server/config* left/config
cp -av "$repo_dir"/programs/server/user* left/config
}
function run
{
left/clickhouse-local --query "select * from system.build_options format PrettySpace" | sed 's/ *$//' | fold -w 80 -s > left-commit.txt
right/clickhouse-local --query "select * from system.build_options format PrettySpace" | sed 's/ *$//' | fold -w 80 -s > right-commit.txt
PATH=right:"$PATH" \
CHPC_TEST_PATH=right/performance \
stage=configure \
"$script_dir/compare.sh" &> >(tee compare.log)
}
download
configure
run
rm output.7z
7z a output.7z ./*.{log,tsv,html,txt,rep,svg} {right,left}/{performance,db/preprocessed_configs}

View File

@ -155,9 +155,6 @@ function run_tests
TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n")
# the grep is to filter out set -x output and keep only time output
{ time "$script_dir/perf.py" --host localhost localhost --port 9001 9002 -- "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; } 2>&1 >/dev/null | grep -v ^+ >> "wall-clock-times.tsv" || continue
# The test completed with zero status, so we treat stderr as warnings
mv "$test_name-err.log" "$test_name-warn.log"
done
unset TIMEFORMAT
@ -217,13 +214,8 @@ function get_profiles
clickhouse-client --port 9002 --query "select 1"
}
# Build and analyze randomization distribution for all queries.
function analyze_queries
function build_log_column_definitions
{
rm -v analyze-commands.txt analyze-errors.log all-queries.tsv unstable-queries.tsv ./*-report.tsv raw-queries.tsv ||:
rm -rfv analyze ||:
mkdir analyze ||:
# FIXME This loop builds column definitons from TSVWithNamesAndTypes in an
# absolutely atrocious way. This should be done by the file() function itself.
for x in {right,left}-{addresses,{query,query-thread,trace,metric}-log}.tsv
@ -233,6 +225,16 @@ do
<(sed -n '2{s/\t/\n/g;p;q}' "$x" ) \
| tr '\n' ', ' | sed 's/,$//' > "$x.columns"
done
}
# Build and analyze randomization distribution for all queries.
function analyze_queries
{
rm -v analyze-commands.txt analyze-errors.log all-queries.tsv unstable-queries.tsv ./*-report.tsv raw-queries.tsv ||:
rm -rf analyze ||:
mkdir analyze analyze/tmp ||:
build_log_column_definitions
# Split the raw test output into files suitable for analysis.
IFS=$'\n'
@ -272,12 +274,13 @@ create table query_metrics engine File(TSV, -- do not add header -- will parse w
, query_runs.time
] metrics
from (
select *, 0 version from left_query_log
select query_duration_ms, memory_usage, query_id, 0 version from left_query_log
union all
select *, 1 version from right_query_log
select query_duration_ms, memory_usage, query_id, 1 version from right_query_log
) query_logs
right join query_runs
using (query_id, version)
order by test, query_index
;
"
@ -291,8 +294,8 @@ query_index=1
IFS=$'\n'
for prefix in $(cut -f1,2 "analyze/query-run-metrics.tsv" | sort | uniq)
do
file="analyze/q$query_index.tmp"
grep -F "$prefix " "analyze/query-run-metrics.tsv" > "$file" &
file="analyze/tmp/$(echo "$prefix" | sed 's/\t/_/g').tsv"
grep "^$prefix " "analyze/query-run-metrics.tsv" > "$file" &
printf "%s\0\n" \
"clickhouse-local \
--file \"$file\" \
@ -301,23 +304,23 @@ do
>> \"analyze/query-reports.tsv\"" \
2>> analyze/errors.log \
>> analyze/commands.txt
query_index=$((query_index + 1))
done
wait
unset IFS
parallel --joblog analyze/parallel-log.txt --null < analyze/commands.txt
parallel --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log
}
# Analyze results
function report
{
rm -r report ||:
mkdir report ||:
mkdir report report/tmp ||:
rm ./*.{rep,svg} test-times.tsv test-dump.tsv unstable.tsv unstable-query-ids.tsv unstable-query-metrics.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv slow-on-client.tsv all-queries.tsv ||:
build_log_column_definitions
cat analyze/errors.log >> report/errors.log ||:
cat profile-errors.log >> report/errors.log ||:
@ -329,7 +332,8 @@ create view query_display_names as select * from
create table query_metric_stats engine File(TSVWithNamesAndTypes,
'report/query-metric-stats.tsv') as
select *, metric_name
select metric_name, left, right, diff, stat_threshold, test, query_index,
query_display_name
from file ('analyze/query-reports.tsv', TSV, 'left Array(float),
right Array(float), diff Array(float), stat_threshold Array(float),
test text, query_index int') reports
@ -375,7 +379,8 @@ create table queries_old_format engine File(TSVWithNamesAndTypes, 'queries.rep')
-- save all test runs as JSON for the new comparison page
create table all_query_runs_json engine File(JSON, 'report/all-query-runs.json') as
select test, query_display_name query,
select test, query_index, query_display_name query,
left, right, diff, stat_threshold, report_threshold,
versions_runs[1] runs_left, versions_runs[2] runs_right
from (
select
@ -391,7 +396,17 @@ create table all_query_runs_json engine File(JSON, 'report/all-query-runs.json')
)
group by test, query_index
) runs
left join query_display_names using (test, query_index)
left join query_display_names
on runs.test = query_display_names.test
and runs.query_index = query_display_names.query_index
left join file('analyze/report-thresholds.tsv',
TSV, 'test text, report_threshold float') thresholds
on runs.test = thresholds.test
left join query_metric_stats
on runs.test = query_metric_stats.test
and runs.query_index = query_metric_stats.query_index
where
query_metric_stats.metric_name = 'server_time'
;
create table changed_perf_tsv engine File(TSV, 'report/changed-perf.tsv') as
@ -462,9 +477,10 @@ create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.ts
order by test, query_index;
" 2> >(tee -a report/errors.log 1>&2)
# Prepare source data for metrics and flamegraphs for unstable queries.
for version in {right,left}
do
do
rm -rf data
clickhouse-local --query "
create view queries_for_flamegraph as
@ -527,7 +543,13 @@ create view trace_log as select *
from file('$version-trace-log.tsv', TSVWithNamesAndTypes,
'$(cat "$version-trace-log.tsv.columns")');
create view addresses_src as select *
create view addresses_src as select addr,
-- Some functions change name between builds, e.g. '__clone' or 'clone' or
-- even '__GI__clone@@GLIBC_2.32'. This breaks differential flame graphs, so
-- filter them out here.
[name, 'clone.S (filtered by script)', 'pthread_cond_timedwait (filtered by script)']
-- this line is a subscript operator of the above array
[1 + multiSearchFirstIndex(name, ['clone.S', 'pthread_cond_timedwait'])] name
from file('$version-addresses.tsv', TSVWithNamesAndTypes,
'$(cat "$version-addresses.tsv.columns")');
@ -539,7 +561,8 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes,
select
test, query_index, query_id,
count() value,
joinGet(addresses_join_$version, 'name', arrayJoin(trace)) metric
joinGet(addresses_join_$version, 'name', arrayJoin(trace))
|| '(' || toString(trace_type) || ')' metric
from trace_log
join unstable_query_runs using query_id
group by test, query_index, query_id, metric
@ -547,7 +570,7 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes,
;
create table metric_devation engine File(TSVWithNamesAndTypes,
'metric-deviation.$version.rep') as
'report/metric-deviation.$version.tsv') as
-- first goes the key used to split the file with grep
select test, query_index, query_display_name,
d, q, metric
@ -562,16 +585,18 @@ create table metric_devation engine File(TSVWithNamesAndTypes,
group by test, query_index, metric
having d > 0.5
) metrics
left join unstable_query_runs using (test, query_index)
left join query_display_names using (test, query_index)
order by test, query_index, d desc
;
create table stacks engine File(TSV, 'stacks.$version.rep') as
create table stacks engine File(TSV, 'report/stacks.$version.tsv') as
select
-- first goes the key used to split the file with grep
test, query_index, any(query_display_name),
test, query_index, trace_type, any(query_display_name),
-- next go the stacks in flamegraph format: 'func1;...;funcN count'
arrayStringConcat(
arrayMap(x -> joinGet(addresses_join_$version, 'name', x),
arrayMap(
addr -> joinGet(addresses_join_$version, 'name', addr),
arrayReverse(trace)
),
';'
@ -579,30 +604,58 @@ create table stacks engine File(TSV, 'stacks.$version.rep') as
count() c
from trace_log
join unstable_query_runs using query_id
group by test, query_index, trace
group by test, query_index, trace_type, trace
order by test, query_index, trace_type, trace
;
" 2> >(tee -a report/errors.log 1>&2) # do not run in parallel because they use the same data dir for StorageJoins which leads to weird errors.
done
wait
# Create per-query flamegraphs and files with metrics
# Create per-query flamegraphs
IFS=$'\n'
for version in {right,left}
do
for query in $(cut -d' ' -f1,2,3 "stacks.$version.rep" | sort | uniq)
for query in $(cut -d' ' -f1-4 "report/stacks.$version.tsv" | sort | uniq)
do
query_file=$(echo "$query" | cut -c-120 | sed 's/[/ ]/_/g')
echo "$query_file" >> report/query-files.txt
# Build separate .svg flamegraph for each query.
# -F is somewhat unsafe because it might match not the beginning of the
# string, but this is unlikely and escaping the query for grep is a pain.
grep -F "$query " "report/stacks.$version.tsv" \
| cut -f 5- \
| sed 's/\t/ /g' \
| tee "report/tmp/$query_file.stacks.$version.tsv" \
| ~/fg/flamegraph.pl --hash > "$query_file.$version.svg" &
done
done
wait
unset IFS
# Create differential flamegraphs.
IFS=$'\n'
for query_file in $(cat report/query-files.txt)
do
~/fg/difffolded.pl "report/tmp/$query_file.stacks.left.tsv" \
"report/tmp/$query_file.stacks.right.tsv" \
| tee "report/tmp/$query_file.stacks.diff.tsv" \
| ~/fg/flamegraph.pl > "$query_file.diff.svg" &
done
unset IFS
wait
# Create per-query files with metrics. Note that the key is different from flamegraphs.
IFS=$'\n'
for version in {right,left}
do
for query in $(cut -d' ' -f1-3 "report/metric-deviation.$version.tsv" | sort | uniq)
do
query_file=$(echo "$query" | cut -c-120 | sed 's/[/ ]/_/g')
# Build separate .svg flamegraph for each query.
grep -F "$query " "stacks.$version.rep" \
| cut -d' ' -f 2- \
| sed 's/\t/ /g' \
| tee "$query_file.stacks.$version.rep" \
| ~/fg/flamegraph.pl > "$query_file.$version.svg" &
# Copy metric stats into separate files as well.
grep -F "$query " "metric-deviation.$version.rep" \
| cut -f2- > "$query_file.$version.metrics.rep" &
# Ditto the above comment about -F.
grep -F "$query " "report/metric-deviation.$version.tsv" \
| cut -f4- > "$query_file.$version.metrics.rep" &
done
done
wait

View File

@ -46,7 +46,13 @@ function download
done
mkdir ~/fg ||:
cd ~/fg && wget -nv -nd -c "https://raw.githubusercontent.com/brendangregg/FlameGraph/master/flamegraph.pl" && chmod +x ~/fg/flamegraph.pl &
(
cd ~/fg
wget -nv -nd -c "https://raw.githubusercontent.com/brendangregg/FlameGraph/master/flamegraph.pl"
wget -nv -nd -c "https://raw.githubusercontent.com/brendangregg/FlameGraph/master/difffolded.pl"
chmod +x ~/fg/difffolded.pl
chmod +x ~/fg/flamegraph.pl
) &
wait
}

View File

@ -81,8 +81,13 @@ if [ "$REF_PR" == "" ]; then echo Reference PR is not specified ; exit 1 ; fi
fi
) | tee right-commit.txt
# Prepare the list of changed tests for use by compare.sh
git -C ch diff --name-only "$SHA_TO_TEST" "$(git -C ch merge-base "$SHA_TO_TEST"~ master)" -- tests/performance | tee changed-tests.txt
if [ "$PR_TO_TEST" != "0" ]
then
# Prepare the list of tests changed in the PR for use by compare.sh. Compare to
# merge base, because master might be far in the future and have unrelated test
# changes.
git -C ch diff --name-only "$SHA_TO_TEST" "$(git -C ch merge-base "$SHA_TO_TEST" master)" -- tests/performance | tee changed-tests.txt
fi
# Set python output encoding so that we can print queries with Russian letters.
export PYTHONIOENCODING=utf-8

View File

@ -11,6 +11,9 @@ import string
import time
import traceback
def tsv_escape(s):
return s.replace('\\', '\\\\').replace('\t', '\\t').replace('\n', '\\n').replace('\r','')
stage_start_seconds = time.perf_counter()
def report_stage_end(stage_name):
@ -112,8 +115,9 @@ for t in tables:
try:
res = c.execute("select 1 from {} limit 1".format(t))
except:
print('skipped\t' + traceback.format_exception_only(*sys.exc_info()[:2])[-1])
traceback.print_exc()
exception_message = traceback.format_exception_only(*sys.exc_info()[:2])[-1]
skipped_message = ' '.join(exception_message.split('\n')[:2])
print(f'skipped\t{tsv_escape(skipped_message)}')
sys.exit(0)
report_stage_end('preconditions')
@ -135,9 +139,6 @@ for c in connections:
report_stage_end('fill')
# Run test queries
def tsv_escape(s):
return s.replace('\\', '\\\\').replace('\t', '\\t').replace('\n', '\\n').replace('\r','')
test_query_templates = [q.text for q in root.findall('query')]
test_queries = substitute_parameters(test_query_templates)
@ -151,7 +152,7 @@ for query_index, q in enumerate(test_queries):
# use the test name + the test-wide query index.
query_display_name = q
if len(query_display_name) > 1000:
query_display_name = f'{query_display_name[:1000]}...({i})'
query_display_name = f'{query_display_name[:1000]}...({query_index})'
print(f'display-name\t{query_index}\t{tsv_escape(query_display_name)}')

View File

@ -182,6 +182,16 @@ if args.report == 'main':
print_tested_commits()
run_error_rows = tsvRows('run-errors.tsv')
error_tests += len(run_error_rows)
printSimpleTable('Run errors', ['Test', 'Error'], run_error_rows)
slow_on_client_rows = tsvRows('report/slow-on-client.tsv')
error_tests += len(slow_on_client_rows)
printSimpleTable('Slow on client',
['Client time, s', 'Server time, s', 'Ratio', 'Query'],
slow_on_client_rows)
def print_changes():
rows = tsvRows('report/changed-perf.tsv')
if not rows:
@ -221,12 +231,6 @@ if args.report == 'main':
print_changes()
slow_on_client_rows = tsvRows('report/slow-on-client.tsv')
error_tests += len(slow_on_client_rows)
printSimpleTable('Slow on client',
['Client time, s', 'Server time, s', 'Ratio', 'Query'],
slow_on_client_rows)
def print_unstable_queries():
global unstable_queries
global very_unstable_queries
@ -265,10 +269,6 @@ if args.report == 'main':
print_unstable_queries()
run_error_rows = tsvRows('run-errors.tsv')
error_tests += len(run_error_rows)
printSimpleTable('Run errors', ['Test', 'Error'], run_error_rows)
skipped_tests_rows = tsvRows('analyze/skipped-tests.tsv')
printSimpleTable('Skipped tests', ['Test', 'Reason'], skipped_tests_rows)

View File

@ -1,6 +1,6 @@
---
toc_priority: 62
toc_title: Overview of ClickHouse Architecture
toc_title: Architecture Overview
---
# Overview of ClickHouse Architecture {#overview-of-clickhouse-architecture}

View File

@ -1,6 +1,6 @@
---
toc_priority: 63
toc_title: Browse Source Code
toc_priority: 71
toc_title: Source Code
---
# Browse ClickHouse Source Code {#browse-clickhouse-source-code}

View File

@ -1,6 +1,8 @@
---
toc_folder_title: Engines
toc_priority: 25
toc_title: hidden
toc_hidden: true
---
{## [Original article](https://clickhouse.tech/docs/en/engines/) ##}

View File

@ -1,6 +1,8 @@
---
toc_folder_title: F.A.Q.
toc_priority: 76
toc_title: hidden
toc_hidden: true
---

View File

@ -5,11 +5,41 @@ toc_title: System Tables
# System Tables {#system-tables}
System tables are used for implementing part of the systems functionality, and for providing access to information about how the system is working.
You cant delete a system table (but you can perform DETACH).
System tables dont have files with data on the disk or files with metadata. The server creates all the system tables when it starts.
System tables are read-only.
They are located in the system database.
## Introduction
System tables provide information about:
- Server states, processes, and environment.
- Server's internal processes.
System tables:
- Located in the `system` database.
- Available only for reading data.
- Can't be dropped or altered, but can be detached.
The `metric_log`, `query_log`, `query_thread_log`, `trace_log` system tables store data in a storage filesystem. Other system tables store their data in RAM. ClickHouse server creates such system tables at the start.
### Sources of System Metrics
For collecting system metrics ClickHouse server uses:
- `CAP_NET_ADMIN` capability.
- [procfs](https://en.wikipedia.org/wiki/Procfs) (only in Linux).
**procfs**
If ClickHouse server doesn't have `CAP_NET_ADMIN` capability, it tries to fall back to `ProcfsMetricsProvider`. `ProcfsMetricsProvider` allows collecting per-query system metrics (for CPU and I/O).
If procfs is supported and enabled on the system, ClickHouse server collects these metrics:
- `OSCPUVirtualTimeMicroseconds`
- `OSCPUWaitMicroseconds`
- `OSIOWaitMicroseconds`
- `OSReadChars`
- `OSWriteChars`
- `OSReadBytes`
- `OSWriteBytes`
## system.asynchronous\_metrics {#system_tables-asynchronous_metrics}

View File

@ -11,7 +11,7 @@ All the functions accept zero arguments or one argument.
If an argument is passed, it can be any type, and its value is not used for anything.
The only purpose of this argument is to prevent common subexpression elimination, so that two different instances of the same function return different columns with different random numbers.
## rand {#rand}
## rand, rand32 {#rand}
Returns a pseudo-random UInt32 number, evenly distributed among all UInt32-type numbers.
Uses a linear congruential generator.

View File

@ -291,30 +291,30 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE minmax GRANULARITY
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4
```
#### Поддержка для функций {#podderzhka-dlia-funktsii}
#### Поддержка для функций {#functions-support}
Условия в секции `WHERE` содержат вызовы функций, оперирующих со столбцами. Если столбец - часть индекса, ClickHouse пытается использовать индекс при выполнении функции. Для разных видов индексов, ClickHouse поддерживает различные наборы функций, которые могут использоваться индексами.
Индекс `set` используется со всеми функциями. Наборы функций для остальных индексов представлены в таблице ниже.
| Function (operator) / Index | primary key | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter |
|----------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------|
| [equals (=, ==)](../../../engines/table-engines/mergetree-family/mergetree.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notEquals(!=, \<\>)](../../../engines/table-engines/mergetree-family/mergetree.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [like](../../../engines/table-engines/mergetree-family/mergetree.md#function-like) | ✔ | ✔ | ✔ | ✗ | ✗ |
| [notLike](../../../engines/table-engines/mergetree-family/mergetree.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ |
| [startsWith](../../../engines/table-engines/mergetree-family/mergetree.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ |
| [endsWith](../../../engines/table-engines/mergetree-family/mergetree.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ |
| [multiSearchAny](../../../engines/table-engines/mergetree-family/mergetree.md#function-multisearchany) | ✗ | ✗ | ✔ | ✔ | ✗ |
| [in](../../../engines/table-engines/mergetree-family/mergetree.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notIn](../../../engines/table-engines/mergetree-family/mergetree.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [less (\<)](../../../engines/table-engines/mergetree-family/mergetree.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [greater (\>)](../../../engines/table-engines/mergetree-family/mergetree.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [lessOrEquals (\<=)](../../../engines/table-engines/mergetree-family/mergetree.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [greaterOrEquals (\>=)](../../../engines/table-engines/mergetree-family/mergetree.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [empty](../../../engines/table-engines/mergetree-family/mergetree.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [notEmpty](../../../engines/table-engines/mergetree-family/mergetree.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ |
| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ |
| Функция (оператор) / Индекс | primary key | minmax | ngrambf\_v1 | tokenbf\_v1 | bloom\_filter |
|------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------|
| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✗ | ✗ |
| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✗ | ✗ |
| [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ |
| [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ |
| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ |
| [in](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notIn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ |
| [less (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [greater (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [lessOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [greaterOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [empty](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ |
| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ |
| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ |
Функции с постоянным агрументом, который меньше, чем размер ngram не могут использовать индекс `ngrambf_v1` для оптимизации запроса.

View File

@ -58,17 +58,6 @@ def build_for_lang(lang, args):
'custom_dir': os.path.join(os.path.dirname(__file__), '..', args.theme_dir),
'language': lang,
'direction': 'rtl' if lang == 'fa' else 'ltr',
# TODO: cleanup
'feature': {
'tabs': False
},
'palette': {
'primary': 'white',
'accent': 'white'
},
'font': False,
'logo': 'images/logo.svg',
'favicon': 'assets/images/favicon.ico',
'static_templates': ['404.html'],
'extra': {
'now': int(time.mktime(datetime.datetime.now().timetuple())) # TODO better way to avoid caching

View File

@ -30,7 +30,7 @@ PyYAML==5.3.1
repackage==0.7.3
requests==2.23.0
singledispatch==3.4.0.3
six==1.14.0
six==1.15.0
soupsieve==2.0.1
termcolor==1.1.0
tornado==5.1.1

View File

@ -205,3 +205,9 @@ endif ()
if (TARGET clickhouse-server AND TARGET copy-headers)
add_dependencies(clickhouse-server copy-headers)
endif ()
if (ENABLE_TESTS AND USE_GTEST)
set (CLICKHOUSE_ALL_TESTS_TARGETS local_date_time_comparison unit_tests_libcommon unit_tests_dbms hashing_write_buffer hashing_read_buffer in_join_subqueries_preprocessor expression_analyzer)
add_custom_target (clickhouse-tests ALL DEPENDS ${CLICKHOUSE_ALL_TESTS_TARGETS})
add_dependencies(clickhouse-bundle clickhouse-tests)
endif()

View File

@ -424,7 +424,7 @@ private:
std::cerr << percent << "%\t\t";
for (const auto & info : infos)
{
std::cerr << info->sampler.quantileNearest(percent / 100.0) << " sec." << "\t";
std::cerr << info->sampler.quantileNearest(percent / 100.0) << " sec.\t";
}
std::cerr << "\n";
};
@ -459,7 +459,7 @@ private:
auto print_percentile = [&json_out](Stats & info, auto percent, bool with_comma = true)
{
json_out << "\"" << percent << "\"" << ": " << info.sampler.quantileNearest(percent / 100.0) << (with_comma ? ",\n" : "\n");
json_out << "\"" << percent << "\": " << info.sampler.quantileNearest(percent / 100.0) << (with_comma ? ",\n" : "\n");
};
json_out << "{\n";
@ -469,7 +469,7 @@ private:
const auto & info = infos[i];
json_out << double_quote << connections[i]->getDescription() << ": {\n";
json_out << double_quote << "statistics" << ": {\n";
json_out << double_quote << "statistics: {\n";
print_key_value("QPS", info->queries / info->work_time);
print_key_value("RPS", info->read_rows / info->work_time);
@ -479,7 +479,7 @@ private:
print_key_value("num_queries", info->queries.load(), false);
json_out << "},\n";
json_out << double_quote << "query_time_percentiles" << ": {\n";
json_out << double_quote << "query_time_percentiles: {\n";
for (int percent = 0; percent <= 90; percent += 10)
print_percentile(*info, percent);

View File

@ -5,6 +5,7 @@
#include <Client/Connection.h>
#include <IO/ConnectionTimeouts.h>
#include <common/LineReader.h>
#include <thread>
namespace DB

View File

@ -26,7 +26,7 @@ void ClusterCopier::init()
if (response.error != Coordination::ZOK)
return;
UInt64 version = ++task_description_version;
LOG_DEBUG(log, "Task description should be updated, local version " << version);
LOG_DEBUG(log, "Task description should be updated, local version {}", version);
};
task_description_path = task_zookeeper_path + "/description";
@ -47,7 +47,7 @@ void ClusterCopier::init()
task_table.initShards(task_cluster->random_engine);
}
LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks");
LOG_DEBUG(log, "Will process {} table tasks", task_cluster->table_tasks.size());
/// Do not initialize tables, will make deferred initialization in process()
@ -85,7 +85,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
{
TaskTable & task_table = task_shard->task_table;
LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription());
LOG_INFO(log, "Discover partitions of shard {}", task_shard->getDescription());
auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); };
auto existing_partitions_names = retry(get_partitions, 60);
@ -132,8 +132,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
{
if (!task_table.enabled_partitions_set.count(partition_name))
{
LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in "
<< "enabled_partitions of " << task_table.table_id);
LOG_DEBUG(log, "Partition {} will not be processed, since it is not in enabled_partitions of {}", partition_name, task_table.table_id);
}
}
}
@ -165,11 +164,10 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
for (const String & missing_partition : missing_partitions)
ss << " " << missing_partition;
LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard "
<< task_shard->getDescription() << " :" << ss.str());
LOG_WARNING(log, "There are no {} partitions from enabled_partitions in shard {} :{}", missing_partitions.size(), task_shard->getDescription(), ss.str());
}
LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription());
LOG_DEBUG(log, "Will copy {} partitions from shard {}", task_shard->partition_tasks.size(), task_shard->getDescription());
}
void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads)
@ -181,7 +179,7 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts,
for (const TaskShardPtr & task_shard : task_table.all_shards)
thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); });
LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs");
LOG_DEBUG(log, "Waiting for {} setup jobs", thread_pool.active());
thread_pool.wait();
}
}
@ -205,7 +203,8 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s
if (code && force)
zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")");
LOG_DEBUG(log, "Task description {} uploaded to {} with result {} ({})",
((code && !force) ? "not " : ""), local_task_description_path, code, zookeeper->error2string(code));
}
void ClusterCopier::reloadTaskDescription()
@ -221,7 +220,7 @@ void ClusterCopier::reloadTaskDescription()
if (code)
throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS);
LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid);
LOG_DEBUG(log, "Loading description, zxid={}", task_description_current_stat.czxid);
auto config = getConfigurationFromXMLString(task_config_str);
/// Setup settings
@ -251,9 +250,7 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts)
{
for (TaskTable & task_table : task_cluster->table_tasks)
{
LOG_INFO(log, "Process table task " << task_table.table_id << " with "
<< task_table.all_shards.size() << " shards, "
<< task_table.local_shards.size() << " of them are local ones");
LOG_INFO(log, "Process table task {} with {} shards, {} of them are local ones", task_table.table_id, task_table.all_shards.size(), task_table.local_shards.size());
if (task_table.all_shards.empty())
continue;
@ -357,8 +354,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee
if (static_cast<UInt64>(stat.numChildren) >= task_cluster->max_workers)
{
LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")"
<< ". Postpone processing " << description);
LOG_DEBUG(log, "Too many workers ({}, maximum {}). Postpone processing {}", stat.numChildren, task_cluster->max_workers, description);
if (unprioritized)
current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time);
@ -419,7 +415,7 @@ bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_tabl
{
bool piece_is_done = checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition);
if (!piece_is_done)
LOG_DEBUG(log, "Partition " << partition_name << " piece " + toString(piece_number) + " is not already done.");
LOG_DEBUG(log, "Partition {} piece {} is not already done.", partition_name, piece_number);
answer &= piece_is_done;
}
@ -435,8 +431,7 @@ bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_tabl
bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name,
size_t piece_number, const TasksShard & shards_with_partition)
{
LOG_DEBUG(log, "Check that all shards processed partition " << partition_name
<< " piece " + toString(piece_number) + " successfully");
LOG_DEBUG(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number);
auto zookeeper = context.getZooKeeper();
@ -465,8 +460,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data);
if (status.state != TaskState::Finished)
{
LOG_INFO(log, "The task " << res.data << " is being rewritten by "
<< status.owner << ". Partition piece will be rechecked");
LOG_INFO(log, "The task {} is being rewritten by {}. Partition piece will be rechecked", res.data, status.owner);
return false;
}
@ -484,7 +478,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
if (!is_clean)
{
LOG_INFO(log, "Partition " << partition_name << " become dirty");
LOG_INFO(log, "Partition {} become dirty", partition_name);
return false;
}
@ -501,8 +495,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
}
catch (const Coordination::Exception & e)
{
LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number "
<< toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText());
LOG_INFO(log, "A ZooKeeper error occurred while checking partition {} piece number {}. Will recheck the partition. Error: {}", partition_name, toString(piece_number), e.displayText());
return false;
}
@ -511,12 +504,12 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
{
if (zxid1[shard_num] != zxid2[shard_num])
{
LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked");
LOG_INFO(log, "The task {} is being modified now. Partition piece will be rechecked", piece_status_paths[shard_num]);
return false;
}
}
LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " is copied successfully");
LOG_INFO(log, "Partition {} piece number {} is copied successfully", partition_name, toString(piece_number));
return true;
}
@ -530,7 +523,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
inject_fault = value < move_fault_probability;
}
LOG_DEBUG(log, "Try to move " << partition_name << " to destionation table");
LOG_DEBUG(log, "Try to move {} to destionation table", partition_name);
auto zookeeper = context.getZooKeeper();
@ -548,7 +541,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
{
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Someone is already moving pieces " << current_partition_attach_is_active);
LOG_DEBUG(log, "Someone is already moving pieces {}", current_partition_attach_is_active);
return TaskStatus::Active;
}
@ -565,16 +558,13 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data);
if (status.state == TaskState::Finished)
{
LOG_DEBUG(log, "All pieces for partition from this task " << current_partition_attach_is_active
<< " has been successfully moved to destination table by " << status.owner);
LOG_DEBUG(log, "All pieces for partition from this task {} has been successfully moved to destination table by {}", current_partition_attach_is_active, status.owner);
return TaskStatus::Finished;
}
/// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process.
/// Initialize DROP PARTITION
LOG_DEBUG(log, "Moving piece for partition " << current_partition_attach_is_active
<< " has not been successfully finished by " << status.owner
<< ". Will try to move by myself.");
LOG_DEBUG(log, "Moving piece for partition {} has not been successfully finished by {}. Will try to move by myself.", current_partition_attach_is_active, status.owner);
/// Remove is_done marker.
zookeeper->remove(current_partition_attach_is_done);
@ -591,9 +581,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
/// Move partition to original destination table.
for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
{
LOG_DEBUG(log, "Trying to move partition " << partition_name
<< " piece " << toString(current_piece_number)
<< " to original table");
LOG_DEBUG(log, "Trying to move partition {} piece {} to original table", partition_name, toString(current_piece_number));
ASTPtr query_alter_ast;
String query_alter_ast_string;
@ -614,7 +602,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
" ATTACH PARTITION " + partition_name +
" FROM " + getQuotedTable(helping_table);
LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string);
LOG_DEBUG(log, "Executing ALTER query: {}", query_alter_ast_string);
try
{
@ -626,13 +614,11 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
PoolMode::GET_MANY,
ClusterExecutionMode::ON_EACH_NODE);
LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes));
LOG_INFO(log, "Number of nodes that executed ALTER query successfully : {}", toString(num_nodes));
}
catch (...)
{
LOG_DEBUG(log, "Error while moving partition " << partition_name
<< " piece " << toString(current_piece_number)
<< "to original table");
LOG_DEBUG(log, "Error while moving partition {} piece {} to original table", partition_name, toString(current_piece_number));
throw;
}
@ -647,7 +633,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) +
" PARTITION " + partition_name + " DEDUPLICATE;";
LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string);
LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_alter_ast_string);
UInt64 num_nodes = executeQueryOnCluster(
task_table.cluster_push,
@ -656,14 +642,12 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
&task_cluster->settings_push,
PoolMode::GET_MANY);
LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : "
<< toString(num_nodes));
LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes));
}
}
catch (...)
{
LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name
<< "in the original table");
LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name);
throw;
}
}
@ -759,8 +743,7 @@ bool ClusterCopier::tryDropPartitionPiece(
{
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " piece "
<< toString(current_piece_number) << " is cleaning now by somebody, sleep");
LOG_DEBUG(log, "Partition {} piece {} is cleaning now by somebody, sleep", task_partition.name, toString(current_piece_number));
std::this_thread::sleep_for(default_sleep_time);
return false;
}
@ -773,8 +756,7 @@ bool ClusterCopier::tryDropPartitionPiece(
{
if (stat.numChildren != 0)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren
<< " active workers while trying to drop it. Going to sleep.");
LOG_DEBUG(log, "Partition {} contains {} active workers while trying to drop it. Going to sleep.", task_partition.name, stat.numChildren);
std::this_thread::sleep_for(default_sleep_time);
return false;
}
@ -794,7 +776,7 @@ bool ClusterCopier::tryDropPartitionPiece(
{
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep");
LOG_DEBUG(log, "Partition {} is being filled now by somebody, sleep", task_partition.name);
return false;
}
@ -832,7 +814,7 @@ bool ClusterCopier::tryDropPartitionPiece(
/// It is important, DROP PARTITION must be done synchronously
settings_push.replication_alter_partitions_sync = 2;
LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query);
LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query);
/// We have to drop partition_piece on each replica
size_t num_shards = executeQueryOnCluster(
cluster_push, query,
@ -841,7 +823,7 @@ bool ClusterCopier::tryDropPartitionPiece(
PoolMode::GET_MANY,
ClusterExecutionMode::ON_EACH_NODE);
LOG_INFO(log, "DROP PARTITION was successfully executed on " << num_shards << " nodes of a cluster.");
LOG_INFO(log, "DROP PARTITION was successfully executed on {} nodes of a cluster.", num_shards);
/// Update the locking node
if (!my_clock.is_stale())
@ -859,13 +841,12 @@ bool ClusterCopier::tryDropPartitionPiece(
return false;
}
LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number)
<< " was dropped on cluster " << task_table.cluster_push_name);
LOG_INFO(log, "Partition {} piece {} was dropped on cluster {}", task_partition.name, toString(current_piece_number), task_table.cluster_push_name);
if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS)
zookeeper->set(current_shards_path, host_id);
}
LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " is safe for work now.");
LOG_INFO(log, "Partition {} piece {} is safe for work now.", task_partition.name, toString(current_piece_number));
return true;
}
@ -889,7 +870,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
++cluster_partition.total_tries;
LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster");
LOG_DEBUG(log, "Processing partition {} for the whole cluster", partition_name);
/// Process each source shard having current partition and copy current partition
/// NOTE: shards are sorted by "distance" to current host
@ -911,7 +892,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
{
const size_t number_of_splits = task_table.number_of_splits;
shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, number_of_splits));
LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription());
LOG_DEBUG(log, "Discovered partition {} in shard {}", partition_name, shard->getDescription());
/// To save references in the future.
auto shard_partition_it = shard->partition_tasks.find(partition_name);
PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces;
@ -924,7 +905,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
}
else
{
LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name);
LOG_DEBUG(log, "Found that shard {} does not contain current partition {}", shard->getDescription(), partition_name);
continue;
}
}
@ -1030,21 +1011,20 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
task_table.rows_copied += cluster_partition.rows_copied;
double elapsed = cluster_partition.elapsed_time_seconds;
LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name
<< ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes"
<< ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows"
<< " and " << cluster_partition.blocks_copied << " source blocks are copied");
LOG_INFO(log, "It took {} seconds to copy partition {}: {} uncompressed bytes, {} rows and {} source blocks are copied",
elapsed, partition_name,
formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied),
formatReadableQuantity(cluster_partition.rows_copied),
cluster_partition.blocks_copied);
if (cluster_partition.rows_copied)
{
LOG_INFO(log, "Average partition speed: "
<< formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second.");
LOG_INFO(log, "Average partition speed: {} per second.", formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed));
}
if (task_table.rows_copied)
{
LOG_INFO(log, "Average table " << task_table.table_id << " speed: "
<< formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second.");
LOG_INFO(log, "Average table {} speed: {} per second.", task_table.table_id, formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed));
}
}
}
@ -1055,8 +1035,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
if (!table_is_done)
{
LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet."
<< "Copied " << finished_partitions << " of " << required_partitions << ", will retry");
LOG_INFO(log, "Table {} is not processed yet.Copied {} of {}, will retry", task_table.table_id, finished_partitions, required_partitions);
}
return table_is_done;
@ -1104,9 +1083,11 @@ TaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTim
{
for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
{
LOG_INFO(log, "Attempt number " << try_num << " to process partition " << task_partition.name
<< " piece number " << piece_number << " on shard number " << task_partition.task_shard.numberInCluster()
<< " with index " << task_partition.task_shard.indexInCluster());
LOG_INFO(log, "Attempt number {} to process partition {} piece number {} on shard number {} with index {}.",
try_num, task_partition.name, piece_number,
task_partition.task_shard.numberInCluster(),
task_partition.task_shard.indexInCluster());
res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task);
/// Exit if success
@ -1210,7 +1191,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
/// Load balancing
auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task);
LOG_DEBUG(log, "Processing " << current_task_piece_status_path);
LOG_DEBUG(log, "Processing {}", current_task_piece_status_path);
const String piece_status_path = partition_piece.getPartitionPieceShardsPath();
@ -1221,14 +1202,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
/// Do not start if partition piece is dirty, try to clean it
if (is_clean)
{
LOG_DEBUG(log, "Partition " << task_partition.name
<< " piece " + toString(current_piece_number) + " appears to be clean");
LOG_DEBUG(log, "Partition {} piece {} appears to be clean", task_partition.name, current_piece_number);
zookeeper->createAncestors(current_task_piece_status_path);
}
else
{
LOG_DEBUG(log, "Partition " << task_partition.name
<< " piece " + toString(current_piece_number) + " is dirty, try to drop it");
LOG_DEBUG(log, "Partition {} piece {} is dirty, try to drop it", task_partition.name, current_piece_number);
try
{
@ -1253,7 +1232,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
{
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path);
LOG_DEBUG(log, "Someone is already processing {}", current_task_piece_is_active_path);
return TaskStatus::Active;
}
@ -1269,16 +1248,13 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data);
if (status.state == TaskState::Finished)
{
LOG_DEBUG(log, "Task " << current_task_piece_status_path
<< " has been successfully executed by " << status.owner);
LOG_DEBUG(log, "Task {} has been successfully executed by {}", current_task_piece_status_path, status.owner);
return TaskStatus::Finished;
}
/// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process.
/// Initialize DROP PARTITION
LOG_DEBUG(log, "Task " << current_task_piece_status_path
<< " has not been successfully finished by " << status.owner
<< ". Partition will be dropped and refilled.");
LOG_DEBUG(log, "Task {} has not been successfully finished by {}. Partition will be dropped and refilled.", current_task_piece_status_path, status.owner);
create_is_dirty_node(clean_state_clock);
return TaskStatus::Error;
@ -1293,11 +1269,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent);
if (res == Coordination::ZNODEEXISTS)
LOG_DEBUG(log, "Partition " << task_partition.name << " piece "
+ toString(current_piece_number) + " is absent on current replica of a shard. But other replicas have already marked it as done.");
LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. But other replicas have already marked it as done.", task_partition.name, current_piece_number);
if (res == Coordination::ZOK)
LOG_DEBUG(log, "Partition " << task_partition.name << " piece "
+ toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.");
LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.", task_partition.name, current_piece_number);
return TaskStatus::Finished;
}
@ -1325,18 +1299,14 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
if (count != 0)
{
LOG_INFO(log, "Partition " << task_partition.name << " piece "
<< current_piece_number << "is not empty. In contains " << count << " rows.");
LOG_INFO(log, "Partition {} piece {}is not empty. In contains {} rows.", task_partition.name, current_piece_number, count);
Coordination::Stat stat_shards{};
zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards);
/// NOTE: partition is still fresh if dirt discovery happens before cleaning
if (stat_shards.numChildren == 0)
{
LOG_WARNING(log, "There are no workers for partition " << task_partition.name
<< " piece " << toString(current_piece_number)
<< ", but destination table contains " << count << " rows"
<< ". Partition will be dropped and refilled.");
LOG_WARNING(log, "There are no workers for partition {} piece {}, but destination table contains {} rows. Partition will be dropped and refilled.", task_partition.name, toString(current_piece_number), count);
create_is_dirty_node(clean_state_clock);
return TaskStatus::Error;
@ -1353,14 +1323,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);
if (clean_state_clock != new_clean_state_clock)
{
LOG_INFO(log, "Partition " << task_partition.name << " piece "
<< toString(current_piece_number) << " clean state changed, cowardly bailing");
LOG_INFO(log, "Partition {} piece {} clean state changed, cowardly bailing", task_partition.name, toString(current_piece_number));
return TaskStatus::Error;
}
else if (!new_clean_state_clock.is_clean())
{
LOG_INFO(log, "Partition " << task_partition.name << " piece "
<< toString(current_piece_number) << " is dirty and will be dropped and refilled");
LOG_INFO(log, "Partition {} piece {} is dirty and will be dropped and refilled", task_partition.name, toString(current_piece_number));
create_is_dirty_node(new_clean_state_clock);
return TaskStatus::Error;
}
@ -1387,12 +1355,11 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
String query = queryToString(create_query_push_ast);
LOG_DEBUG(log, "Create destination tables. Query: " << query);
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query,
create_query_push_ast, &task_cluster->settings_push,
PoolMode::GET_MANY);
LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push)
<< " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount());
LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
}
/// Do the copying
@ -1407,8 +1374,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
// Select all fields
ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ true, inject_fault ? "1" : "");
LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription()
<< " : " << queryToString(query_select_ast));
LOG_DEBUG(log, "Executing SELECT query and pull from {} : {}", task_shard.getDescription(), queryToString(query_select_ast));
ASTPtr query_insert_ast;
{
@ -1419,7 +1385,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
const auto & settings = context.getSettingsRef();
query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
LOG_DEBUG(log, "Executing INSERT query: " << query);
LOG_DEBUG(log, "Executing INSERT query: {}", query);
}
try
@ -1501,8 +1467,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
}
}
LOG_INFO(log, "Partition " << task_partition.name << " piece "
<< toString(current_piece_number) << " copied. But not moved to original destination table.");
LOG_INFO(log, "Partition {} piece {} copied. But not moved to original destination table.", task_partition.name, toString(current_piece_number));
/// Try create original table (if not exists) on each shard
@ -1513,12 +1478,11 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
String query = queryToString(create_query_push_ast);
LOG_DEBUG(log, "Create destination tables. Query: " << query);
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query,
create_query_push_ast, &task_cluster->settings_push,
PoolMode::GET_MANY);
LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push)
<< " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount());
LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
}
catch (...)
{
@ -1531,14 +1495,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);
if (clean_state_clock != new_clean_state_clock)
{
LOG_INFO(log, "Partition " << task_partition.name << " piece "
<< toString(current_piece_number) << " clean state changed, cowardly bailing");
LOG_INFO(log, "Partition {} piece {} clean state changed, cowardly bailing", task_partition.name, toString(current_piece_number));
return TaskStatus::Error;
}
else if (!new_clean_state_clock.is_clean())
{
LOG_INFO(log, "Partition " << task_partition.name << " piece "
<< toString(current_piece_number) << " became dirty and will be dropped and refilled");
LOG_INFO(log, "Partition {} piece {} became dirty and will be dropped and refilled", task_partition.name, toString(current_piece_number));
create_is_dirty_node(new_clean_state_clock);
return TaskStatus::Error;
}
@ -1582,7 +1544,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
const ClusterPtr & cluster_push = task_table.cluster_push;
Settings settings_push = task_cluster->settings_push;
LOG_DEBUG(log, "Execute distributed DROP TABLE: " << query);
LOG_DEBUG(log, "Execute distributed DROP TABLE: {}", query);
/// We have to drop partition_piece on each replica
UInt64 num_nodes = executeQueryOnCluster(
cluster_push, query,
@ -1591,7 +1553,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
PoolMode::GET_MANY,
ClusterExecutionMode::ON_EACH_NODE);
LOG_DEBUG(log, "DROP TABLE query was successfully executed on " << toString(num_nodes) << " nodes.");
LOG_DEBUG(log, "DROP TABLE query was successfully executed on {} nodes.", toString(num_nodes));
}
}
@ -1609,7 +1571,7 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT
const ClusterPtr & cluster_push = task_table.cluster_push;
Settings settings_push = task_cluster->settings_push;
LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query);
LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query);
/// We have to drop partition_piece on each replica
UInt64 num_nodes = executeQueryOnCluster(
cluster_push, query,
@ -1618,9 +1580,9 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT
PoolMode::GET_MANY,
ClusterExecutionMode::ON_EACH_NODE);
LOG_DEBUG(log, "DROP PARTITION query was successfully executed on " << toString(num_nodes) << " nodes.");
LOG_DEBUG(log, "DROP PARTITION query was successfully executed on {} nodes.", toString(num_nodes));
}
LOG_DEBUG(log, "All helping tables dropped partition " << partition_name);
LOG_DEBUG(log, "All helping tables dropped partition {}", partition_name);
}
String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings)
@ -1724,7 +1686,7 @@ std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti
const auto & settings = context.getSettingsRef();
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
LOG_DEBUG(log, "Computing destination partition set, executing query: " << query);
LOG_DEBUG(log, "Computing destination partition set, executing query: {}", query);
Context local_context = context;
local_context.setSettings(task_cluster->settings_pull);
@ -1744,7 +1706,7 @@ std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti
}
}
LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription());
LOG_DEBUG(log, "There are {} destination partitions in shard {}", res.size(), task_shard.getDescription());
return res;
}
@ -1765,8 +1727,7 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts,
query += " LIMIT 1";
LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition "
<< partition_quoted_name << " existence, executing query: " << query);
LOG_DEBUG(log, "Checking shard {} for partition {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, query);
ParserQuery parser_query(query.data() + query.size());
const auto & settings = context.getSettingsRef();
@ -1805,9 +1766,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
query += " LIMIT 1";
LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition "
<< partition_quoted_name << " piece " << std::to_string(current_piece_number)
<< "existence, executing query: " << query);
LOG_DEBUG(log, "Checking shard {} for partition {} piece {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query);
ParserQuery parser_query(query.data() + query.size());
const auto & settings = context.getSettingsRef();
@ -1817,11 +1776,9 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
local_context.setSettings(task_cluster->settings_pull);
auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows();
if (result != 0)
LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number "
<< std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription());
LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
else
LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number "
<< std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription());
LOG_DEBUG(log, "Partition {} piece number {} is ABSENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
return result != 0;
}
@ -1938,8 +1895,7 @@ UInt64 ClusterCopier::executeQueryOnCluster(
if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number)
{
LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on "
<< toString(successful_nodes) << " nodes. But had to be executed on " << toString(origin_replicas_number.load()));
LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on {} nodes. But had to be executed on {}", toString(successful_nodes), toString(origin_replicas_number.load()));
}

View File

@ -95,11 +95,7 @@ void ClusterCopierApp::mainImpl()
ThreadStatus thread_status;
auto * log = &logger();
LOG_INFO(log, "Starting clickhouse-copier ("
<< "id " << process_id << ", "
<< "host_id " << host_id << ", "
<< "path " << process_path << ", "
<< "revision " << ClickHouseRevision::get() << ")");
LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::get());
SharedContextHolder shared_context = Context::createShared();
auto context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));

View File

@ -183,11 +183,11 @@ public:
switch (rsp.type)
{
case Coordination::CREATED:
LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path);
LOG_DEBUG(logger, "CleanStateClock change: CREATED, at {}", rsp.path);
stale->store(true);
break;
case Coordination::CHANGED:
LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path);
LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at {}", rsp.path);
stale->store(true);
}
}

View File

@ -8,7 +8,6 @@
#include <Poco/NullChannel.h>
#include <Databases/DatabaseMemory.h>
#include <Storages/System/attachSystemTables.h>
#include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/loadMetadata.h>
@ -212,7 +211,7 @@ try
/// Lock path directory before read
status.emplace(context->getPath() + "status");
LOG_DEBUG(log, "Loading metadata from " << context->getPath());
LOG_DEBUG(log, "Loading metadata from {}", context->getPath());
loadMetadataSystem(*context);
attachSystemTables();
loadMetadata(*context);

View File

@ -4,13 +4,12 @@
#include <Poco/Util/Application.h>
#include <memory>
#include <loggers/Loggers.h>
#include <Interpreters/Context.h>
namespace DB
{
class Context;
/// Lightweight Application for clickhouse-local
/// No networking, no extra configs and working directories, no pid and status files, no dictionaries, no logging.
/// Quiet mode by default

View File

@ -62,7 +62,7 @@ namespace
void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
Poco::Net::HTMLForm params(request, request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
LOG_TRACE(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message)
{
@ -89,11 +89,11 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
if (params.has("schema"))
{
schema_name = params.get("schema");
LOG_TRACE(log, "Will fetch info for table '" << schema_name + "." + table_name << "'");
LOG_TRACE(log, "Will fetch info for table '{}'", schema_name + "." + table_name);
}
else
LOG_TRACE(log, "Will fetch info for table '" << table_name << "'");
LOG_TRACE(log, "Got connection str '" << connection_string << "'");
LOG_TRACE(log, "Will fetch info for table '{}'", table_name);
LOG_TRACE(log, "Got connection str '{}'", connection_string);
try
{
@ -124,7 +124,7 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
select->format(settings);
std::string query = ss.str();
LOG_TRACE(log, "Inferring structure with query '" << query << "'");
LOG_TRACE(log, "Inferring structure with query '{}'", query);
if (POCO_SQL_ODBC_CLASS::Utility::isError(POCO_SQL_ODBC_CLASS::SQLPrepare(hstmt, reinterpret_cast<SQLCHAR *>(query.data()), query.size())))
throw POCO_SQL_ODBC_CLASS::DescriptorException(session.dbc());

View File

@ -10,7 +10,7 @@ namespace DB
Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
{
Poco::URI uri{request.getURI()};
LOG_TRACE(log, "Request URI: " + uri.toString());
LOG_TRACE(log, "Request URI: {}", uri.toString());
if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
return new PingHandler(keep_alive_timeout);

View File

@ -25,7 +25,7 @@ namespace DB
void IdentifierQuoteHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
Poco::Net::HTMLForm params(request, request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
LOG_TRACE(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message)
{

View File

@ -84,7 +84,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
Poco::Net::HTMLForm params(request);
if (mode == "read")
params.read(request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
LOG_TRACE(log, "Request URI: {}", request.getURI());
if (mode == "read" && !params.has("query"))
{
@ -132,7 +132,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
std::string format = params.get("format", "RowBinary");
std::string connection_string = params.get("connection_string");
LOG_TRACE(log, "Connection string: '" << connection_string << "'");
LOG_TRACE(log, "Connection string: '{}'", connection_string);
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);
@ -152,7 +152,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
}
std::string db_name = params.get("db_name");
std::string table_name = params.get("table_name");
LOG_TRACE(log, "DB name: '" << db_name << "', table name: '" << table_name << "'");
LOG_TRACE(log, "DB name: '{}', table name: '{}'", db_name, table_name);
auto quoting_style = IdentifierQuotingStyle::None;
#if USE_ODBC
@ -171,7 +171,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
else
{
std::string query = params.get("query");
LOG_TRACE(log, "Query: " << query);
LOG_TRACE(log, "Query: {}", query);
BlockOutputStreamPtr writer = FormatFactory::instance().getOutput(format, out, *sample_block, context);
auto pool = getPool(connection_string);

View File

@ -48,12 +48,7 @@ namespace
#endif
)
{
LOG_ERROR(log,
"Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message()
<< ". "
"If it is an IPv6 address and your host has disabled IPv6, then consider to "
"specify IPv4 address to listen in <listen_host> element of configuration "
"file. Example: <listen_host>0.0.0.0</listen_host>");
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host>", host, e.code(), e.message());
}
throw;
@ -188,7 +183,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
new HandlerFactory("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), server_pool, socket, http_params);
server.start();
LOG_INFO(log, "Listening http://" + address.toString());
LOG_INFO(log, "Listening http://{}", address.toString());
SCOPE_EXIT({
LOG_DEBUG(log, "Received termination signal.");
@ -198,7 +193,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
{
if (server.currentConnections() == 0)
break;
LOG_DEBUG(log, "Waiting for " << server.currentConnections() << " connections, try " << count);
LOG_DEBUG(log, "Waiting for {} connections, try {}", server.currentConnections(), count);
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
}
});

View File

@ -241,7 +241,7 @@ void HTTPHandler::processQuery(
CurrentThread::QueryScope query_scope(context);
LOG_TRACE(log, "Request URI: " << request.getURI());
LOG_TRACE(log, "Request URI: {}", request.getURI());
std::istream & istr = request.stream();

View File

@ -30,13 +30,10 @@ HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string &
Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
{
LOG_TRACE(log, "HTTP Request for " << name << ". "
<< "Method: " << request.getMethod()
<< ", Address: " << request.clientAddress().toString()
<< ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none")
<< (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : (""))
<< ", Content Type: " << request.getContentType()
<< ", Transfer Encoding: " << request.getTransferEncoding());
LOG_TRACE(log, "HTTP Request for {}. Method: {}, Address: {}, User-Agent: {}{}, Content Type: {}, Transfer Encoding: {}",
name, request.getMethod(), request.clientAddress().toString(), request.has("User-Agent") ? request.get("User-Agent") : "none",
(request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")),
request.getContentType(), request.getTransferEncoding());
for (auto & handler_factory : child_factories)
{

View File

@ -53,7 +53,7 @@ void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & reque
{
HTMLForm params(request);
LOG_TRACE(log, "Request URI: " << request.getURI());
LOG_TRACE(log, "Request URI: {}", request.getURI());
String endpoint_name = params.get("endpoint");
bool compress = params.get("compress") == "true";
@ -103,7 +103,7 @@ void InterserverIOHTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & requ
response.setStatusAndReason(Poco::Net::HTTPServerResponse::HTTP_UNAUTHORIZED);
if (!response.sent())
writeString(message, *used_output.out);
LOG_WARNING(log, "Query processing failed request: '" << request.getURI() << "' authentication failed");
LOG_WARNING(log, "Query processing failed request: '{}' authentication failed", request.getURI());
}
}
catch (Exception & e)

View File

@ -83,19 +83,15 @@ void MySQLHandler::run()
if (!connection_context.mysql.max_packet_size)
connection_context.mysql.max_packet_size = MAX_PACKET_LENGTH;
LOG_TRACE(log, "Capabilities: " << handshake_response.capability_flags
<< ", max_packet_size: "
<< handshake_response.max_packet_size
<< ", character_set: "
<< static_cast<int>(handshake_response.character_set)
<< ", user: "
<< handshake_response.username
<< ", auth_response length: "
<< handshake_response.auth_response.length()
<< ", database: "
<< handshake_response.database
<< ", auth_plugin_name: "
<< handshake_response.auth_plugin_name);
LOG_TRACE(log,
"Capabilities: {}, max_packet_size: {}, character_set: {}, user: {}, auth_response length: {}, database: {}, auth_plugin_name: {}",
handshake_response.capability_flags,
handshake_response.max_packet_size,
static_cast<int>(handshake_response.character_set),
handshake_response.username,
handshake_response.auth_response.length(),
handshake_response.database,
handshake_response.auth_plugin_name);
client_capability_flags = handshake_response.capability_flags;
if (!(client_capability_flags & CLIENT_PROTOCOL_41))
@ -129,7 +125,9 @@ void MySQLHandler::run()
// For commands which are executed without MemoryTracker.
LimitReadBuffer limited_payload(payload, 10000, true, "too long MySQL packet.");
LOG_DEBUG(log, "Received command: " << static_cast<int>(static_cast<unsigned char>(command)) << ". Connection id: " << connection_id << ".");
LOG_DEBUG(log, "Received command: {}. Connection id: {}.",
static_cast<int>(static_cast<unsigned char>(command)), connection_id);
try
{
switch (command)
@ -197,7 +195,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::HandshakeResponse & packet)
read_bytes(3); /// We can find out whether it is SSLRequest of HandshakeResponse by first 3 bytes.
size_t payload_size = unalignedLoad<uint32_t>(buf) & 0xFFFFFFu;
LOG_TRACE(log, "payload size: " << payload_size);
LOG_TRACE(log, "payload size: {}", payload_size);
if (payload_size == SSL_REQUEST_PAYLOAD_SIZE)
{
@ -234,18 +232,18 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl
}
catch (const Exception & exc)
{
LOG_ERROR(log, "Authentication for user " << user_name << " failed.");
LOG_ERROR(log, "Authentication for user {} failed.", user_name);
packet_sender->sendPacket(ERR_Packet(exc.code(), "00000", exc.message()), true);
throw;
}
LOG_INFO(log, "Authentication for user " << user_name << " succeeded.");
LOG_INFO(log, "Authentication for user {} succeeded.", user_name);
}
void MySQLHandler::comInitDB(ReadBuffer & payload)
{
String database;
readStringUntilEOF(database, payload);
LOG_DEBUG(log, "Setting current database to " << database);
LOG_DEBUG(log, "Setting current database to {}", database);
connection_context.setCurrentDatabase(database);
packet_sender->sendPacket(OK_Packet(0, client_capability_flags, 0, 0, 1), true);
}

View File

@ -32,7 +32,7 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
}
catch (...)
{
LOG_TRACE(log, "Failed to create SSL context. SSL will be disabled. Error: " << getCurrentExceptionMessage(false));
LOG_TRACE(log, "Failed to create SSL context. SSL will be disabled. Error: {}", getCurrentExceptionMessage(false));
ssl_enabled = false;
}
@ -43,7 +43,7 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
}
catch (...)
{
LOG_TRACE(log, "Failed to read RSA key pair from server certificate. Error: " << getCurrentExceptionMessage(false));
LOG_TRACE(log, "Failed to read RSA key pair from server certificate. Error: {}", getCurrentExceptionMessage(false));
generateRSAKeys();
}
#endif
@ -122,7 +122,7 @@ void MySQLHandlerFactory::generateRSAKeys()
Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket)
{
size_t connection_id = last_connection_id++;
LOG_TRACE(log, "MySQL connection. Id: " << connection_id << ". Address: " << socket.peerAddress().toString());
LOG_TRACE(log, "MySQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString());
#if USE_SSL
return new MySQLHandlerSSL(server, socket, ssl_enabled, connection_id, *public_key, *private_key);
#else

View File

@ -91,7 +91,7 @@ namespace
void setupTmpPath(Logger * log, const std::string & path)
{
LOG_DEBUG(log, "Setting up " << path << " to store temporary data in it");
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
Poco::File(path).createDirectories();
@ -101,11 +101,11 @@ void setupTmpPath(Logger * log, const std::string & path)
{
if (it->isFile() && startsWith(it.name(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file " << it->path());
LOG_DEBUG(log, "Removing old temporary file {}", it->path());
it->remove();
}
else
LOG_DEBUG(log, "Skipped file in temporary path " << it->path());
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path());
}
}
@ -276,7 +276,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlockall(MCL_CURRENT))
LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR));
LOG_WARNING(log, "Failed mlockall: {}", errnoToString(ErrorCodes::SYSTEM_ERROR));
else
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
}
@ -284,8 +284,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep " << executable_path << "'."
" Note that it will not work on 'nosuid' mounted filesystems.");
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep {}'."
" Note that it will not work on 'nosuid' mounted filesystems.", executable_path);
}
}
}
@ -349,7 +349,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (rlim.rlim_cur == rlim.rlim_max)
{
LOG_DEBUG(log, "rlimit on number of file descriptors is " << rlim.rlim_cur);
LOG_DEBUG(log, "rlimit on number of file descriptors is {}", rlim.rlim_cur);
}
else
{
@ -357,12 +357,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
rlim.rlim_cur = config().getUInt("max_open_files", rlim.rlim_max);
int rc = setrlimit(RLIMIT_NOFILE, &rlim);
if (rc != 0)
LOG_WARNING(log,
"Cannot set max number of file descriptors to " << rlim.rlim_cur
<< ". Try to specify max_open_files according to your system limits. error: "
<< strerror(errno));
LOG_WARNING(log, "Cannot set max number of file descriptors to {}. Try to specify max_open_files according to your system limits. error: {}", rlim.rlim_cur, strerror(errno));
else
LOG_DEBUG(log, "Set max number of file descriptors to " << rlim.rlim_cur << " (was " << old << ").");
LOG_DEBUG(log, "Set max number of file descriptors to {} (was {}).", rlim.rlim_cur, old);
}
}
@ -372,7 +369,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Initialize DateLUT early, to not interfere with running time of first query.
LOG_DEBUG(log, "Initializing DateLUT.");
DateLUT::instance();
LOG_TRACE(log, "Initialized DateLUT with time zone '" << DateLUT::instance().getTimeZone() << "'.");
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
/// Storage with temporary data for processing of heavy queries.
@ -431,9 +428,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (this_host.empty())
{
this_host = getFQDNOrHostName();
LOG_DEBUG(log,
"Configuration parameter '" + String(host_tag) + "' doesn't exist or exists and empty. Will use '" + this_host
+ "' as replica host.");
LOG_DEBUG(log, "Configuration parameter '{}' doesn't exist or exists and empty. Will use '{}' as replica host.",
host_tag, this_host);
}
String port_str = config().getString(port_tag);
@ -538,8 +534,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (uncompressed_cache_size > max_cache_size)
{
uncompressed_cache_size = max_cache_size;
LOG_INFO(log, "Uncompressed cache size was lowered to " << formatReadableSizeWithBinarySuffix(uncompressed_cache_size)
<< " because the system has low amount of memory");
LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
}
global_context->setUncompressedCache(uncompressed_cache_size);
@ -554,8 +549,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (mark_cache_size > max_cache_size)
{
mark_cache_size = max_cache_size;
LOG_INFO(log, "Mark cache size was lowered to " << formatReadableSizeWithBinarySuffix(uncompressed_cache_size)
<< " because the system has low amount of memory");
LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
}
global_context->setMarkCache(mark_cache_size);
@ -579,20 +573,19 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (max_server_memory_usage == 0)
{
max_server_memory_usage = default_max_server_memory_usage;
LOG_INFO(log, "Setting max_server_memory_usage was set to " << formatReadableSizeWithBinarySuffix(max_server_memory_usage));
LOG_INFO(log, "Setting max_server_memory_usage was set to {}", formatReadableSizeWithBinarySuffix(max_server_memory_usage));
}
else if (max_server_memory_usage > default_max_server_memory_usage)
{
max_server_memory_usage = default_max_server_memory_usage;
LOG_INFO(log, "Setting max_server_memory_usage was lowered to " << formatReadableSizeWithBinarySuffix(max_server_memory_usage)
<< " because the system has low amount of memory");
LOG_INFO(log, "Setting max_server_memory_usage was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(max_server_memory_usage));
}
total_memory_tracker.setOrRaiseHardLimit(max_server_memory_usage);
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
LOG_INFO(log, "Loading metadata from " + path);
LOG_INFO(log, "Loading metadata from {}", path);
try
{
@ -694,17 +687,19 @@ int Server::main(const std::vector<std::string> & /*args*/)
" neither clickhouse-server process has CAP_NET_ADMIN capability."
" 'taskstats' performance statistics will be disabled."
" It could happen due to incorrect ClickHouse package installation."
" You can try to resolve the problem manually with 'sudo setcap cap_net_admin=+ep " << executable_path << "'."
" You can try to resolve the problem manually with 'sudo setcap cap_net_admin=+ep {}'."
" Note that it will not work on 'nosuid' mounted filesystems."
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.");
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.",
executable_path);
}
if (!hasLinuxCapability(CAP_SYS_NICE))
{
LOG_INFO(log, "It looks like the process has no CAP_SYS_NICE capability, the setting 'os_thread_nice' will have no effect."
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep " << executable_path << "'."
" Note that it will not work on 'nosuid' mounted filesystems.");
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep {}'."
" Note that it will not work on 'nosuid' mounted filesystems.",
executable_path);
}
#else
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
@ -746,11 +741,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
#endif
)
{
LOG_ERROR(log,
"Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message() << ". "
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. "
"If it is an IPv6 address and your host has disabled IPv6, then consider to "
"specify IPv4 address to listen in <listen_host> element of configuration "
"file. Example: <listen_host>0.0.0.0</listen_host>");
"file. Example: <listen_host>0.0.0.0</listen_host>",
host, e.code(), e.message());
}
throw;
@ -802,11 +797,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (listen_try)
{
LOG_ERROR(log, message
<< ". If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
LOG_ERROR(log, "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
"specify not disabled IPv4 or IPv6 address to listen in <listen_host> element of configuration "
"file. Example for disabled IPv6: <listen_host>0.0.0.0</listen_host> ."
" Example for disabled IPv4: <listen_host>::</listen_host>");
" Example for disabled IPv4: <listen_host>::</listen_host>",
message);
}
else
{
@ -826,7 +821,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for http://" + address.toString());
LOG_INFO(log, "Listening for http://{}", address.toString());
});
/// HTTPS
@ -840,7 +835,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for https://" + address.toString());
LOG_INFO(log, "Listening for https://{}", address.toString());
#else
UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
@ -861,7 +856,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with native protocol (tcp): " + address.toString());
LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString());
});
/// TCP with SSL
@ -877,7 +872,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): " + address.toString());
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): {}", address.toString());
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
@ -895,7 +890,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString());
LOG_INFO(log, "Listening for replica communication (interserver): http://{}", address.toString());
});
create_server("interserver_https_port", [&](UInt16 port)
@ -908,7 +903,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for secure replica communication (interserver): https://" + address.toString());
LOG_INFO(log, "Listening for secure replica communication (interserver): https://{}", address.toString());
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
@ -928,7 +923,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for MySQL compatibility protocol: " + address.toString());
LOG_INFO(log, "Listening for MySQL compatibility protocol: {}", address.toString());
});
/// Prometheus (if defined and not setup yet with http_port)
@ -941,7 +936,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for Prometheus: http://" + address.toString());
LOG_INFO(log, "Listening for Prometheus: http://{}", address.toString());
});
}
@ -966,12 +961,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
dns_cache_updater->start();
{
std::stringstream message;
message << "Available RAM: " << formatReadableSizeWithBinarySuffix(memory_amount) << ";"
<< " physical cores: " << getNumberOfPhysicalCPUCores() << ";"
// on ARM processors it can show only enabled at current moment cores
<< " logical cores: " << std::thread::hardware_concurrency() << ".";
LOG_INFO(log, message.str());
LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",
formatReadableSizeWithBinarySuffix(memory_amount),
getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
std::thread::hardware_concurrency());
}
LOG_INFO(log, "Ready for connections.");
@ -989,9 +982,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
current_connections += server->currentConnections();
}
LOG_INFO(log,
"Closed all listening sockets."
<< (current_connections ? " Waiting for " + toString(current_connections) + " outstanding connections." : ""));
if (current_connections)
LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections);
else
LOG_INFO(log, "Closed all listening sockets.");
/// Killing remaining queries.
global_context->getProcessList().killAllQueries();
@ -1013,9 +1007,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
}
LOG_INFO(
log, "Closed connections." << (current_connections ? " But " + toString(current_connections) + " remains."
" Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>" : ""));
if (current_connections)
LOG_INFO(log, "Closed connections. But {} remain."
" Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>", current_connections);
else
LOG_INFO(log, "Closed connections.");
dns_cache_updater.reset();
main_config_reloader.reset();

View File

@ -28,7 +28,7 @@
#include <Compression/CompressionFactory.h>
#include <common/logger_useful.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include "TCPHandler.h"
@ -115,8 +115,7 @@ void TCPHandler::runImpl()
if (!DatabaseCatalog::instance().isDatabaseExist(default_database))
{
Exception e("Database " + backQuote(default_database) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText()
<< ", Stack trace:\n\n" << e.getStackTraceString());
LOG_ERROR(log, "Code: {}, e.displayText() = {}, Stack trace:\n\n{}", e.code(), e.displayText(), e.getStackTraceString());
sendException(e, connection_context.getSettingsRef().calculate_text_stack_trace);
return;
}
@ -278,8 +277,11 @@ void TCPHandler::runImpl()
sendLogs();
sendEndOfStream();
query_scope.reset();
/// QueryState should be cleared before QueryScope, since otherwise
/// the MemoryTracker will be wrong for possible deallocations.
/// (i.e. deallocations from the Aggregator with two-level aggregation)
state.reset();
query_scope.reset();
}
catch (const Exception & e)
{
@ -359,8 +361,11 @@ void TCPHandler::runImpl()
try
{
query_scope.reset();
/// QueryState should be cleared before QueryScope, since otherwise
/// the MemoryTracker will be wrong for possible deallocations.
/// (i.e. deallocations from the Aggregator with two-level aggregation)
state.reset();
query_scope.reset();
}
catch (...)
{
@ -373,8 +378,7 @@ void TCPHandler::runImpl()
watch.stop();
LOG_INFO(log, std::fixed << std::setprecision(3)
<< "Processed in " << watch.elapsedSeconds() << " sec.");
LOG_INFO(log, "Processed in {} sec.", watch.elapsedSeconds());
/// It is important to destroy query context here. We do not want it to live arbitrarily longer than the query.
query_context.reset();
@ -560,7 +564,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
}
{
PullingPipelineExecutor executor(pipeline);
PullingAsyncPipelineExecutor executor(pipeline);
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
Block block;
@ -726,14 +730,12 @@ void TCPHandler::receiveHello()
readStringBinary(user, *in);
readStringBinary(password, *in);
LOG_DEBUG(log, "Connected " << client_name
<< " version " << client_version_major
<< "." << client_version_minor
<< "." << client_version_patch
<< ", revision: " << client_revision
<< (!default_database.empty() ? ", database: " + default_database : "")
<< (!user.empty() ? ", user: " + user : "")
<< ".");
LOG_DEBUG(log, "Connected {} version {}.{}.{}, revision: {}{}{}.",
client_name,
client_version_major, client_version_minor, client_version_patch,
client_revision,
(!default_database.empty() ? ", database: " + default_database : ""),
(!user.empty() ? ", user: " + user : ""));
connection_context.setUser(user, password, socket().peerAddress());
}
@ -1199,8 +1201,7 @@ void TCPHandler::run()
/// Timeout - not an error.
if (!strcmp(e.what(), "Timeout"))
{
LOG_DEBUG(log, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
LOG_DEBUG(log, "Poco::Exception. Code: {}, e.code() = {}, e.displayText() = {}, e.what() = {}", ErrorCodes::POCO_EXCEPTION, e.code(), e.displayText(), e.what());
}
else
throw;

View File

@ -35,7 +35,7 @@ public:
{
try
{
LOG_TRACE(log, "TCP Request. Address: " << socket.peerAddress().toString());
LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString());
return new TCPHandler(server, socket);
}
catch (const Poco::Net::NetException &)

View File

@ -227,7 +227,7 @@
and to prevent clickhouse executable from being paged out under high IO load.
Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
-->
<mlock_executable>false</mlock_executable>
<mlock_executable>true</mlock_executable>
<!-- Configuration of clusters that could be used in Distributed tables.
https://clickhouse.tech/docs/en/operations/table_engines/distributed/

View File

@ -251,12 +251,11 @@ public:
void logTree(Poco::Logger * log) const
{
LOG_TRACE(log, "Tree(" << level << "): name=" << (node_name ? *node_name : "NULL")
<< ", access=" << access.toString()
<< ", final_access=" << final_access.toString()
<< ", min_access=" << min_access.toString()
<< ", max_access=" << max_access.toString()
<< ", num_children=" << (children ? children->size() : 0));
LOG_TRACE(log, "Tree({}): name={}, access={}, final_access={}, min_access={}, max_access={}, num_children={}",
level, node_name ? *node_name : "NULL", access.toString(),
final_access.toString(), min_access.toString(), max_access.toString(),
(children ? children->size() : 0));
if (children)
{
for (auto & child : *children | boost::adaptors::map_values)

View File

@ -310,8 +310,8 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
&Logger::get("AddressPatterns"),
"Failed to check if the allowed client hosts contain address " << client_address.toString() << ". " << e.displayText()
<< ", code = " << e.code());
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
client_address.toString(), e.displayText(), e.code());
return false;
}
};
@ -343,8 +343,8 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
&Logger::get("AddressPatterns"),
"Failed to check if the allowed client hosts contain address " << client_address.toString() << ". " << e.displayText()
<< ", code = " << e.code());
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
client_address.toString(), e.displayText(), e.code());
return false;
}
};

View File

@ -200,7 +200,7 @@ bool ContextAccess::calculateResultAccessAndCheck(Poco::Logger * log_, const Acc
bool is_granted = access->isGranted(flags, args...);
if (trace_log)
LOG_TRACE(trace_log, "Access " << (is_granted ? "granted" : "denied") << ": " << (AccessRightsElement{flags, args...}.toString()));
LOG_TRACE(trace_log, "Access {}: {}", (is_granted ? "granted" : "denied"), (AccessRightsElement{flags, args...}.toString()));
if (is_granted)
return true;
@ -219,7 +219,7 @@ bool ContextAccess::calculateResultAccessAndCheck(Poco::Logger * log_, const Acc
if constexpr (mode == THROW_IF_ACCESS_DENIED)
throw Exception(user_name + ": " + msg, error_code);
else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
LOG_WARNING(log_, user_name + ": " + msg + formatSkippedMessage(args...));
LOG_WARNING(log_, "{}: {}{}", user_name, msg, formatSkippedMessage(args...));
};
if (!user)
@ -451,15 +451,18 @@ boost::shared_ptr<const AccessRights> ContextAccess::calculateResultAccess(bool
if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_))
{
LOG_TRACE(trace_log, "List of all grants: " << merged_access->toString() << (grant_option ? " WITH GRANT OPTION" : ""));
if (grant_option)
LOG_TRACE(trace_log, "List of all grants: {} WITH GRANT OPTION", merged_access->toString());
else
LOG_TRACE(trace_log, "List of all grants: {}", merged_access->toString());
if (roles_info && !roles_info->getCurrentRolesNames().empty())
{
LOG_TRACE(
trace_log,
"Current_roles: " << boost::algorithm::join(roles_info->getCurrentRolesNames(), ", ")
<< ", enabled_roles: " << boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
LOG_TRACE(trace_log, "Current_roles: {}, enabled_roles: {}",
boost::algorithm::join(roles_info->getCurrentRolesNames(), ", "),
boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
}
LOG_TRACE(trace_log, "Settings: readonly=" << readonly_ << ", allow_ddl=" << allow_ddl_ << ", allow_introspection_functions=" << allow_introspection_);
LOG_TRACE(trace_log, "Settings: readonly={}, allow_ddl={}, allow_introspection_functions={}", readonly_, allow_ddl_, allow_introspection_);
}
res = std::move(merged_access);

View File

@ -367,7 +367,7 @@ bool DiskAccessStorage::readLists()
auto file_path = getListFilePath(directory_path, type);
if (!std::filesystem::exists(file_path))
{
LOG_WARNING(getLogger(), "File " + file_path.string() + " doesn't exist");
LOG_WARNING(getLogger(), "File {} doesn't exist", file_path.string());
ok = false;
break;
}
@ -496,7 +496,7 @@ void DiskAccessStorage::listsWritingThreadFunc()
/// and then saves the files "users.list", "roles.list", etc. to the same directory.
bool DiskAccessStorage::rebuildLists()
{
LOG_WARNING(getLogger(), "Recovering lists in directory " + directory_path);
LOG_WARNING(getLogger(), "Recovering lists in directory {}", directory_path);
clear();
for (const auto & directory_entry : std::filesystem::directory_iterator(directory_path))

View File

@ -103,6 +103,7 @@ class QuantileTDigest
struct RadixSortTraits
{
using Element = Centroid;
using Result = Element;
using Key = Value;
using CountType = UInt32;
using KeyBits = UInt32;
@ -114,6 +115,7 @@ class QuantileTDigest
/// The function to get the key from an array element.
static Key & extractKey(Element & elem) { return elem.mean; }
static Result & extractResult(Element & elem) { return elem; }
};
/** Adds a centroid `c` to the digest

View File

@ -61,11 +61,11 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
if (connected)
disconnect();
LOG_TRACE(log_wrapper.get(), "Connecting. Database: "
<< (default_database.empty() ? "(not specified)" : default_database)
<< ". User: " << user
<< (static_cast<bool>(secure) ? ". Secure" : "")
<< (static_cast<bool>(compression) ? "" : ". Uncompressed"));
LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}",
default_database.empty() ? "(not specified)" : default_database,
user,
static_cast<bool>(secure) ? ". Secure" : "",
static_cast<bool>(compression) ? "" : ". Uncompressed");
if (static_cast<bool>(secure))
{
@ -107,11 +107,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
sendHello();
receiveHello();
LOG_TRACE(log_wrapper.get(), "Connected to " << server_name
<< " server version " << server_version_major
<< "." << server_version_minor
<< "." << server_version_patch
<< ".");
LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.",
server_name, server_version_major, server_version_minor, server_version_patch);
}
catch (Poco::Net::NetException & e)
{
@ -132,8 +129,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
void Connection::disconnect()
{
//LOG_TRACE(log_wrapper.get(), "Disconnecting");
in = nullptr;
last_input_packet_type.reset();
out = nullptr; // can write to socket
@ -186,8 +181,6 @@ void Connection::sendHello()
void Connection::receiveHello()
{
//LOG_TRACE(log_wrapper.get(), "Receiving hello");
/// Receive hello packet.
UInt64 packet_type = 0;
@ -391,8 +384,6 @@ void Connection::sendQuery(
query_id = query_id_;
//LOG_TRACE(log_wrapper.get(), "Sending query");
writeVarUInt(Protocol::Client::Query, *out);
writeStringBinary(query_id, *out);
@ -441,8 +432,6 @@ void Connection::sendCancel()
if (!out)
return;
//LOG_TRACE(log_wrapper.get(), "Sending cancel");
writeVarUInt(Protocol::Client::Cancel, *out);
out->next();
}
@ -450,8 +439,6 @@ void Connection::sendCancel()
void Connection::sendData(const Block & block, const String & name, bool scalar)
{
//LOG_TRACE(log_wrapper.get(), "Sending data");
if (!block_out)
{
if (compression == Protocol::Compression::Enable)
@ -516,19 +503,23 @@ void Connection::sendScalarsData(Scalars & data)
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
double elapsed = watch.elapsedSeconds();
std::stringstream msg;
msg << std::fixed << std::setprecision(3);
msg << "Sent data for " << data.size() << " scalars, total " << rows << " rows in " << elapsed << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
if (compression == Protocol::Compression::Enable)
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
LOG_DEBUG(log_wrapper.get(),
"Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)",
data.size(), rows, elapsed,
static_cast<size_t>(rows / watch.elapsedSeconds()),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()),
static_cast<double>(maybe_compressed_out_bytes) / out_bytes,
formatReadableSizeWithBinarySuffix(out_bytes),
formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds()));
else
msg << ", no compression.";
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
LOG_DEBUG(log_wrapper.get(),
"Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.",
data.size(), rows, elapsed,
static_cast<size_t>(rows / watch.elapsedSeconds()),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()));
}
namespace
@ -616,19 +607,23 @@ void Connection::sendExternalTablesData(ExternalTablesData & data)
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
double elapsed = watch.elapsedSeconds();
std::stringstream msg;
msg << std::fixed << std::setprecision(3);
msg << "Sent data for " << data.size() << " external tables, total " << rows << " rows in " << elapsed << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
if (compression == Protocol::Compression::Enable)
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
LOG_DEBUG(log_wrapper.get(),
"Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)",
data.size(), rows, elapsed,
static_cast<size_t>(rows / watch.elapsedSeconds()),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()),
static_cast<double>(maybe_compressed_out_bytes) / out_bytes,
formatReadableSizeWithBinarySuffix(out_bytes),
formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds()));
else
msg << ", no compression.";
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
LOG_DEBUG(log_wrapper.get(),
"Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.",
data.size(), rows, elapsed,
static_cast<size_t>(rows / watch.elapsedSeconds()),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()));
}
std::optional<Poco::Net::SocketAddress> Connection::getResolvedAddress() const
@ -682,12 +677,9 @@ Packet Connection::receivePacket()
}
else
{
//LOG_TRACE(log_wrapper.get(), "Receiving packet type");
readVarUInt(res.type, *in);
}
//LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type));
//std::cerr << "Client got packet: " << Protocol::Server::toString(res.type) << "\n";
switch (res.type)
{
case Protocol::Server::Data: [[fallthrough]];
@ -740,8 +732,6 @@ Packet Connection::receivePacket()
Block Connection::receiveData()
{
//LOG_TRACE(log_wrapper.get(), "Receiving data");
initBlockInput();
return receiveDataImpl(block_in);
}
@ -820,8 +810,6 @@ void Connection::setDescription()
std::unique_ptr<Exception> Connection::receiveException()
{
//LOG_TRACE(log_wrapper.get(), "Receiving exception");
return std::make_unique<Exception>(readException(*in, "Received from " + getDescription()));
}
@ -838,8 +826,6 @@ std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
Progress Connection::receiveProgress()
{
//LOG_TRACE(log_wrapper.get(), "Receiving progress");
Progress progress;
progress.read(*in, server_revision);
return progress;

View File

@ -50,6 +50,8 @@ class Connection;
using ConnectionPtr = std::shared_ptr<Connection>;
using Connections = std::vector<ConnectionPtr>;
using Scalars = std::map<String, Block>;
/// Packet that could be received from server.
struct Packet

View File

@ -222,8 +222,8 @@ ConnectionPoolWithFailover::tryGetEntry(
auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
if (table_status_it == status_response.table_states_by_id.end())
{
fail_message = "There is no table " + table_to_check->database + "." + table_to_check->table
+ " on server: " + result.entry->getDescription();
const char * message_pattern = "There is no table {}.{} on server: {}";
fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
LOG_WARNING(log, fail_message);
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
@ -248,10 +248,7 @@ ConnectionPoolWithFailover::tryGetEntry(
result.is_up_to_date = false;
result.staleness = delay;
LOG_TRACE(
log, "Server " << result.entry->getDescription() << " has unacceptable replica delay "
<< "for table " << table_to_check->database << "." << table_to_check->table
<< ": " << delay);
LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay);
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
}
}

View File

@ -35,7 +35,7 @@ TimeoutSetter::~TimeoutSetter()
catch (std::exception & e)
{
// Sometimes catched on macos
LOG_ERROR(&Logger::get("Client"), std::string{"TimeoutSetter: Can't reset timeouts: "} + e.what());
LOG_ERROR(&Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what());
}
}
}

View File

@ -17,6 +17,7 @@
#include <DataStreams/ColumnGathererStream.h>
#include <ext/bit_cast.h>
#include <pdqsort.h>
#include <numeric>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
@ -117,7 +118,10 @@ namespace
struct RadixSortTraits : RadixSortNumTraits<T>
{
using Element = ValueWithIndex<T>;
using Result = size_t;
static T & extractKey(Element & elem) { return elem.value; }
static size_t extractResult(Element & elem) { return elem.index; }
};
}
@ -179,53 +183,27 @@ void ColumnVector<T>::getPermutation(bool reverse, size_t limit, int nan_directi
for (UInt32 i = 0; i < UInt32(s); ++i)
pairs[i] = {data[i], i};
RadixSort<RadixSortTraits<T>>::executeLSD(pairs.data(), s);
RadixSort<RadixSortTraits<T>>::executeLSD(pairs.data(), s, reverse, res.data());
/// Radix sort treats all NaNs to be greater than all numbers.
/// If the user needs the opposite, we must move them accordingly.
size_t nans_to_move = 0;
if (std::is_floating_point_v<T> && nan_direction_hint < 0)
{
for (ssize_t i = s - 1; i >= 0; --i)
size_t nans_to_move = 0;
for (size_t i = 0; i < s; ++i)
{
if (isNaN(pairs[i].value))
if (isNaN(data[res[reverse ? i : s - 1 - i]]))
++nans_to_move;
else
break;
}
}
if (reverse)
{
if (nans_to_move)
{
for (size_t i = 0; i < s - nans_to_move; ++i)
res[i] = pairs[s - nans_to_move - 1 - i].index;
for (size_t i = s - nans_to_move; i < s; ++i)
res[i] = pairs[s - 1 - (i - (s - nans_to_move))].index;
}
else
{
for (size_t i = 0; i < s; ++i)
res[s - 1 - i] = pairs[i].index;
std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : s - nans_to_move), std::end(res));
}
}
else
{
if (nans_to_move)
{
for (size_t i = 0; i < nans_to_move; ++i)
res[i] = pairs[i + s - nans_to_move].index;
for (size_t i = nans_to_move; i < s; ++i)
res[i] = pairs[i - nans_to_move].index;
}
else
{
for (size_t i = 0; i < s; ++i)
res[i] = pairs[i].index;
}
}
return;
}
}

View File

@ -5,6 +5,7 @@
#include <cstdlib>
#include <cstring>
#include <algorithm>
#include <sstream>
#include <iostream>
#include <functional>
#include <Poco/DOM/Text.h>
@ -303,7 +304,7 @@ void ConfigProcessor::doIncludesRecursive(
else if (throw_on_bad_incl)
throw Poco::Exception(error_msg + name);
else
LOG_WARNING(log, error_msg << name);
LOG_WARNING(log, "{}{}", error_msg, name);
}
else
{
@ -440,7 +441,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event)
{
LOG_DEBUG(log, "Processing configuration file '" + path + "'.");
LOG_DEBUG(log, "Processing configuration file '{}'.", path);
XMLDocumentPtr config = dom_parser.parse(path);
@ -451,7 +452,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
{
try
{
LOG_DEBUG(log, "Merging configuration file '" + merge_file + "'.");
LOG_DEBUG(log, "Merging configuration file '{}'.", merge_file);
XMLDocumentPtr with = dom_parser.parse(merge_file);
merge(config, with);
@ -488,7 +489,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
}
if (!include_from_path.empty())
{
LOG_DEBUG(log, "Including configuration file '" + include_from_path + "'.");
LOG_DEBUG(log, "Including configuration file '{}'.", include_from_path);
contributing_files.push_back(include_from_path);
include_from = dom_parser.parse(include_from_path);
@ -568,10 +569,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
if (!zk_exception)
throw;
LOG_WARNING(
log,
"Error while processing from_zk config includes: " + zk_exception->message() +
". Config will be loaded from preprocessed file: " + preprocessed_path);
LOG_WARNING(log, "Error while processing from_zk config includes: {}. Config will be loaded from preprocessed file: {}", zk_exception->message(), preprocessed_path);
config_xml = dom_parser.parse(preprocessed_path);
}
@ -619,11 +617,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
Poco::File(preprocessed_path_parent).createDirectories();
}
DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml);
LOG_DEBUG(log, "Saved preprocessed configuration to '" << preprocessed_path << "'.");
LOG_DEBUG(log, "Saved preprocessed configuration to '{}'.", preprocessed_path);
}
catch (Poco::Exception & e)
{
LOG_WARNING(log, "Couldn't save preprocessed config to " << preprocessed_path << ": " << e.displayText());
LOG_WARNING(log, "Couldn't save preprocessed config to {}: {}", preprocessed_path, e.displayText());
}
}

View File

@ -87,7 +87,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
ConfigProcessor::LoadedConfig loaded_config;
try
{
LOG_DEBUG(log, "Loading config '" << path << "'");
LOG_DEBUG(log, "Loading config '{}'", path);
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (loaded_config.has_zk_includes)

View File

@ -202,7 +202,7 @@ bool DNSResolver::updateCache()
}
if (!lost_hosts.empty())
LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: " << lost_hosts);
LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts);
return updated;
}

View File

@ -36,7 +36,7 @@ Exception::Exception(const std::string & msg, int code)
#ifndef NDEBUG
if (code == ErrorCodes::LOGICAL_ERROR)
{
LOG_ERROR(&Poco::Logger::root(), "Logical error: '" + msg + "'.");
LOG_ERROR(&Poco::Logger::root(), "Logical error: '{}'.", msg);
assert(false);
}
#endif
@ -125,7 +125,10 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
{
try
{
LOG_ERROR(logger, start_of_message << (start_of_message.empty() ? "" : ": ") << getCurrentExceptionMessage(true));
if (start_of_message.empty())
LOG_ERROR(logger, "{}", getCurrentExceptionMessage(true));
else
LOG_ERROR(logger, "{}: {}", start_of_message, getCurrentExceptionMessage(true));
}
catch (...)
{

View File

@ -30,7 +30,7 @@ template
typename Cell,
typename Hash,
typename Grower,
typename Allocator, /// TODO WithStackMemory
typename Allocator,
typename ImplTable = HashTable<Key, Cell, Hash, Grower, Allocator>,
size_t BITS_FOR_BUCKET = 8
>

View File

@ -49,16 +49,12 @@ MemoryTracker::~MemoryTracker()
void MemoryTracker::logPeakMemoryUsage() const
{
LOG_DEBUG(&Logger::get("MemoryTracker"),
"Peak memory usage" << (description ? " " + std::string(description) : "")
<< ": " << formatReadableSizeWithBinarySuffix(peak) << ".");
LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(peak));
}
void MemoryTracker::logMemoryUsage(Int64 current) const
{
LOG_DEBUG(&Logger::get("MemoryTracker"),
"Current memory usage" << (description ? " " + std::string(description) : "")
<< ": " << formatReadableSizeWithBinarySuffix(current) << ".");
LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(current));
}

View File

@ -90,7 +90,7 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size)
{
if (errno == EINVAL)
{
LOG_INFO(log, "Cannot get pipe capacity, " << errnoToString(ErrorCodes::CANNOT_FCNTL) << ". Very old Linux kernels have no support for this fcntl.");
LOG_INFO(log, "Cannot get pipe capacity, {}. Very old Linux kernels have no support for this fcntl.", errnoToString(ErrorCodes::CANNOT_FCNTL));
/// It will work nevertheless.
}
else
@ -102,7 +102,7 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size)
if (-1 == fcntl(fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM)
throwFromErrno("Cannot increase pipe capacity to " + std::to_string(pipe_size * 2), ErrorCodes::CANNOT_FCNTL);
LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size)));
LOG_TRACE(log, "Pipe capacity is {}", formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size)));
}
#else
(void)desired_size;

View File

@ -239,8 +239,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
}
else
{
LOG_WARNING(log, "Connection failed at try №"
<< (shuffled_pool.error_count + 1) << ", reason: " << fail_message);
LOG_WARNING(log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message);
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry);
shuffled_pool.error_count = std::min(max_error_cap, shuffled_pool.error_count + 1);

View File

@ -8,6 +8,9 @@
M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \
M(SelectQuery, "Same as Query, but only for SELECT queries.") \
M(InsertQuery, "Same as Query, but only for INSERT queries.") \
M(FailedQuery, "Number of failed queries.") \
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \
M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \
M(FileOpen, "Number of files opened.") \
M(Seek, "Number of times the 'lseek' function was called.") \
M(ReadBufferFromFileDescriptorRead, "Number of reads (read/pread) from a file descriptor. Does not include sockets.") \

View File

@ -165,10 +165,10 @@ void QueryProfilerBase<ProfilerImpl>::tryCleanup()
{
#if USE_UNWIND
if (timer_id != nullptr && timer_delete(timer_id))
LOG_ERROR(log, "Failed to delete query profiler timer " + errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
LOG_ERROR(log, "Failed to delete query profiler timer {}", errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
if (previous_handler != nullptr && sigaction(pause_signal, previous_handler, nullptr))
LOG_ERROR(log, "Failed to restore signal handler after query profiler " + errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
LOG_ERROR(log, "Failed to restore signal handler after query profiler {}", errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
#endif
}

View File

@ -9,6 +9,7 @@
#include <cmath>
#include <cstdlib>
#include <cstdint>
#include <cassert>
#include <type_traits>
#include <ext/bit_cast.h>
@ -17,13 +18,17 @@
/** Radix sort, has the following functionality:
*
* Can sort unsigned, signed numbers, and floats.
* Can sort an array of fixed length elements that contain something else besides the key.
* Can sort an array and form sorted result containing some transformation of elements.
* Can do partial sort.
* Customizable radix size.
*
* LSB, stable.
* NOTE For some applications it makes sense to add MSB-radix-sort,
* as well as radix-select, radix-partial-sort, radix-get-permutation algorithms based on it.
* Two flavours of radix sort are implemented:
*
* 1. LSB, stable.
* 2. MSB, unstable, with support for partial sort.
*/
@ -70,9 +75,18 @@ struct RadixSortFloatTransform
template <typename TElement>
struct RadixSortFloatTraits
{
using Element = TElement; /// The type of the element. It can be a structure with a key and some other payload. Or just a key.
using Key = Element; /// The key to sort by.
using CountType = uint32_t; /// Type for calculating histograms. In the case of a known small number of elements, it can be less than size_t.
/// The type of the element. It can be a structure with a key and some other payload. Or just a key.
using Element = TElement;
/// The key to sort by.
using Key = Element;
/// Part of the element that you need in the result array.
/// There are cases when elements are sorted by one part but you need other parts in array of results.
using Result = Element;
/// Type for calculating histograms. In the case of a known small number of elements, it can be less than size_t.
using CountType = uint32_t;
/// The type to which the key is transformed to do bit operations. This UInt is the same size as the key.
using KeyBits = std::conditional_t<sizeof(Key) == 8, uint64_t, uint32_t>;
@ -90,6 +104,9 @@ struct RadixSortFloatTraits
/// The function to get the key from an array element.
static Key & extractKey(Element & elem) { return elem; }
/// The function to get the result part from an array element.
static Result & extractResult(Element & elem) { return elem; }
/// Used when fallback to comparison based sorting is needed.
/// TODO: Correct handling of NaNs, NULLs, etc
static bool less(Key x, Key y)
@ -113,6 +130,7 @@ template <typename TElement>
struct RadixSortUIntTraits
{
using Element = TElement;
using Result = Element;
using Key = Element;
using CountType = uint32_t;
using KeyBits = Key;
@ -123,6 +141,7 @@ struct RadixSortUIntTraits
using Allocator = RadixSortMallocAllocator;
static Key & extractKey(Element & elem) { return elem; }
static Result & extractResult(Element & elem) { return elem; }
static bool less(Key x, Key y)
{
@ -145,6 +164,7 @@ template <typename TElement>
struct RadixSortIntTraits
{
using Element = TElement;
using Result = Element;
using Key = Element;
using CountType = uint32_t;
using KeyBits = std::make_unsigned_t<Key>;
@ -155,6 +175,7 @@ struct RadixSortIntTraits
using Allocator = RadixSortMallocAllocator;
static Key & extractKey(Element & elem) { return elem; }
static Result & extractResult(Element & elem) { return elem; }
static bool less(Key x, Key y)
{
@ -175,6 +196,7 @@ struct RadixSort
{
private:
using Element = typename Traits::Element;
using Result = typename Traits::Result;
using Key = typename Traits::Key;
using CountType = typename Traits::CountType;
using KeyBits = typename Traits::KeyBits;
@ -187,6 +209,10 @@ private:
static constexpr size_t KEY_BITS = sizeof(Key) * 8;
static constexpr size_t NUM_PASSES = (KEY_BITS + (Traits::PART_SIZE_BITS - 1)) / Traits::PART_SIZE_BITS;
static KeyBits keyToBits(Key x) { return ext::bit_cast<KeyBits>(x); }
static Key bitsToKey(KeyBits x) { return ext::bit_cast<Key>(x); }
static ALWAYS_INLINE KeyBits getPart(size_t N, KeyBits x)
{
if (Traits::Transform::transform_is_simple)
@ -195,10 +221,12 @@ private:
return (x >> (N * Traits::PART_SIZE_BITS)) & PART_BITMASK;
}
static KeyBits keyToBits(Key x) { return ext::bit_cast<KeyBits>(x); }
static Key bitsToKey(KeyBits x) { return ext::bit_cast<Key>(x); }
static ALWAYS_INLINE KeyBits extractPart(size_t N, Element & elem)
{
return getPart(N, keyToBits(Traits::extractKey(elem)));
}
static void insertionSortInternal(Element *arr, size_t size)
static void insertionSortInternal(Element * arr, size_t size)
{
Element * end = arr + size;
for (Element * i = arr + 1; i < end; ++i)
@ -215,96 +243,9 @@ private:
}
}
/* Main MSD radix sort subroutine
* Puts elements to buckets based on PASS-th digit, then recursively calls insertion sort or itself on the buckets
*/
template <size_t PASS>
static inline void radixSortMSDInternal(Element * arr, size_t size, size_t limit)
{
Element * last_list[HISTOGRAM_SIZE + 1];
Element ** last = last_list + 1;
size_t count[HISTOGRAM_SIZE] = {0};
for (Element * i = arr; i < arr + size; ++i)
++count[getPart(PASS, *i)];
last_list[0] = last_list[1] = arr;
size_t buckets_for_recursion = HISTOGRAM_SIZE;
Element * finish = arr + size;
for (size_t i = 1; i < HISTOGRAM_SIZE; ++i)
{
last[i] = last[i - 1] + count[i - 1];
if (last[i] >= arr + limit)
{
buckets_for_recursion = i;
finish = last[i];
}
}
/* At this point, we have the following variables:
* count[i] is the size of i-th bucket
* last[i] is a pointer to the beginning of i-th bucket, last[-1] == last[0]
* buckets_for_recursion is the number of buckets that should be sorted, the last of them only partially
* finish is a pointer to the end of the first buckets_for_recursion buckets
*/
// Scatter array elements to buckets until the first buckets_for_recursion buckets are full
for (size_t i = 0; i < buckets_for_recursion; ++i)
{
Element * end = last[i - 1] + count[i];
if (end == finish)
{
last[i] = end;
break;
}
while (last[i] != end)
{
Element swapper = *last[i];
KeyBits tag = getPart(PASS, swapper);
if (tag != i)
{
do
{
std::swap(swapper, *last[tag]++);
} while ((tag = getPart(PASS, swapper)) != i);
*last[i] = swapper;
}
++last[i];
}
}
if constexpr (PASS > 0)
{
// Recursively sort buckets, except the last one
for (size_t i = 0; i < buckets_for_recursion - 1; ++i)
{
Element * start = last[i - 1];
size_t subsize = last[i] - last[i - 1];
radixSortMSDInternalHelper<PASS - 1>(start, subsize, subsize);
}
// Sort last necessary bucket with limit
Element * start = last[buckets_for_recursion - 2];
size_t subsize = last[buckets_for_recursion - 1] - last[buckets_for_recursion - 2];
size_t sublimit = limit - (last[buckets_for_recursion - 1] - arr);
radixSortMSDInternalHelper<PASS - 1>(start, subsize, sublimit);
}
}
// A helper to choose sorting algorithm based on array length
template <size_t PASS>
static inline void radixSortMSDInternalHelper(Element * arr, size_t size, size_t limit)
{
if (size <= INSERTION_SORT_THRESHOLD)
insertionSortInternal(arr, size);
else
radixSortMSDInternal<PASS>(arr, size, limit);
}
public:
/// Least significant digit radix sort (stable)
static void executeLSD(Element * arr, size_t size)
template <bool DIRECT_WRITE_TO_DESTINATION>
static NO_INLINE void radixSortLSDInternal(Element * arr, size_t size, bool reverse, Result * destination)
{
/// If the array is smaller than 256, then it is better to use another algorithm.
@ -326,7 +267,7 @@ public:
Traits::extractKey(arr[i]) = bitsToKey(Traits::Transform::forward(keyToBits(Traits::extractKey(arr[i]))));
for (size_t pass = 0; pass < NUM_PASSES; ++pass)
++histograms[pass * HISTOGRAM_SIZE + getPart(pass, keyToBits(Traits::extractKey(arr[i])))];
++histograms[pass * HISTOGRAM_SIZE + extractPart(pass, arr[i])];
}
{
@ -345,14 +286,14 @@ public:
}
/// Move the elements in the order starting from the least bit piece, and then do a few passes on the number of pieces.
for (size_t pass = 0; pass < NUM_PASSES; ++pass)
for (size_t pass = 0; pass < NUM_PASSES - DIRECT_WRITE_TO_DESTINATION; ++pass)
{
Element * writer = pass % 2 ? arr : swap_buffer;
Element * reader = pass % 2 ? swap_buffer : arr;
for (size_t i = 0; i < size; ++i)
{
size_t pos = getPart(pass, keyToBits(Traits::extractKey(reader[i])));
size_t pos = extractPart(pass, reader[i]);
/// Place the element on the next free position.
auto & dest = writer[++histograms[pass * HISTOGRAM_SIZE + pos]];
@ -364,16 +305,235 @@ public:
}
}
/// If the number of passes is odd, the result array is in a temporary buffer. Copy it to the place of the original array.
/// NOTE Sometimes it will be more optimal to provide non-destructive interface, that will not modify original array.
if (NUM_PASSES % 2)
memcpy(arr, swap_buffer, size * sizeof(Element));
if (DIRECT_WRITE_TO_DESTINATION)
{
constexpr size_t pass = NUM_PASSES - 1;
Result * writer = destination;
Element * reader = pass % 2 ? swap_buffer : arr;
if (reverse)
{
for (size_t i = 0; i < size; ++i)
{
size_t pos = extractPart(pass, reader[i]);
writer[size - 1 - (++histograms[pass * HISTOGRAM_SIZE + pos])] = Traits::extractResult(reader[i]);
}
}
else
{
for (size_t i = 0; i < size; ++i)
{
size_t pos = extractPart(pass, reader[i]);
writer[++histograms[pass * HISTOGRAM_SIZE + pos]] = Traits::extractResult(reader[i]);
}
}
}
else
{
/// If the number of passes is odd, the result array is in a temporary buffer. Copy it to the place of the original array.
if (NUM_PASSES % 2)
memcpy(arr, swap_buffer, size * sizeof(Element));
/// TODO This is suboptimal, we can embed it to the last pass.
if (reverse)
std::reverse(arr, arr + size);
}
allocator.deallocate(swap_buffer, size * sizeof(Element));
}
/* Main MSD radix sort subroutine.
* Puts elements to buckets based on PASS-th digit, then recursively calls insertion sort or itself on the buckets.
*
* TODO: Provide support for 'reverse' and 'DIRECT_WRITE_TO_DESTINATION'.
*
* Invariant: higher significant parts of the elements than PASS are constant within arr or is is the first PASS.
* PASS is counted from least significant (0), so the first pass is NUM_PASSES - 1.
*/
template <size_t PASS>
static inline void radixSortMSDInternal(Element * arr, size_t size, size_t limit)
{
// std::cerr << PASS << ", " << size << ", " << limit << "\n";
/// The beginning of every i-1-th bucket. 0th element will be equal to 1st.
/// Last element will point to array end.
Element * prev_buckets[HISTOGRAM_SIZE + 1];
/// The beginning of every i-th bucket (the same array shifted by one).
Element ** buckets = &prev_buckets[1];
prev_buckets[0] = arr;
prev_buckets[1] = arr;
/// The end of the range of buckets that we need with limit.
Element * finish = arr + size;
/// Count histogram of current element parts.
/// We use loop unrolling to minimize data dependencies and increase instruction level parallelism.
/// Unroll 8 times looks better on experiments;
/// also it corresponds with the results from https://github.com/powturbo/TurboHist
static constexpr size_t UNROLL_COUNT = 8;
CountType count[HISTOGRAM_SIZE * UNROLL_COUNT]{};
size_t unrolled_size = size / UNROLL_COUNT * UNROLL_COUNT;
for (Element * elem = arr; elem < arr + unrolled_size; elem += UNROLL_COUNT)
for (size_t i = 0; i < UNROLL_COUNT; ++i)
++count[i * HISTOGRAM_SIZE + extractPart(PASS, elem[i])];
for (Element * elem = arr + unrolled_size; elem < arr + size; ++elem)
++count[extractPart(PASS, *elem)];
for (size_t i = 0; i < HISTOGRAM_SIZE; ++i)
for (size_t j = 1; j < UNROLL_COUNT; ++j)
count[i] += count[j * HISTOGRAM_SIZE + i];
/// Fill pointers to buckets according to the histogram.
/// How many buckets we will recurse into.
ssize_t buckets_for_recursion = HISTOGRAM_SIZE;
bool finish_early = false;
for (size_t i = 1; i < HISTOGRAM_SIZE; ++i)
{
/// Positions are just a cumulative sum of counts.
buckets[i] = buckets[i - 1] + count[i - 1];
/// If this bucket starts after limit, we don't need it.
if (!finish_early && buckets[i] >= arr + limit)
{
buckets_for_recursion = i;
finish = buckets[i];
finish_early = true;
/// We cannot break here, because we need correct pointers to all buckets, see the next loop.
}
}
/* At this point, we have the following variables:
* count[i] is the size of i-th bucket
* buckets[i] is a pointer to the beginning of i-th bucket, buckets[-1] == buckets[0]
* buckets_for_recursion is the number of buckets that should be sorted, the last of them only partially
* finish is a pointer to the end of the first buckets_for_recursion buckets
*/
/// Scatter array elements to buckets until the first buckets_for_recursion buckets are full
/// After the above loop, buckets are shifted towards the end and now pointing to the beginning of i+1th bucket.
for (ssize_t i = 0; /* guarded by 'finish' */; ++i)
{
assert(i < buckets_for_recursion);
/// We look at i-1th index, because bucket pointers are shifted right on every loop iteration,
/// and all buckets before i was completely shifted to the beginning of the next bucket.
/// So, the beginning of i-th bucket is at buckets[i - 1].
Element * bucket_end = buckets[i - 1] + count[i];
/// Fill this bucket.
while (buckets[i] != bucket_end)
{
Element swapper = *buckets[i];
KeyBits tag = extractPart(PASS, swapper);
if (tag != KeyBits(i))
{
/// Invariant: tag > i, because the elements with less tags are already at the right places.
assert(tag > KeyBits(i));
/// While the tag (digit) of the element is not that we need,
/// swap the element with the next element in the bucket for that tag.
/// Interesting observation:
/// - we will definitely find the needed element,
/// because the tag's bucket will contain at least one "wrong" element,
/// because the "right" element is appeared in our bucket.
/// After this loop we shift buckets[i] and buckets[tag] pointers to the right for all found tags.
/// And all positions that were traversed are filled with the proper values.
do
{
std::swap(swapper, *buckets[tag]);
++buckets[tag];
tag = extractPart(PASS, swapper);
} while (tag != KeyBits(i));
*buckets[i] = swapper;
}
/// Now we have the right element at this place.
++buckets[i];
}
if (bucket_end == finish)
break;
}
/// Recursion for the relevant buckets.
if constexpr (PASS > 0)
{
/// Recursively sort buckets, except the last one
for (ssize_t i = 0; i < buckets_for_recursion - 1; ++i)
{
Element * start = buckets[i - 1];
ssize_t subsize = count[i];
radixSortMSDInternalHelper<PASS - 1>(start, subsize, subsize);
}
/// Sort the last necessary bucket with limit
{
ssize_t i = buckets_for_recursion - 1;
Element * start = buckets[i - 1];
ssize_t subsize = count[i];
ssize_t sublimit = limit - (start - arr);
radixSortMSDInternalHelper<PASS - 1>(start, subsize, sublimit);
}
}
}
// A helper to choose sorting algorithm based on array length
template <size_t PASS>
static inline void radixSortMSDInternalHelper(Element * arr, size_t size, size_t limit)
{
if (size <= INSERTION_SORT_THRESHOLD)
insertionSortInternal(arr, size);
else
radixSortMSDInternal<PASS>(arr, size, limit);
}
public:
/** Least significant digit radix sort (stable).
* This function will sort inplace (modify 'arr')
*/
static void executeLSD(Element * arr, size_t size)
{
radixSortLSDInternal<false>(arr, size, false, nullptr);
}
/** This function will start to sort inplace (modify 'arr')
* but on the last step it will write result directly to the destination
* instead of finishing sorting 'arr'.
* In this case it will fill only Result parts of the Element into destination.
* It is handy to avoid unnecessary data movements.
*/
static void executeLSD(Element * arr, size_t size, bool reverse, Result * destination)
{
radixSortLSDInternal<true>(arr, size, reverse, destination);
}
/* Most significant digit radix sort
* Usually slower than LSD and is not stable, but allows partial sorting
* Is not stable, but allows partial sorting.
* And it's more cache-friendly and usually faster than LSD variant.
*
* NOTE: It's beneficial over std::partial_sort only if limit is above ~2% of size for 8 bit radix.
* NOTE: When lowering down limit to 1%, the radix of 4..6 or 10..12 bit started to become beneficial.
* For less than 1% limit, it's not recommended to use.
* NOTE: For huge arrays without limit, the radix 11 suddenly becomes better... but not for smaller arrays.
* Maybe it because histogram will fit in half of L1d cache (2048 * 4 = 16384).
*
* Based on https://github.com/voutcn/kxsort, license:
* The MIT License
@ -411,13 +571,13 @@ public:
/// Use RadixSort with custom traits for complex types instead.
template <typename T>
void radixSortLSD(T *arr, size_t size)
void radixSortLSD(T * arr, size_t size)
{
RadixSort<RadixSortNumTraits<T>>::executeLSD(arr, size);
}
template <typename T>
void radixSortMSD(T *arr, size_t size, size_t limit)
void radixSortMSD(T * arr, size_t size, size_t limit)
{
RadixSort<RadixSortNumTraits<T>>::executeMSD(arr, size, limit);
}

View File

@ -144,15 +144,13 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration
}
else
{
LOG_WARNING(logger, "Unused param " << config_prefix << '.' << rule);
LOG_WARNING(logger, "Unused param {}.{}", config_prefix, rule);
}
}
auto rules_count = rulesCount();
if (rules_count > 0)
{
LOG_INFO(logger, rules_count << " query masking rules loaded.");
}
LOG_INFO(logger, "{} query masking rules loaded.", rules_count);
}
void SensitiveDataMasker::addMaskingRule(

View File

@ -50,10 +50,10 @@ ShellCommand::~ShellCommand()
{
if (terminate_in_destructor)
{
LOG_TRACE(getLogger(), "Will kill shell command pid " << pid << " with SIGTERM");
LOG_TRACE(getLogger(), "Will kill shell command pid {} with SIGTERM", pid);
int retcode = kill(pid, SIGTERM);
if (retcode != 0)
LOG_WARNING(getLogger(), "Cannot kill shell command pid " << pid << " errno '" << errnoToString(retcode) << "'");
LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString(retcode));
}
else if (!wait_called)
tryWait();
@ -61,19 +61,16 @@ ShellCommand::~ShellCommand()
void ShellCommand::logCommand(const char * filename, char * const argv[])
{
std::stringstream log_message;
log_message << "Will start shell command '" << filename << "' with arguments ";
std::stringstream args;
for (int i = 0; argv != nullptr && argv[i] != nullptr; ++i)
{
if (i > 0)
{
log_message << ", ";
}
args << ", ";
/// NOTE: No escaping is performed.
log_message << "'" << argv[i] << "'";
args << "'" << argv[i] << "'";
}
LOG_TRACE(ShellCommand::getLogger(), log_message.str());
LOG_TRACE(ShellCommand::getLogger(), "Will start shell command '{}' with arguments {}", filename, args.str());
}
std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor)
@ -134,7 +131,7 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor));
LOG_TRACE(getLogger(), "Started shell command '" << filename << "' with pid " << pid);
LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid);
/// Now the ownership of the file descriptors is passed to the result.
pipe_stdin.fds_rw[1] = -1;
@ -188,13 +185,13 @@ int ShellCommand::tryWait()
{
wait_called = true;
LOG_TRACE(getLogger(), "Will wait for shell command pid " << pid);
LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid);
int status = 0;
if (-1 == waitpid(pid, &status, 0))
throwFromErrno("Cannot waitpid", ErrorCodes::CANNOT_WAITPID);
LOG_TRACE(getLogger(), "Wait for shell command pid " << pid << " completed with status " << status);
LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status);
if (WIFEXITED(status))
return WEXITSTATUS(status);

View File

@ -43,9 +43,9 @@ StatusFile::StatusFile(const std::string & path_)
}
if (!contents.empty())
LOG_INFO(&Logger::get("StatusFile"), "Status file " << path << " already exists - unclean restart. Contents:\n" << contents);
LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents);
else
LOG_INFO(&Logger::get("StatusFile"), "Status file " << path << " already exists and is empty - probably unclean hardware restart.");
LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path);
}
fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666);
@ -90,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_)
StatusFile::~StatusFile()
{
if (0 != close(fd))
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
if (0 != unlink(path.c_str()))
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
}
}

View File

@ -148,7 +148,7 @@ public:
{
if (!checkBridgeIsRunning())
{
LOG_TRACE(log, BridgeHelperMixin::serviceAlias() + " is not running, will try to start it");
LOG_TRACE(log, "{} is not running, will try to start it", BridgeHelperMixin::serviceAlias());
startBridge();
bool started = false;
@ -157,7 +157,7 @@ public:
while (milliseconds_to_wait < 10000)
{
++counter;
LOG_TRACE(log, "Checking " + BridgeHelperMixin::serviceAlias() + " is running, try " << counter);
LOG_TRACE(log, "Checking {} is running, try {}", BridgeHelperMixin::serviceAlias(), counter);
if (checkBridgeIsRunning())
{
started = true;
@ -309,7 +309,7 @@ struct ODBCBridgeMixin
cmd_args.push_back(config.getString("logger." + configPrefix() + "_level"));
}
LOG_TRACE(log, "Starting " + serviceAlias());
LOG_TRACE(log, "Starting {}", serviceAlias());
return ShellCommand::executeDirect(path.toString(), cmd_args, true);
}

View File

@ -64,7 +64,7 @@ Lock::Status Lock::tryCheck() const
}
if (locked && lock_status != LOCKED_BY_ME)
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: " << lock_path << "/" << lock_message);
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: {}/{}", lock_path, lock_message);
return lock_status;
}

View File

@ -79,7 +79,7 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
}
catch (const Poco::Net::DNSException & e)
{
LOG_ERROR(log, "Cannot use ZooKeeper host " << host_string << ", reason: " << e.displayText());
LOG_ERROR(log, "Cannot use ZooKeeper host {}, reason: {}", host_string, e.displayText());
}
}
@ -95,7 +95,10 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000),
Poco::Timespan(0, operation_timeout_ms_ * 1000));
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
if (chroot.empty())
LOG_TRACE(log, "Initialized, hosts: {}", hosts);
else
LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", hosts, chroot);
}
else if (implementation == "testkeeper")
{

View File

@ -34,6 +34,7 @@ target_link_libraries (compact_array PRIVATE clickhouse_common_io)
add_executable (radix_sort radix_sort.cpp)
target_link_libraries (radix_sort PRIVATE clickhouse_common_io)
target_include_directories(radix_sort SYSTEM PRIVATE ${PDQSORT_INCLUDE_DIR})
if (USE_OPENCL)
add_executable (bitonic_sort bitonic_sort.cpp)

View File

@ -95,7 +95,7 @@ int main(int, char **)
mut->set(3);
std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n";
std::cerr << "addresses: " << x.get() << ", " << ", " << mut.get() << "\n";
std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n";
y = std::move(mut);
}

View File

@ -62,6 +62,6 @@ TEST(ShellCommand, AutoWait)
//command->wait(); // now automatic
}
// std::cerr << "inspect me: ps auxwwf" << "\n";
// std::cerr << "inspect me: ps auxwwf\n";
// std::this_thread::sleep_for(std::chrono::seconds(100));
}

View File

@ -1,14 +1,23 @@
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#include <malloc.h>
#endif
#include <iomanip>
#include <pcg_random.hpp>
#include <ext/bit_cast.h>
//#if defined(NDEBUG)
//#undef NDEBUG
#include <Common/RadixSort.h>
//#endif
#include <Common/Stopwatch.h>
#include <Common/randomSeed.h>
#include <IO/ReadHelpers.h>
#include <Core/Defines.h>
#include <pdqsort.h>
using Key = double;
/// Example:
/// for i in {6,8} {11..26}; do echo $i; for j in {1..10}; do ./radix_sort $i 65536 1000; done; echo; done
using Key = UInt64;
static void NO_INLINE sort1(Key * data, size_t size)
{
@ -24,29 +33,150 @@ static void NO_INLINE sort3(Key * data, size_t size)
{
std::sort(data, data + size, [](Key a, Key b)
{
return RadixSortFloatTransform<uint32_t>::forward(ext::bit_cast<uint32_t>(a))
< RadixSortFloatTransform<uint32_t>::forward(ext::bit_cast<uint32_t>(b));
return RadixSortFloatTransform<uint64_t>::forward(ext::bit_cast<uint64_t>(a))
< RadixSortFloatTransform<uint64_t>::forward(ext::bit_cast<uint64_t>(b));
});
}
static void NO_INLINE sort4(Key * data, size_t size)
{
radixSortMSD(data, size, size);
}
static void NO_INLINE sort5(Key * data, size_t size)
{
pdqsort(data, data + size);
}
static void NO_INLINE sort6(Key * data, size_t size, size_t limit)
{
std::partial_sort(data, data + limit, data + size);
}
static void NO_INLINE sort7(Key * data, size_t size, size_t limit)
{
std::partial_sort(data, data + limit, data + size, [](Key a, Key b)
{
return RadixSortFloatTransform<uint64_t>::forward(ext::bit_cast<uint64_t>(a))
< RadixSortFloatTransform<uint64_t>::forward(ext::bit_cast<uint64_t>(b));
});
}
static void NO_INLINE sort8(Key * data, size_t size, size_t limit)
{
radixSortMSD(data, size, limit);
}
template <size_t N>
struct RadixSortTraitsWithCustomBits : RadixSortNumTraits<Key>
{
static constexpr size_t PART_SIZE_BITS = N;
};
static void NO_INLINE sort11(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<1>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort12(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<2>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort13(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<3>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort14(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<4>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort15(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<5>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort16(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<6>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort17(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<7>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort18(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<8>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort19(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<9>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort20(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<10>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort21(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<11>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort22(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<12>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort23(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<13>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort24(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<14>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort25(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<15>>::executeMSD(data, size, limit);
}
static void NO_INLINE sort26(Key * data, size_t size, size_t limit)
{
RadixSort<RadixSortTraitsWithCustomBits<16>>::executeMSD(data, size, limit);
}
int main(int argc, char ** argv)
{
pcg64 rng;
pcg64 rng(randomSeed());
if (argc < 3)
if (argc < 3 || argc > 4)
{
std::cerr << "Usage: program n method\n";
std::cerr << "Usage: program method n [limit]\n";
return 1;
}
size_t n = DB::parse<size_t>(argv[1]);
size_t method = DB::parse<size_t>(argv[2]);
size_t method = DB::parse<size_t>(argv[1]);
size_t n = DB::parse<size_t>(argv[2]);
size_t limit = n;
if (argc == 4)
limit = DB::parse<size_t>(argv[3]);
std::cerr << std::fixed << std::setprecision(3);
std::vector<Key> data(n);
// srand(time(nullptr));
{
Stopwatch watch;
@ -54,12 +184,12 @@ int main(int argc, char ** argv)
elem = rng();
watch.stop();
double elapsed = watch.elapsedSeconds();
/* double elapsed = watch.elapsedSeconds();
std::cerr
<< "Filled in " << elapsed
<< " (" << n / elapsed << " elem/sec., "
<< n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)"
<< std::endl;
<< std::endl;*/
}
if (n <= 100)
@ -70,13 +200,34 @@ int main(int argc, char ** argv)
std::cerr << std::endl;
}
{
Stopwatch watch;
if (method == 1) sort1(data.data(), n);
if (method == 2) sort2(data.data(), n);
if (method == 3) sort3(data.data(), n);
if (method == 1) sort1(data.data(), n);
if (method == 2) sort2(data.data(), n);
if (method == 3) sort3(data.data(), n);
if (method == 4) sort4(data.data(), n);
if (method == 5) sort5(data.data(), n);
if (method == 6) sort6(data.data(), n, limit);
if (method == 7) sort7(data.data(), n, limit);
if (method == 8) sort8(data.data(), n, limit);
if (method == 11) sort11(data.data(), n, limit);
if (method == 12) sort12(data.data(), n, limit);
if (method == 13) sort13(data.data(), n, limit);
if (method == 14) sort14(data.data(), n, limit);
if (method == 15) sort15(data.data(), n, limit);
if (method == 16) sort16(data.data(), n, limit);
if (method == 17) sort17(data.data(), n, limit);
if (method == 18) sort18(data.data(), n, limit);
if (method == 19) sort19(data.data(), n, limit);
if (method == 20) sort20(data.data(), n, limit);
if (method == 21) sort21(data.data(), n, limit);
if (method == 22) sort22(data.data(), n, limit);
if (method == 23) sort23(data.data(), n, limit);
if (method == 24) sort24(data.data(), n, limit);
if (method == 25) sort25(data.data(), n, limit);
if (method == 26) sort26(data.data(), n, limit);
watch.stop();
double elapsed = watch.elapsedSeconds();
@ -87,33 +238,39 @@ int main(int argc, char ** argv)
<< std::endl;
}
bool ok = true;
{
Stopwatch watch;
size_t i = 1;
while (i < n)
while (i < limit)
{
if (!(data[i - 1] <= data[i]))
{
ok = false;
break;
}
++i;
}
watch.stop();
double elapsed = watch.elapsedSeconds();
std::cerr
<< "Checked in " << elapsed
<< " (" << n / elapsed << " elem/sec., "
<< n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)"
<< std::endl
<< "Result: " << (i == n ? "Ok." : "Fail!") << std::endl;
if (!ok)
std::cerr
<< "Checked in " << elapsed
<< " (" << limit / elapsed << " elem/sec., "
<< limit * sizeof(Key) / elapsed / 1048576 << " MB/sec.)"
<< std::endl
<< "Result: " << (ok ? "Ok." : "Fail!") << std::endl;
}
if (n <= 1000)
if (!ok && limit <= 100000)
{
std::cerr << std::endl;
std::cerr << data[0] << ' ';
for (size_t i = 1; i < n; ++i)
for (size_t i = 1; i < limit; ++i)
{
if (!(data[i - 1] <= data[i]))
std::cerr << "*** ";

View File

@ -220,7 +220,7 @@ template <typename T, typename ContainerLeft, typename ContainerRight>
if (l_size != r_size)
{
result = ::testing::AssertionFailure() << "size mismatch" << " expected: " << l_size << " got:" << r_size;
result = ::testing::AssertionFailure() << "size mismatch expected: " << l_size << " got:" << r_size;
}
if (l_size == 0 || r_size == 0)
{

View File

@ -111,7 +111,7 @@ void BackgroundSchedulePoolTaskInfo::execute()
static const int32_t slow_execution_threshold_ms = 200;
if (milliseconds >= slow_execution_threshold_ms)
LOG_TRACE(&Logger::get(log_name), "Execution took " << milliseconds << " ms.");
LOG_TRACE(&Logger::get(log_name), "Execution took {} ms.", milliseconds);
{
std::lock_guard lock_schedule(schedule_mutex);
@ -156,7 +156,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met
, memory_metric(memory_metric_)
, thread_name(thread_name_)
{
LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with " << size << " threads");
LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size);
threads.resize(size);
for (auto & thread : threads)

View File

@ -1068,7 +1068,7 @@ public:
# pragma GCC diagnostic pop
String pem(pem_buf, pem_size);
LOG_TRACE(log, "Key: " << pem);
LOG_TRACE(log, "Key: {}", pem);
AuthMoreData data(pem);
packet_sender->sendPacket(data, true);

View File

@ -423,6 +423,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \
M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\

View File

@ -599,7 +599,7 @@ namespace details
void SettingsCollectionUtils::warningNameNotFound(const StringRef & name)
{
static auto * log = &Logger::get("Settings");
LOG_WARNING(log, "Unknown setting " << name << ", skipping");
LOG_WARNING(log, "Unknown setting {}, skipping", name);
}
void SettingsCollectionUtils::throwNameNotFound(const StringRef & name)

View File

@ -3,12 +3,12 @@
#include <DataStreams/IBlockOutputStream.h>
#include <Columns/ColumnConst.h>
#include <Storages/ColumnDefault.h>
#include <Interpreters/Context.h>
namespace DB
{
class Context;
/** This stream adds three types of columns into block
* 1. Columns, that are missed inside request, but present in table without defaults (missed columns)

View File

@ -2,12 +2,13 @@
#include <DataStreams/IBlockInputStream.h>
#include <Storages/ColumnDefault.h>
#include <Interpreters/Context.h>
namespace DB
{
class Context;
/// Adds defaults to columns using BlockDelayedDefaults bitmask attached to Block by child InputStream.
class AddingDefaultsBlockInputStream : public IBlockInputStream
{

View File

@ -1,4 +1,4 @@
#include <Common/ClickHouseRevision.h>
#include <Common/formatReadable.h>
#include <DataStreams/BlocksListBlockInputStream.h>
#include <DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
@ -60,9 +60,7 @@ Block AggregatingBlockInputStream::readImpl()
input_streams.emplace_back(temporary_inputs.back()->block_in);
}
LOG_TRACE(log, "Will merge " << files.files.size() << " temporary files of size "
<< (files.sum_size_compressed / 1048576.0) << " MiB compressed, "
<< (files.sum_size_uncompressed / 1048576.0) << " MiB uncompressed.");
LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed));
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(input_streams, params, final, 1, 1);
}

View File

@ -1,9 +1,31 @@
#include <DataStreams/BlockIO.h>
#include <Interpreters/ProcessList.h>
#include <Processors/Executors/PipelineExecutingBlockInputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
BlockInputStreamPtr BlockIO::getInputStream()
{
if (out)
throw Exception("Cannot get input stream from BlockIO because output stream is not empty",
ErrorCodes::LOGICAL_ERROR);
if (in)
return in;
if (pipeline.initialized())
return std::make_shared<PipelineExecutingBlockInputStream>(std::move(pipeline));
throw Exception("Cannot get input stream from BlockIO because query pipeline was not initialized",
ErrorCodes::LOGICAL_ERROR);
}
void BlockIO::reset()
{
/** process_list_entry should be destroyed after in, after out and after pipeline,

View File

@ -50,6 +50,9 @@ struct BlockIO
exception_callback();
}
/// Returns in or converts pipeline to stream. Throws if out is not empty.
BlockInputStreamPtr getInputStream();
private:
void reset();
};

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