Merge remote-tracking branch 'origin/master' into musysroot

This commit is contained in:
Michael Kolupaev 2024-08-19 21:52:41 +00:00
commit 11658c07b6
246 changed files with 1762 additions and 1245 deletions

View File

@ -482,7 +482,7 @@ jobs:
if: ${{ !failure() }}
run: |
# update overall ci report
python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }}
python3 ./tests/ci/finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }}
- name: Check Workflow results
if: ${{ !cancelled() }}
run: |
@ -490,5 +490,4 @@ jobs:
cat > "$WORKFLOW_RESULT_FILE" << 'EOF'
${{ toJson(needs) }}
EOF
python3 ./tests/ci/ci_buddy.py --check-wf-status

View File

@ -8,6 +8,8 @@ endif ()
# when instantiated from JSON.cpp. Try again when libcxx(abi) and Clang are upgraded to 16.
set (CMAKE_CXX_STANDARD 20)
configure_file(GitHash.cpp.in GitHash.generated.cpp)
set (SRCS
argsToConfig.cpp
cgroupsv2.cpp
@ -33,6 +35,7 @@ set (SRCS
safeExit.cpp
throwError.cpp
Numa.cpp
GitHash.generated.cpp
)
add_library (common ${SRCS})

View File

@ -42,19 +42,9 @@ endif ()
# But use 2 parallel jobs, since:
# - this is what llvm does
# - and I've verfied that lld-11 does not use all available CPU time (in peak) while linking one binary
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO)
if (ARCH_AARCH64)
# aarch64 builds start to often fail with OOMs (reason not yet clear), for now let's limit the concurrency
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 1.")
set (PARALLEL_LINK_JOBS 1)
if (LINKER_NAME MATCHES "lld")
math(EXPR LTO_JOBS ${NUMBER_OF_LOGICAL_CORES}/4)
set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -Wl,--thinlto-jobs=${LTO_JOBS}")
endif()
elseif (PARALLEL_LINK_JOBS GREATER 2)
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.")
set (PARALLEL_LINK_JOBS 2)
endif ()
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLEL_LINK_JOBS GREATER 2)
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.")
set (PARALLEL_LINK_JOBS 2)
endif()
message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB RAM, 'OFF' means the native core count).")

2
contrib/aws vendored

@ -1 +1 @@
Subproject commit 1c2946bfcb7f1e3ae0a858de0b59d4f1a7b4ccaf
Subproject commit d5450d76abda556ce145ddabe7e0cc6a7644ec59

2
contrib/aws-crt-cpp vendored

@ -1 +1 @@
Subproject commit f532d6abc0d2b0d8b5d6fe9e7c51eaedbe4afbd0
Subproject commit e5aa45cacfdcda7719ead38760e7c61076f5745f

View File

@ -256,22 +256,6 @@ function configure
rm -f "$FASTTEST_DATA/config.d/secure_ports.xml"
}
function timeout_with_logging() {
local exit_code=0
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
echo "Checking if it is a timeout. The code 124 will indicate a timeout."
if [[ "${exit_code}" -eq "124" ]]
then
echo "The command 'timeout ${*}' has been killed by timeout."
else
echo "No, it isn't a timeout."
fi
return $exit_code
}
function run_tests
{
clickhouse-server --version
@ -340,7 +324,7 @@ case "$stage" in
configure 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/install_log.txt"
;&
"run_tests")
timeout_with_logging 35m bash -c run_tests ||:
run_tests ||:
/process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \
--out-results-file "$FASTTEST_OUTPUT/test_results.tsv" \
--out-status-file "$FASTTEST_OUTPUT/check_status.tsv" || echo -e "failure\tCannot parse results" > "$FASTTEST_OUTPUT/check_status.tsv"

View File

@ -35,7 +35,6 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \
ENV TZ=Europe/Amsterdam
ENV MAX_RUN_TIME=9000
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ARG sqllogic_test_repo="https://github.com/gregrahn/sqllogictest.git"

View File

@ -94,7 +94,7 @@ function run_tests()
export -f run_tests
timeout "${MAX_RUN_TIME:-9000}" bash -c run_tests || echo "timeout reached" >&2
run_tests
#/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv

View File

@ -22,7 +22,6 @@ ARG sqltest_repo="https://github.com/elliotchance/sqltest/"
RUN git clone ${sqltest_repo}
ENV TZ=UTC
ENV MAX_RUN_TIME=900
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
COPY run.sh /

View File

@ -4,9 +4,6 @@
source /setup_export_logs.sh
set -e -x
MAX_RUN_TIME=${MAX_RUN_TIME:-3600}
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 3600 : MAX_RUN_TIME))
# Choose random timezone for this test run
TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)"
echo "Choosen random timezone $TZ"
@ -118,14 +115,11 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
clickhouse-client --query "CREATE TABLE test.hits AS datasets.hits_v1"
clickhouse-client --query "CREATE TABLE test.visits AS datasets.visits_v1"
clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1"
clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1"
clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1"
clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1"
clickhouse-client --query "DROP TABLE datasets.hits_v1"
clickhouse-client --query "DROP TABLE datasets.visits_v1"
MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours)
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
else
clickhouse-client --query "CREATE DATABASE test"
clickhouse-client --query "SHOW TABLES FROM test"
@ -191,8 +185,8 @@ else
ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)
SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'"
clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16"
clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16"
clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16"
clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16"
clickhouse-client --query "DROP TABLE datasets.visits_v1 SYNC"
clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC"
else
@ -200,7 +194,8 @@ else
clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
fi
clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'"
clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16"
# AWS S3 is very inefficient, so increase memory even further:
clickhouse-client --max_memory_usage 30G --max_memory_usage_for_user 30G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16"
fi
clickhouse-client --query "SHOW TABLES FROM test"
@ -257,24 +252,7 @@ function run_tests()
export -f run_tests
function timeout_with_logging() {
local exit_code=0
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
echo "Checking if it is a timeout. The code 124 will indicate a timeout."
if [[ "${exit_code}" -eq "124" ]]
then
echo "The command 'timeout ${*}' has been killed by timeout."
else
echo "No, it isn't a timeout."
fi
return $exit_code
}
TIMEOUT=$((MAX_RUN_TIME - 700))
timeout_with_logging "$TIMEOUT" bash -c run_tests ||:
run_tests ||:
echo "Files in current directory"
ls -la ./

View File

@ -65,7 +65,6 @@ ENV TZ=Europe/Amsterdam
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV NUM_TRIES=1
ENV MAX_RUN_TIME=0
# Unrelated to vars in setup_minio.sh, but should be the same there
# to have the same binaries for local running scenario

View File

@ -12,9 +12,6 @@ dmesg --clear
# fail on errors, verbose and export all env variables
set -e -x -a
MAX_RUN_TIME=${MAX_RUN_TIME:-9000}
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 9000 : MAX_RUN_TIME))
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
@ -308,8 +305,6 @@ function run_tests()
try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')"
TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800))
START_TIME=${SECONDS}
set +e
TEST_ARGS=(
@ -324,32 +319,22 @@ function run_tests()
--test-runs "$NUM_TRIES"
"${ADDITIONAL_OPTIONS[@]}"
)
timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s clickhouse-test "${TEST_ARGS[@]}" 2>&1 \
clickhouse-test "${TEST_ARGS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
set -e
DURATION=$((SECONDS - START_TIME))
echo "Elapsed ${DURATION} seconds."
if [[ $DURATION -ge $TIMEOUT ]]
then
echo "It looks like the command is terminated by the timeout, which is ${TIMEOUT} seconds."
fi
}
export -f run_tests
# This should be enough to setup job and collect artifacts
TIMEOUT=$((MAX_RUN_TIME - 700))
if [ "$NUM_TRIES" -gt "1" ]; then
# We don't run tests with Ordinary database in PRs, only in master.
# So run new/changed tests with Ordinary at least once in flaky check.
timeout_with_logging "$TIMEOUT" bash -c 'NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests' \
NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests \
| sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||:
fi
timeout_with_logging "$TIMEOUT" bash -c run_tests ||:
run_tests ||:
echo "Files in current directory"
ls -la ./
@ -391,8 +376,8 @@ done
# wait for minio to flush its batch if it has any
sleep 1
clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE"
clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow"
clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow"
clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 --max_result_bytes 0 --max_result_rows 0 --max_rows_to_read 0 --max_bytes_to_read 0 -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow"
clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 --max_result_bytes 0 --max_result_rows 0 --max_rows_to_read 0 --max_bytes_to_read 0 -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow"
# Stop server so we can safely read data with clickhouse-local.
# Why do we read data with clickhouse-local?

View File

@ -40,22 +40,6 @@ function fn_exists() {
declare -F "$1" > /dev/null;
}
function timeout_with_logging() {
local exit_code=0
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
echo "Checking if it is a timeout. The code 124 will indicate a timeout."
if [[ "${exit_code}" -eq "124" ]]
then
echo "The command 'timeout ${*}' has been killed by timeout."
else
echo "No, it isn't a timeout."
fi
return $exit_code
}
function collect_core_dumps()
{
find . -type f -maxdepth 1 -name 'core.*' | while read -r core; do

View File

@ -1381,7 +1381,7 @@ Default value: `2`.
Close connection before returning connection to the pool.
Default value: true.
Default value: false.
## odbc_bridge_connection_pool_size {#odbc-bridge-connection-pool-size}

View File

@ -1164,6 +1164,9 @@ void Client::processOptions(const OptionsDescription & options_description,
/// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.)
client_context = global_context;
initClientContext();
/// Allow to pass-through unknown settings to the server.
client_context->getAccessControl().allowAllSettings();
}

View File

@ -264,7 +264,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
if (!backslash)
{
WriteBufferFromOwnString str_buf;
formatAST(*res, str_buf, hilite, oneline || approx_query_length < max_line_length);
bool oneline_current_query = oneline || approx_query_length < max_line_length;
IAST::FormatSettings settings(str_buf, oneline_current_query, hilite);
settings.show_secrets = true;
settings.print_pretty_type_names = !oneline_current_query;
res->format(settings);
if (insert_query_payload)
{
@ -307,7 +311,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
else
{
WriteBufferFromOwnString str_buf;
formatAST(*res, str_buf, hilite, oneline);
bool oneline_current_query = oneline || approx_query_length < max_line_length;
IAST::FormatSettings settings(str_buf, oneline_current_query, hilite);
settings.show_secrets = true;
settings.print_pretty_type_names = !oneline_current_query;
res->format(settings);
auto res_string = str_buf.str();
WriteBufferFromOStream res_cout(std::cout, 4096);

View File

@ -66,6 +66,8 @@
/// A minimal file used when the keeper is run without installation
INCBIN(keeper_resource_embedded_xml, SOURCE_DIR "/programs/keeper/keeper_embedded.xml");
extern const char * GIT_HASH;
int mainEntryClickHouseKeeper(int argc, char ** argv)
{
DB::Keeper app;
@ -675,7 +677,7 @@ void Keeper::logRevision() const
"Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}",
VERSION_STRING,
ClickHouseRevision::getVersionRevision(),
git_hash.empty() ? "<unknown>" : git_hash,
GIT_HASH,
build_id.empty() ? "<unknown>" : build_id,
getpid());
}

View File

@ -367,7 +367,7 @@ std::string LocalServer::getInitialCreateTableQuery()
else
table_structure = "(" + table_structure + ")";
return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});",
return fmt::format("CREATE TEMPORARY TABLE {} {} ENGINE = File({}, {});",
table_name, table_structure, data_format, table_file);
}

View File

@ -1769,6 +1769,8 @@ try
new_server_settings.http_connections_store_limit,
});
DNSResolver::instance().setFilterSettings(new_server_settings.dns_allow_resolve_names_to_ipv4, new_server_settings.dns_allow_resolve_names_to_ipv6);
if (global_context->isServerCompletelyStarted())
CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability);
@ -1920,7 +1922,7 @@ try
auto & access_control = global_context->getAccessControl();
try
{
access_control.setUpFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
access_control.setupFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
}
catch (...)
{

View File

@ -280,7 +280,7 @@ void AccessControl::shutdown()
}
void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
void AccessControl::setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
{
if (config_.has("custom_settings_prefixes"))
@ -868,4 +868,10 @@ const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const
return *external_authenticators;
}
void AccessControl::allowAllSettings()
{
custom_settings_prefixes->registerPrefixes({""});
}
}

View File

@ -57,7 +57,7 @@ public:
void shutdown() override;
/// Initializes access storage (user directories).
void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
void setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_);
/// Parses access entities from a configuration loaded from users.xml.
@ -238,6 +238,9 @@ public:
/// Gets manager of notifications.
AccessChangesNotifier & getChangesNotifier();
/// Allow all setting names - this can be used in clients to pass-through unknown settings to the server.
void allowAllSettings();
private:
class ContextAccessCache;
class CustomSettingsPrefixes;

View File

@ -82,7 +82,8 @@ public:
Result authenticate(const String & user_name, const String & password) const
{
Poco::Net::HTTPRequest request{Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery()};
Poco::Net::HTTPRequest request{
Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1};
Poco::Net::HTTPBasicCredentials basic_credentials{user_name, password};
basic_credentials.authenticate(request);

View File

@ -219,8 +219,8 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang
});
}
template <class T>
bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure)
template <typename SettingsT>
bool getNewValueToCheck(const SettingsT & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure)
{
Field current_value;
bool has_current_value = current_settings.tryGet(change.name, current_value);
@ -230,12 +230,12 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel
return false;
if (throw_on_failure)
new_value = T::castValueUtil(change.name, change.value);
new_value = SettingsT::castValueUtil(change.name, change.value);
else
{
try
{
new_value = T::castValueUtil(change.name, change.value);
new_value = SettingsT::castValueUtil(change.name, change.value);
}
catch (...)
{

View File

@ -114,7 +114,7 @@ private:
{
if (ind < first.size())
return first[ind];
return second[ind % first.size()];
return second[ind - first.size()];
}
size_t size() const

View File

@ -58,6 +58,7 @@
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/CompressionMethod.h>
@ -331,7 +332,11 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting
{
output_stream << std::endl;
WriteBufferFromOStream res_buf(output_stream, 4096);
formatAST(*res, res_buf);
IAST::FormatSettings format_settings(res_buf, /* one_line */ false);
format_settings.hilite = true;
format_settings.show_secrets = true;
format_settings.print_pretty_type_names = true;
res->format(format_settings);
res_buf.finalize();
output_stream << std::endl << std::endl;
}
@ -1604,14 +1609,14 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
auto metadata = storage->getInMemoryMetadataPtr();
QueryPlan plan;
storage->read(
plan,
sample.getNames(),
storage->getStorageSnapshot(metadata, client_context),
query_info,
client_context,
{},
client_context->getSettingsRef().max_block_size,
getNumberOfPhysicalCPUCores());
plan,
sample.getNames(),
storage->getStorageSnapshot(metadata, client_context),
query_info,
client_context,
{},
client_context->getSettingsRef().max_block_size,
getNumberOfPhysicalCPUCores());
auto builder = plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(client_context),
@ -1888,48 +1893,19 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
profile_events.watch.restart();
{
/// Temporarily apply query settings to context.
std::optional<Settings> old_settings;
SCOPE_EXIT_SAFE({
if (old_settings)
client_context->setSettings(*old_settings);
/// Temporarily apply query settings to the context.
Settings old_settings = client_context->getSettingsCopy();
SCOPE_EXIT_SAFE(
{
client_context->setSettings(old_settings);
});
auto apply_query_settings = [&](const IAST & settings_ast)
{
if (!old_settings)
old_settings.emplace(client_context->getSettingsRef());
client_context->applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
client_context->resetSettingsToDefaultValue(settings_ast.as<ASTSetQuery>()->default_settings);
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (const auto * select = parsed_query->as<ASTSelectQuery>(); select && select->settings())
apply_query_settings(*select->settings());
else if (const auto * select_with_union = parsed_query->as<ASTSelectWithUnionQuery>())
{
const ASTs & children = select_with_union->list_of_selects->children;
if (!children.empty())
{
// On the client it is enough to apply settings only for the
// last SELECT, since the only thing that is important to apply
// on the client is format settings.
const auto * last_select = children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
{
apply_query_settings(*last_select->settings());
}
}
}
else if (const auto * query_with_output = parsed_query->as<ASTQueryWithOutput>(); query_with_output && query_with_output->settings_ast)
apply_query_settings(*query_with_output->settings_ast);
else if (insert && insert->settings_ast)
apply_query_settings(*insert->settings_ast);
InterpreterSetQuery::applySettingsFromQuery(parsed_query, client_context);
if (!connection->checkConnected(connection_parameters.timeouts))
connect();
ASTPtr input_function;
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (insert && insert->select)
insert->tryFindInputFunction(input_function);

View File

@ -12,6 +12,7 @@
#include <atomic>
#include <optional>
#include <string_view>
#include "Common/MultiVersion.h"
#include <unordered_set>
#include "DNSPTRResolverProvider.h"
@ -139,12 +140,6 @@ DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host)
return addresses;
}
DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase<std::string, DNSResolver::CacheEntry> & cache, const std::string & host)
{
auto [result, _ ] = cache.getOrSet(host, [&host]() {return std::make_shared<DNSResolver::CacheEntry>(resolveIPAddressImpl(host), std::chrono::system_clock::now());});
return result->addresses;
}
std::unordered_set<String> reverseResolveImpl(const Poco::Net::IPAddress & address)
{
auto ptr_resolver = DB::DNSPTRResolverProvider::get();
@ -198,21 +193,89 @@ struct DNSResolver::Impl
std::atomic<bool> disable_cache{false};
};
struct DNSResolver::AddressFilter
{
struct DNSFilterSettings
{
bool dns_allow_resolve_names_to_ipv4{true};
bool dns_allow_resolve_names_to_ipv6{true};
};
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(getLogger("DNSResolver")) {}
AddressFilter() : settings(std::make_unique<DNSFilterSettings>()) {}
void performAddressFiltering(DNSResolver::IPAddresses & addresses) const
{
const auto current_settings = settings.get();
bool dns_resolve_ipv4 = current_settings->dns_allow_resolve_names_to_ipv4;
bool dns_resolve_ipv6 = current_settings->dns_allow_resolve_names_to_ipv6;
if (dns_resolve_ipv4 && dns_resolve_ipv6)
{
return;
}
if (!dns_resolve_ipv4 && !dns_resolve_ipv6)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "DNS can't resolve any address, because dns_resolve_ipv6_interfaces and dns_resolve_ipv4_interfaces both are disabled");
}
std::erase_if(addresses, [dns_resolve_ipv6, dns_resolve_ipv4](const Poco::Net::IPAddress& address)
{
return (address.family() == Poco::Net::IPAddress::IPv6 && !dns_resolve_ipv6)
|| (address.family() == Poco::Net::IPAddress::IPv4 && !dns_resolve_ipv4);
});
}
void setSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6)
{
settings.set(std::make_unique<DNSFilterSettings>(dns_allow_resolve_names_to_ipv4, dns_allow_resolve_names_to_ipv6));
}
MultiVersion<DNSFilterSettings> settings;
};
DNSResolver::DNSResolver()
: impl(std::make_unique<DNSResolver::Impl>())
, addressFilter(std::make_unique<DNSResolver::AddressFilter>())
, log(getLogger("DNSResolver")) {}
DNSResolver::IPAddresses DNSResolver::getResolvedIPAdressessWithFiltering(const std::string & host)
{
auto addresses = resolveIPAddressImpl(host);
addressFilter->performAddressFiltering(addresses);
if (addresses.empty())
{
ProfileEvents::increment(ProfileEvents::DNSError);
throw DB::NetException(ErrorCodes::DNS_ERROR, "After filtering there are no resolved address for host({}).", host);
}
return addresses;
}
DNSResolver::IPAddresses DNSResolver::resolveIPAddressWithCache(const std::string & host)
{
auto [result, _ ] = impl->cache_host.getOrSet(host, [&host, this]() {return std::make_shared<DNSResolver::CacheEntry>(getResolvedIPAdressessWithFiltering(host), std::chrono::system_clock::now());});
return result->addresses;
}
Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host)
{
return pickAddress(resolveHostAll(host)); // random order -> random pick
}
void DNSResolver::setFilterSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6)
{
addressFilter->setSettings(dns_allow_resolve_names_to_ipv4, dns_allow_resolve_names_to_ipv6);
}
DNSResolver::IPAddresses DNSResolver::resolveHostAllInOriginOrder(const std::string & host)
{
if (impl->disable_cache)
return resolveIPAddressImpl(host);
return getResolvedIPAdressessWithFiltering(host);
addToNewHosts(host);
return resolveIPAddressWithCache(impl->cache_host, host);
return resolveIPAddressWithCache(host);
}
DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host)
@ -232,7 +295,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_an
splitHostAndPort(host_and_port, host, port);
addToNewHosts(host);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(impl->cache_host, host)), port);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(host)), port);
}
Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, UInt16 port)
@ -241,7 +304,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U
return Poco::Net::SocketAddress(host, port);
addToNewHosts(host);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(impl->cache_host, host)), port);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(host)), port);
}
std::vector<Poco::Net::SocketAddress> DNSResolver::resolveAddressList(const std::string & host, UInt16 port)
@ -254,7 +317,7 @@ std::vector<Poco::Net::SocketAddress> DNSResolver::resolveAddressList(const std:
if (!impl->disable_cache)
addToNewHosts(host);
std::vector<Poco::Net::IPAddress> ips = impl->disable_cache ? hostByName(host) : resolveIPAddressWithCache(impl->cache_host, host);
std::vector<Poco::Net::IPAddress> ips = impl->disable_cache ? hostByName(host) : resolveIPAddressWithCache(host);
auto ips_end = std::unique(ips.begin(), ips.end());
addresses.reserve(ips_end - ips.begin());
@ -419,8 +482,8 @@ bool DNSResolver::updateCache(UInt32 max_consecutive_failures)
bool DNSResolver::updateHost(const String & host)
{
const auto old_value = resolveIPAddressWithCache(impl->cache_host, host);
auto new_value = resolveIPAddressImpl(host);
const auto old_value = resolveIPAddressWithCache(host);
auto new_value = getResolvedIPAdressessWithFiltering(host);
const bool result = old_value != new_value;
impl->cache_host.set(host, std::make_shared<DNSResolver::CacheEntry>(std::move(new_value), std::chrono::system_clock::now()));
return result;

View File

@ -68,6 +68,8 @@ public:
/// Returns true if IP of any host has been changed or an element was dropped (too many failures)
bool updateCache(UInt32 max_consecutive_failures);
void setFilterSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6);
/// Returns a copy of cache entries
std::vector<std::pair<std::string, CacheEntry>> cacheEntries() const;
@ -86,6 +88,10 @@ private:
struct Impl;
std::unique_ptr<Impl> impl;
struct AddressFilter;
std::unique_ptr<AddressFilter> addressFilter;
LoggerPtr log;
/// Updates cached value and returns true it has been changed.
@ -94,6 +100,9 @@ private:
void addToNewHosts(const String & host);
void addToNewAddresses(const Poco::Net::IPAddress & address);
IPAddresses resolveIPAddressWithCache(const std::string & host);
IPAddresses getResolvedIPAdressessWithFiltering(const std::string & host);
};
}

View File

@ -112,9 +112,8 @@ namespace
return configuration.has(config_prefix + ".uri");
}
/*
* New syntax requires protocol prefix "<http> or <https>"
* */
/* New syntax requires protocol prefix "<http> or <https>"
*/
std::optional<std::string> getProtocolPrefix(
ProxyConfiguration::Protocol request_protocol,
const String & config_prefix,
@ -130,22 +129,18 @@ namespace
return protocol_prefix;
}
template <bool new_syntax>
std::optional<std::string> calculatePrefixBasedOnSettingsSyntax(
bool new_syntax,
ProxyConfiguration::Protocol request_protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
)
{
if (!configuration.has(config_prefix))
{
return std::nullopt;
}
if constexpr (new_syntax)
{
if (new_syntax)
return getProtocolPrefix(request_protocol, config_prefix, configuration);
}
return config_prefix;
}
@ -155,24 +150,21 @@ std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::
Protocol request_protocol,
const Poco::Util::AbstractConfiguration & configuration)
{
if (auto resolver = getFromSettings(request_protocol, "proxy", configuration))
{
if (auto resolver = getFromSettings(true, request_protocol, "proxy", configuration))
return resolver;
}
return std::make_shared<EnvironmentProxyConfigurationResolver>(
request_protocol,
isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration));
}
template <bool is_new_syntax>
std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::getFromSettings(
bool new_syntax,
Protocol request_protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
)
const Poco::Util::AbstractConfiguration & configuration)
{
auto prefix_opt = calculatePrefixBasedOnSettingsSyntax<is_new_syntax>(request_protocol, config_prefix, configuration);
auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(new_syntax, request_protocol, config_prefix, configuration);
if (!prefix_opt)
{
@ -195,20 +187,17 @@ std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::
std::shared_ptr<ProxyConfigurationResolver> ProxyConfigurationResolverProvider::getFromOldSettingsFormat(
Protocol request_protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
)
const Poco::Util::AbstractConfiguration & configuration)
{
/*
* First try to get it from settings only using the combination of config_prefix and configuration.
/* First try to get it from settings only using the combination of config_prefix and configuration.
* This logic exists for backward compatibility with old S3 storage specific proxy configuration.
* */
if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings<false>(request_protocol, config_prefix + ".proxy", configuration))
if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(false, request_protocol, config_prefix + ".proxy", configuration))
{
return resolver;
}
/*
* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings.
/* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings.
* Falls back to Environment resolver if no configuration is found.
* */
return ProxyConfigurationResolverProvider::get(request_protocol, configuration);

View File

@ -33,12 +33,11 @@ public:
);
private:
template <bool is_new_syntax = true>
static std::shared_ptr<ProxyConfigurationResolver> getFromSettings(
bool is_new_syntax,
Protocol protocol,
const String & config_prefix,
const Poco::Util::AbstractConfiguration & configuration
);
const Poco::Util::AbstractConfiguration & configuration);
};
}

View File

@ -18,13 +18,17 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_SET_SIGNAL_HANDLER;
extern const int CANNOT_SEND_SIGNAL;
}
}
extern const char * GIT_HASH;
using namespace DB;
@ -334,7 +338,7 @@ void SignalListener::onTerminate(std::string_view message, UInt32 thread_num) co
size_t pos = message.find('\n');
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", thread_num, message.substr(0, pos));
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, thread_num, message.substr(0, pos));
/// Print trace from std::terminate exception line-by-line to make it easy for grep.
while (pos != std::string_view::npos)
@ -368,7 +372,7 @@ try
LOG_FATAL(log, "########## Short fault info ############");
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
thread_num, sig);
std::string signal_description = "Unknown signal";
@ -434,13 +438,13 @@ try
if (query_id.empty())
{
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
thread_num, signal_description, sig);
}
else
{
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
thread_num, query_id, query, signal_description, sig);
}

View File

@ -51,7 +51,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
std::string contents;
{
ReadBufferFromFile in(path, 1024);
LimitReadBuffer limit_in(in, 1024, /* trow_exception */ false, /* exact_limit */ {});
LimitReadBuffer limit_in(in, 1024, /* throw_exception */ false, /* exact_limit */ {});
readStringUntilEOF(contents, limit_in);
}

View File

@ -85,7 +85,7 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg
/// We use `formatWithPossiblyHidingSensitiveData` instead of `getColumnNameWithoutAlias` because `column->type` is an ASTFunction.
/// `getColumnNameWithoutAlias` will return name of the function with `(arguments)` even if arguments is empty.
if (column)
structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true));
structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false));
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage());
}
@ -102,7 +102,7 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error);
for (size_t i = 0; i < type_list_raw->children.size(); ++i)
structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true));
structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true, false));
}
void BaseExternalTable::initSampleBlock()

View File

@ -23,7 +23,7 @@ namespace postgres
{
PoolWithFailover::PoolWithFailover(
const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority,
const ReplicasConfigurationByPriority & configurations_by_priority,
size_t pool_size,
size_t pool_wait_timeout_,
size_t max_tries_,

View File

@ -8,7 +8,6 @@
#include "ConnectionHolder.h"
#include <mutex>
#include <Poco/Util/AbstractConfiguration.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Storages/StoragePostgreSQL.h>
@ -20,12 +19,12 @@ namespace postgres
class PoolWithFailover
{
using RemoteDescription = std::vector<std::pair<String, uint16_t>>;
public:
using ReplicasConfigurationByPriority = std::map<size_t, std::vector<DB::StoragePostgreSQL::Configuration>>;
using RemoteDescription = std::vector<std::pair<String, uint16_t>>;
PoolWithFailover(
const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority,
const ReplicasConfigurationByPriority & configurations_by_priority,
size_t pool_size,
size_t pool_wait_timeout,
size_t max_tries_,

View File

@ -106,6 +106,8 @@ namespace DB
M(UInt64, dns_cache_max_entries, 10000, "Internal DNS cache max entries.", 0) \
M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \
M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \
M(Bool, dns_allow_resolve_names_to_ipv4, true, "Allows resolve names to ipv4 addresses.", 0) \
M(Bool, dns_allow_resolve_names_to_ipv6, true, "Allows resolve names to ipv6 addresses.", 0) \
\
M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \
M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \

View File

@ -882,7 +882,7 @@ class IColumn;
M(Bool, use_json_alias_for_old_object_type, false, "When enabled, JSON type alias will create old experimental Object type instead of a new JSON type", 0) \
M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \
M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \
M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \
M(Bool, print_pretty_type_names, true, "Print pretty type names in the DESCRIBE query and `toTypeName` function, as well as in the `SHOW CREATE TABLE` query and the `formatQuery` function.", 0) \
M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \
M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0) \
M(SQLSecurityType, default_normal_view_sql_security, SQLSecurityType::INVOKER, "Allows to set a default value for SQL SECURITY option when creating a normal view.", 0) \

View File

@ -16,39 +16,29 @@
#include <sys/resource.h>
#if defined(OS_LINUX)
#include <sys/prctl.h>
#include <sys/prctl.h>
#endif
#include <cerrno>
#include <cstring>
#include <unistd.h>
#include <algorithm>
#include <typeinfo>
#include <iostream>
#include <fstream>
#include <memory>
#include <base/scope_guard.h>
#include <Poco/Message.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <Poco/ErrorHandler.h>
#include <Poco/Pipe.h>
#include <Common/ErrorHandlers.h>
#include <Common/SignalHandlers.h>
#include <base/argsToConfig.h>
#include <base/getThreadId.h>
#include <base/coverage.h>
#include <base/sleep.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/Exception.h>
#include <Common/PipeFDs.h>
#include <Common/StackTrace.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/ClickHouseRevision.h>
#include <Common/Config/ConfigProcessor.h>
@ -459,17 +449,9 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
signal_listener_thread.start(*signal_listener);
#if defined(__ELF__) && !defined(OS_FREEBSD)
String build_id_hex = SymbolIndex::instance().getBuildIDHex();
if (build_id_hex.empty())
build_id = "";
else
build_id = build_id_hex;
#else
build_id = "";
build_id = SymbolIndex::instance().getBuildIDHex();
#endif
git_hash = GIT_HASH;
#if defined(OS_LINUX)
std::string executable_path = getExecutablePath();
@ -482,7 +464,7 @@ void BaseDaemon::logRevision() const
{
logger().information("Starting " + std::string{VERSION_FULL}
+ " (revision: " + std::to_string(ClickHouseRevision::getVersionRevision())
+ ", git hash: " + (git_hash.empty() ? "<unknown>" : git_hash)
+ ", git hash: " + std::string(GIT_HASH)
+ ", build id: " + (build_id.empty() ? "<unknown>" : build_id) + ")"
+ ", PID " + std::to_string(getpid()));
}

View File

@ -165,7 +165,6 @@ protected:
Poco::Util::AbstractConfiguration * last_configuration = nullptr;
String build_id;
String git_hash;
String stored_binary_hash;
bool should_setup_watchdog = false;

View File

@ -1,10 +1,7 @@
configure_file(GitHash.cpp.in GitHash.generated.cpp)
add_library (daemon
BaseDaemon.cpp
GraphiteWriter.cpp
SentryWriter.cpp
GitHash.generated.cpp
)
target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_parsers clickhouse_common_io clickhouse_common_config)

View File

@ -18,13 +18,15 @@
#include <Core/Settings.h>
#include <IO/Operators.h>
#include "config.h"
#if USE_SIMDJSON
#include <Common/JSONParsers/SimdJSONParser.h>
# include <Common/JSONParsers/SimdJSONParser.h>
#elif USE_RAPIDJSON
# include <Common/JSONParsers/RapidJSONParser.h>
#else
# include <Common/JSONParsers/DummyJSONParser.h>
#endif
#if USE_RAPIDJSON
#include <Common/JSONParsers/RapidJSONParser.h>
#endif
#include <Common/JSONParsers/DummyJSONParser.h>
namespace DB
{
@ -105,7 +107,7 @@ SerializationPtr DataTypeObject::doGetDefaultSerialization() const
switch (schema_format)
{
case SchemaFormat::JSON:
#ifdef USE_SIMDJSON
#if USE_SIMDJSON
return std::make_shared<SerializationJSON<SimdJSONParser>>(
std::move(typed_path_serializations),
paths_to_skip,

View File

@ -8,7 +8,6 @@
#include <Common/quoteString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeCustom.h>

View File

@ -115,7 +115,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix(
dynamic_state->max_dynamic_types = column_dynamic.getMaxDynamicTypes();
/// Write max_dynamic_types parameter, because it can differ from the max_dynamic_types
/// that is specified in the Dynamic type (we could decrease it before merge).
writeBinaryLittleEndian(dynamic_state->max_dynamic_types, *stream);
writeVarUInt(dynamic_state->max_dynamic_types, *stream);
dynamic_state->variant_type = variant_info.variant_type;
dynamic_state->variant_names = variant_info.variant_names;
@ -123,7 +123,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix(
/// Write information about variants.
size_t num_variants = dynamic_state->variant_names.size() - 1; /// Don't write shared variant, Dynamic column should always have it.
writeBinaryLittleEndian(num_variants, *stream);
writeVarUInt(num_variants, *stream);
if (settings.data_types_binary_encoding)
{
const auto & variants = assert_cast<const DataTypeVariant &>(*dynamic_state->variant_type).getVariants();
@ -252,11 +252,11 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD
readBinaryLittleEndian(structure_version, *structure_stream);
auto structure_state = std::make_shared<DeserializeBinaryBulkStateDynamicStructure>(structure_version);
/// Read max_dynamic_types parameter.
readBinaryLittleEndian(structure_state->max_dynamic_types, *structure_stream);
readVarUInt(structure_state->max_dynamic_types, *structure_stream);
/// Read information about variants.
DataTypes variants;
size_t num_variants;
readBinaryLittleEndian(num_variants, *structure_stream);
readVarUInt(num_variants, *structure_stream);
variants.reserve(num_variants + 1); /// +1 for shared variant.
if (settings.data_types_binary_encoding)
{

View File

@ -199,7 +199,7 @@ void SerializationObject::serializeBinaryBulkStatePrefix(
auto object_state = std::make_shared<SerializeBinaryBulkStateObject>(serialization_version);
object_state->max_dynamic_paths = column_object.getMaxDynamicPaths();
/// Write max_dynamic_paths parameter.
writeBinaryLittleEndian(object_state->max_dynamic_paths, *stream);
writeVarUInt(object_state->max_dynamic_paths, *stream);
/// Write all dynamic paths in sorted order.
object_state->sorted_dynamic_paths.reserve(dynamic_paths.size());
for (const auto & [path, _] : dynamic_paths)
@ -354,7 +354,7 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationObject::deserializeOb
readBinaryLittleEndian(serialization_version, *structure_stream);
auto structure_state = std::make_shared<DeserializeBinaryBulkStateObjectStructure>(serialization_version);
/// Read max_dynamic_paths parameter.
readBinaryLittleEndian(structure_state->max_dynamic_paths, *structure_stream);
readVarUInt(structure_state->max_dynamic_paths, *structure_stream);
/// Read the sorted list of dynamic paths.
size_t dynamic_paths_size;
readVarUInt(dynamic_paths_size, *structure_stream);

View File

@ -52,7 +52,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/)
{
iterateMetadataFiles(local_context, [this, &local_context](const String & file_name)
iterateMetadataFiles([this, &local_context](const String & file_name)
{
const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4));

View File

@ -12,7 +12,7 @@ class DatabaseLazyIterator;
class Context;
/** Lazy engine of databases.
* Works like DatabaseOrdinary, but stores in memory only the cache.
* Works like DatabaseOrdinary, but stores only recently accessed tables in memory.
* Can be used only with *Log engines.
*/
class DatabaseLazy final : public DatabaseOnDisk

View File

@ -504,7 +504,7 @@ void DatabaseOnDisk::renameTable(
}
/// It returns create table statement (even if table is detached)
/// It returns the create table statement (even if table is detached)
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const
{
ASTPtr ast;
@ -568,14 +568,14 @@ void DatabaseOnDisk::drop(ContextPtr local_context)
assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty());
if (local_context->getSettingsRef().force_remove_data_recursively_on_drop)
{
(void)fs::remove_all(local_context->getPath() + getDataPath());
(void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path);
(void)fs::remove_all(getMetadataPath());
}
else
{
try
{
(void)fs::remove(local_context->getPath() + getDataPath());
(void)fs::remove(std::filesystem::path(getContext()->getPath()) / data_path);
(void)fs::remove(getMetadataPath());
}
catch (const fs::filesystem_error & e)
@ -613,7 +613,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n
}
}
void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const
void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const
{
auto process_tmp_drop_metadata_file = [&](const String & file_name)
{
@ -621,7 +621,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
static const char * tmp_drop_ext = ".sql.tmp_drop";
const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext));
if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name))
if (fs::exists(std::filesystem::path(getContext()->getPath()) / data_path / object_name))
{
fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql");
LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name));
@ -638,7 +638,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
std::vector<std::pair<String, bool>> metadata_files;
fs::directory_iterator dir_end;
for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
for (fs::directory_iterator dir_it(metadata_path); dir_it != dir_end; ++dir_it)
{
String file_name = dir_it->path().filename();
/// For '.svn', '.gitignore' directory and similar.

View File

@ -64,7 +64,7 @@ public:
time_t getObjectMetadataModificationTime(const String & object_name) const override;
String getDataPath() const override { return data_path; }
String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; }
String getTableDataPath(const String & table_name) const override { return std::filesystem::path(data_path) / escapeForFileName(table_name) / ""; }
String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); }
String getMetadataPath() const override { return metadata_path; }
@ -83,7 +83,7 @@ protected:
using IteratingFunction = std::function<void(const String &)>;
void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const;
void iterateMetadataFiles(const IteratingFunction & process_metadata_file) const;
ASTPtr getCreateTableQueryImpl(
const String & table_name,

View File

@ -55,7 +55,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
static constexpr const char * const CONVERT_TO_REPLICATED_FLAG_NAME = "convert_to_replicated";
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_)
: DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_)
: DatabaseOrdinary(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseOrdinary (" + name_ + ")", context_)
{
}
@ -265,7 +265,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
}
};
iterateMetadataFiles(local_context, process_metadata);
iterateMetadataFiles(process_metadata);
size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count;
size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries;

View File

@ -14,6 +14,8 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_TABLE;
}
DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_)
@ -124,6 +126,39 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab
getEngineName());
}
void DatabasesOverlay::renameTable(
ContextPtr current_context,
const String & name,
IDatabase & to_database,
const String & to_name,
bool exchange,
bool dictionary)
{
for (auto & db : databases)
{
if (db->isTableExist(name, current_context))
{
if (DatabasesOverlay * to_overlay_database = typeid_cast<DatabasesOverlay *>(&to_database))
{
/// Renaming from Overlay database inside itself or into another Overlay database.
/// Just use the first database in the overlay as a destination.
if (to_overlay_database->databases.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The destination Overlay database {} does not have any members", to_database.getDatabaseName());
db->renameTable(current_context, name, *to_overlay_database->databases[0], to_name, exchange, dictionary);
}
else
{
/// Renaming into a different type of database. E.g. from Overlay on top of Atomic database into just Atomic database.
db->renameTable(current_context, name, to_database, to_name, exchange, dictionary);
}
return;
}
}
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(getDatabaseName()), backQuote(name));
}
ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const
{
ASTPtr result = nullptr;
@ -178,6 +213,18 @@ String DatabasesOverlay::getTableDataPath(const ASTCreateQuery & query) const
return result;
}
UUID DatabasesOverlay::getUUID() const
{
UUID result = UUIDHelpers::Nil;
for (const auto & db : databases)
{
result = db->getUUID();
if (result != UUIDHelpers::Nil)
break;
}
return result;
}
UUID DatabasesOverlay::tryGetTableUUID(const String & table_name) const
{
UUID result = UUIDHelpers::Nil;

View File

@ -35,12 +35,21 @@ public:
StoragePtr detachTable(ContextPtr context, const String & table_name) override;
void renameTable(
ContextPtr current_context,
const String & name,
IDatabase & to_database,
const String & to_name,
bool exchange,
bool dictionary) override;
ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override;
ASTPtr getCreateDatabaseQuery() const override;
String getTableDataPath(const String & table_name) const override;
String getTableDataPath(const ASTCreateQuery & query) const override;
UUID getUUID() const override;
UUID tryGetTableUUID(const String & table_name) const override;
void drop(ContextPtr context) override;

View File

@ -8,12 +8,12 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Processors/Formats/IInputFormat.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Poco/Net/HTTPRequest.h>
#include <Common/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
#include <Storages/NamedCollectionsHelpers.h>
#include "registerDictionaries.h"
@ -223,21 +223,23 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
String endpoint;
String format;
auto named_collection = created_from_ddl
? getURLBasedDataSourceConfiguration(config, settings_config_prefix, global_context)
: std::nullopt;
auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix, global_context) : nullptr;
if (named_collection)
{
url = named_collection->configuration.url;
endpoint = named_collection->configuration.endpoint;
format = named_collection->configuration.format;
validateNamedCollection(
*named_collection,
/* required_keys */{},
/* optional_keys */ValidateKeysMultiset<ExternalDatabaseEqualKeysSet>{
"url", "endpoint", "user", "credentials.user", "password", "credentials.password", "format", "compression_method", "structure", "name"});
credentials.setUsername(named_collection->configuration.user);
credentials.setPassword(named_collection->configuration.password);
url = named_collection->getOrDefault<String>("url", "");
endpoint = named_collection->getOrDefault<String>("endpoint", "");
format = named_collection->getOrDefault<String>("format", "");
header_entries.reserve(named_collection->configuration.headers.size());
for (const auto & [key, value] : named_collection->configuration.headers)
header_entries.emplace_back(key, value);
credentials.setUsername(named_collection->getAnyOrDefault<String>({"user", "credentials.user"}, ""));
credentials.setPassword(named_collection->getAnyOrDefault<String>({"password", "credentials.password"}, ""));
header_entries = getHeadersFromNamedCollection(*named_collection);
}
else
{

View File

@ -1,15 +1,12 @@
#include "MongoDBDictionarySource.h"
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Storages/StorageMongoDBSocketFactory.h>
#include <Storages/NamedCollectionsHelpers.h>
namespace DB
{
static const std::unordered_set<std::string_view> dictionary_allowed_keys = {
"host", "port", "user", "password", "db", "database", "uri", "collection", "name", "method", "options"};
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
{
auto create_mongo_db_dictionary = [](
@ -22,35 +19,53 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
bool created_from_ddl)
{
const auto config_prefix = root_config_prefix + ".mongodb";
ExternalDataSourceConfiguration configuration;
auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); };
auto named_collection = getExternalDataSourceConfiguration(config, config_prefix, context, has_config_key);
auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, config_prefix, context) : nullptr;
String host, username, password, database, method, options, collection;
UInt16 port;
if (named_collection)
{
configuration = named_collection->configuration;
validateNamedCollection(
*named_collection,
/* required_keys */{"collection"},
/* optional_keys */ValidateKeysMultiset<ExternalDatabaseEqualKeysSet>{
"host", "port", "user", "password", "db", "database", "uri", "name", "method", "options"});
host = named_collection->getOrDefault<String>("host", "");
port = static_cast<UInt16>(named_collection->getOrDefault<UInt64>("port", 0));
username = named_collection->getOrDefault<String>("user", "");
password = named_collection->getOrDefault<String>("password", "");
database = named_collection->getAnyOrDefault<String>({"db", "database"}, "");
method = named_collection->getOrDefault<String>("method", "");
collection = named_collection->getOrDefault<String>("collection", "");
options = named_collection->getOrDefault<String>("options", "");
}
else
{
configuration.host = config.getString(config_prefix + ".host", "");
configuration.port = config.getUInt(config_prefix + ".port", 0);
configuration.username = config.getString(config_prefix + ".user", "");
configuration.password = config.getString(config_prefix + ".password", "");
configuration.database = config.getString(config_prefix + ".db", "");
host = config.getString(config_prefix + ".host", "");
port = config.getUInt(config_prefix + ".port", 0);
username = config.getString(config_prefix + ".user", "");
password = config.getString(config_prefix + ".password", "");
database = config.getString(config_prefix + ".db", "");
method = config.getString(config_prefix + ".method", "");
collection = config.getString(config_prefix + ".collection");
options = config.getString(config_prefix + ".options", "");
}
if (created_from_ddl)
context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port));
context->getRemoteHostFilter().checkHostAndPort(host, toString(port));
return std::make_unique<MongoDBDictionarySource>(dict_struct,
return std::make_unique<MongoDBDictionarySource>(
dict_struct,
config.getString(config_prefix + ".uri", ""),
configuration.host,
configuration.port,
configuration.username,
configuration.password,
config.getString(config_prefix + ".method", ""),
configuration.database,
config.getString(config_prefix + ".collection"),
config.getString(config_prefix + ".options", ""),
host,
port,
username,
password,
method,
database,
collection,
options,
sample_block);
};

View File

@ -4,6 +4,7 @@
#include <Core/QualifiedTableName.h>
#include <Core/Settings.h>
#include "DictionarySourceFactory.h"
#include <Storages/NamedCollectionsHelpers.h>
#include "registerDictionaries.h"
#if USE_LIBPQXX
@ -13,7 +14,6 @@
#include "readInvalidateQuery.h"
#include <Interpreters/Context.h>
#include <QueryPipeline/QueryPipeline.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Common/logger_useful.h>
#endif
@ -24,16 +24,17 @@ namespace DB
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
extern const int BAD_ARGUMENTS;
}
static const ValidateKeysMultiset<ExternalDatabaseEqualKeysSet> dictionary_allowed_keys = {
"host", "port", "user", "password", "db", "database", "table", "schema",
"update_field", "update_lag", "invalidate_query", "query", "where", "name", "priority"};
#if USE_LIBPQXX
static const UInt64 max_block_size = 8192;
static const std::unordered_set<std::string_view> dictionary_allowed_keys = {
"host", "port", "user", "password", "db", "database", "table", "schema",
"update_field", "update_lag", "invalidate_query", "query", "where", "name", "priority"};
namespace
{
ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct, const String & schema, const String & table, const String & query, const String & where)
@ -177,6 +178,19 @@ std::string PostgreSQLDictionarySource::toString() const
return "PostgreSQL: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where);
}
static void validateConfigKeys(
const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
dict_config.keys(config_prefix, config_keys);
for (const auto & config_key : config_keys)
{
if (dictionary_allowed_keys.contains(config_key) || startsWith(config_key, "replica"))
continue;
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key);
}
}
#endif
void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
@ -191,38 +205,117 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
{
#if USE_LIBPQXX
const auto settings_config_prefix = config_prefix + ".postgresql";
auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key) || key.starts_with("replica"); };
auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context, has_config_key);
const auto & settings = context->getSettingsRef();
std::optional<PostgreSQLDictionarySource::Configuration> dictionary_configuration;
postgres::PoolWithFailover::ReplicasConfigurationByPriority replicas_by_priority;
auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix, context) : nullptr;
if (named_collection)
{
validateNamedCollection<ValidateKeysMultiset<ExternalDatabaseEqualKeysSet>>(*named_collection, {}, dictionary_allowed_keys);
StoragePostgreSQL::Configuration common_configuration;
common_configuration.host = named_collection->getOrDefault<String>("host", "");
common_configuration.port = named_collection->getOrDefault<UInt64>("port", 0);
common_configuration.username = named_collection->getOrDefault<String>("user", "");
common_configuration.password = named_collection->getOrDefault<String>("password", "");
common_configuration.database = named_collection->getAnyOrDefault<String>({"database", "db"}, "");
common_configuration.schema = named_collection->getOrDefault<String>("schema", "");
common_configuration.table = named_collection->getOrDefault<String>("table", "");
dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration{
.db = common_configuration.database,
.schema = common_configuration.schema,
.table = common_configuration.table,
.query = named_collection->getOrDefault<String>("query", ""),
.where = named_collection->getOrDefault<String>("where", ""),
.invalidate_query = named_collection->getOrDefault<String>("invalidate_query", ""),
.update_field = named_collection->getOrDefault<String>("update_field", ""),
.update_lag = named_collection->getOrDefault<UInt64>("update_lag", 1),
});
replicas_by_priority[0].emplace_back(common_configuration);
}
else
{
validateConfigKeys(config, settings_config_prefix);
StoragePostgreSQL::Configuration common_configuration;
common_configuration.host = config.getString(settings_config_prefix + ".host", "");
common_configuration.port = config.getUInt(settings_config_prefix + ".port", 0);
common_configuration.username = config.getString(settings_config_prefix + ".user", "");
common_configuration.password = config.getString(settings_config_prefix + ".password", "");
common_configuration.database = config.getString(fmt::format("{}.database", settings_config_prefix), config.getString(fmt::format("{}.db", settings_config_prefix), ""));
common_configuration.schema = config.getString(fmt::format("{}.schema", settings_config_prefix), "");
common_configuration.table = config.getString(fmt::format("{}.table", settings_config_prefix), "");
dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration
{
.db = common_configuration.database,
.schema = common_configuration.schema,
.table = common_configuration.table,
.query = config.getString(fmt::format("{}.query", settings_config_prefix), ""),
.where = config.getString(fmt::format("{}.where", settings_config_prefix), ""),
.invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""),
.update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""),
.update_lag = config.getUInt64(fmt::format("{}.update_lag", settings_config_prefix), 1)
});
if (config.has(settings_config_prefix + ".replica"))
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(settings_config_prefix, config_keys);
for (const auto & config_key : config_keys)
{
if (config_key.starts_with("replica"))
{
String replica_name = settings_config_prefix + "." + config_key;
StoragePostgreSQL::Configuration replica_configuration{common_configuration};
size_t priority = config.getInt(replica_name + ".priority", 0);
replica_configuration.host = config.getString(replica_name + ".host", common_configuration.host);
replica_configuration.port = config.getUInt(replica_name + ".port", common_configuration.port);
replica_configuration.username = config.getString(replica_name + ".user", common_configuration.username);
replica_configuration.password = config.getString(replica_name + ".password", common_configuration.password);
if (replica_configuration.host.empty() || replica_configuration.port == 0
|| replica_configuration.username.empty() || replica_configuration.password.empty())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Named collection of connection parameters is missing some "
"of the parameters and no other dictionary parameters are added");
}
replicas_by_priority[priority].emplace_back(replica_configuration);
}
}
}
else
{
replicas_by_priority[0].emplace_back(common_configuration);
}
}
if (created_from_ddl)
{
for (const auto & replicas : configuration.replicas_configurations)
for (const auto & replica : replicas.second)
for (const auto & [_, replicas] : replicas_by_priority)
for (const auto & replica : replicas)
context->getRemoteHostFilter().checkHostAndPort(replica.host, toString(replica.port));
}
auto pool = std::make_shared<postgres::PoolWithFailover>(
configuration.replicas_configurations,
replicas_by_priority,
settings.postgresql_connection_pool_size,
settings.postgresql_connection_pool_wait_timeout,
settings.postgresql_connection_pool_retries,
settings.postgresql_connection_pool_auto_close_connection,
settings.postgresql_connection_attempt_timeout);
PostgreSQLDictionarySource::Configuration dictionary_configuration
{
.db = configuration.database,
.schema = configuration.schema,
.table = configuration.table,
.query = config.getString(fmt::format("{}.query", settings_config_prefix), ""),
.where = config.getString(fmt::format("{}.where", settings_config_prefix), ""),
.invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""),
.update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""),
.update_lag = config.getUInt64(fmt::format("{}.update_lag", settings_config_prefix), 1)
};
return std::make_unique<PostgreSQLDictionarySource>(dict_struct, dictionary_configuration, pool, sample_block);
return std::make_unique<PostgreSQLDictionarySource>(dict_struct, dictionary_configuration.value(), pool, sample_block);
#else
(void)dict_struct;
(void)config;

View File

@ -80,20 +80,27 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
if (with_file_cache)
{
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
buf = std::make_unique<CachedOnDiskReadBufferFromFile>(
object_path,
cache_key,
settings.remote_fs_cache,
FileCache::getCommonUser(),
[=, this]() { return read_buffer_creator(/* restricted_seek */true, object); },
settings,
query_id,
object.bytes_size,
/* allow_seeks */false,
/* use_external_buffer */true,
/* read_until_position */std::nullopt,
cache_log);
if (settings.remote_fs_cache->isInitialized())
{
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
buf = std::make_unique<CachedOnDiskReadBufferFromFile>(
object_path,
cache_key,
settings.remote_fs_cache,
FileCache::getCommonUser(),
[=, this]() { return read_buffer_creator(/* restricted_seek */true, object); },
settings,
query_id,
object.bytes_size,
/* allow_seeks */false,
/* use_external_buffer */true,
/* read_until_position */std::nullopt,
cache_log);
}
else
{
settings.remote_fs_cache->throwInitExceptionIfNeeded();
}
}
/// Can't wrap CachedOnDiskReadBufferFromFile in CachedInMemoryReadBufferFromFile because the

View File

@ -99,7 +99,7 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
/// Need to remove even if cache_on_write == false.
removeCacheIfExists(object.remote_path);
if (cache_on_write)
if (cache_on_write && cache->isInitialized())
{
auto key = getCacheKey(object.remote_path);
return std::make_unique<CachedOnDiskWriteBufferFromFile>(
@ -122,7 +122,8 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
return;
/// Add try catch?
cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id);
if (cache->isInitialized())
cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id);
}
void CachedObjectStorage::removeObject(const StoredObject & object)

View File

@ -43,6 +43,7 @@ public:
max_query_size = settings.max_query_size;
max_parser_depth = settings.max_parser_depth;
max_parser_backtracks = settings.max_parser_backtracks;
print_pretty_type_names = settings.print_pretty_type_names;
}
String getName() const override { return name; }
@ -138,7 +139,11 @@ private:
}
}
formatAST(*ast, buf, /*hilite*/ false, /*single_line*/ output_formatting == OutputFormatting::SingleLine);
IAST::FormatSettings settings(buf, output_formatting == OutputFormatting::SingleLine, /*hilite*/ false);
settings.show_secrets = true;
settings.print_pretty_type_names = print_pretty_type_names;
ast->format(settings);
auto formatted = buf.stringView();
const size_t res_data_new_size = res_data_size + formatted.size() + 1;
@ -165,6 +170,7 @@ private:
size_t max_query_size;
size_t max_parser_depth;
size_t max_parser_backtracks;
bool print_pretty_type_names;
};
}

View File

@ -238,7 +238,7 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
if (iter == http_header_entries.end())
{
http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING));
http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}{}", VERSION_STRING, VERSION_OFFICIAL));
}
if (!delay_initialization && use_external_buffer)

View File

@ -982,10 +982,10 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT
{
auto context = Context::getGlobalContextInstance();
chassert(context);
auto proxy_configuration_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::protocolFromString(protocol), context->getConfigRef());
auto proxy_configuration_resolver = ProxyConfigurationResolverProvider::get(ProxyConfiguration::protocolFromString(protocol), context->getConfigRef());
auto per_request_configuration = [=] () { return proxy_configuration_resolver->resolve(); };
auto error_report = [=] (const DB::ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); };
auto per_request_configuration = [=]{ return proxy_configuration_resolver->resolve(); };
auto error_report = [=](const ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); };
auto config = PocoHTTPClientConfiguration(
per_request_configuration,

View File

@ -1,4 +1,5 @@
#include <Poco/Timespan.h>
#include <Common/config_version.h>
#include "config.h"
#if USE_AWS_S3
@ -17,6 +18,7 @@
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <IO/S3/ProviderType.h>
#include <Interpreters/Context.h>
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/HttpResponse.h>
@ -29,6 +31,7 @@
#include <boost/algorithm/string.hpp>
static const int SUCCESS_RESPONSE_MIN = 200;
static const int SUCCESS_RESPONSE_MAX = 299;
@ -84,7 +87,7 @@ namespace DB::S3
{
PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
std::function<DB::ProxyConfiguration()> per_request_configuration_,
std::function<ProxyConfiguration()> per_request_configuration_,
const String & force_region_,
const RemoteHostFilter & remote_host_filter_,
unsigned int s3_max_redirects_,
@ -94,7 +97,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
bool s3_use_adaptive_timeouts_,
const ThrottlerPtr & get_request_throttler_,
const ThrottlerPtr & put_request_throttler_,
std::function<void(const DB::ProxyConfiguration &)> error_report_)
std::function<void(const ProxyConfiguration &)> error_report_)
: per_request_configuration(per_request_configuration_)
, force_region(force_region_)
, remote_host_filter(remote_host_filter_)
@ -107,6 +110,8 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
, s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_)
, error_report(error_report_)
{
/// This is used to identify configurations created by us.
userAgent = std::string(VERSION_FULL) + VERSION_OFFICIAL;
}
void PocoHTTPClientConfiguration::updateSchemeAndRegion()
@ -166,6 +171,17 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config
{
}
PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration)
: timeouts(ConnectionTimeouts()
.withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000))
.withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withTCPKeepAliveTimeout(Poco::Timespan(
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))),
remote_host_filter(Context::getGlobalContextInstance()->getRemoteHostFilter())
{
}
std::shared_ptr<Aws::Http::HttpResponse> PocoHTTPClient::MakeRequest(
const std::shared_ptr<Aws::Http::HttpRequest> & request,
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
@ -381,8 +397,11 @@ void PocoHTTPClient::makeRequestInternalImpl(
try
{
const auto proxy_configuration = per_request_configuration();
for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt)
ProxyConfiguration proxy_configuration;
if (per_request_configuration)
proxy_configuration = per_request_configuration();
for (size_t attempt = 0; attempt <= s3_max_redirects; ++attempt)
{
Poco::URI target_uri(uri);
@ -500,7 +519,6 @@ void PocoHTTPClient::makeRequestInternalImpl(
LOG_TEST(log, "Redirecting request to new location: {}", location);
addMetric(request, S3MetricType::Redirects);
continue;
}
@ -548,9 +566,9 @@ void PocoHTTPClient::makeRequestInternalImpl(
}
else
{
if (status_code == 429 || status_code == 503)
{ // API throttling
{
/// API throttling
addMetric(request, S3MetricType::Throttling);
}
else if (status_code >= 300)

View File

@ -20,6 +20,7 @@
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/standard/StandardHttpResponse.h>
namespace Aws::Http::Standard
{
class StandardHttpResponse;
@ -27,18 +28,20 @@ class StandardHttpResponse;
namespace DB
{
class Context;
}
namespace DB::S3
{
class ClientFactory;
class PocoHTTPClient;
struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
{
std::function<DB::ProxyConfiguration()> per_request_configuration;
std::function<ProxyConfiguration()> per_request_configuration;
String force_region;
const RemoteHostFilter & remote_host_filter;
unsigned int s3_max_redirects;
@ -54,13 +57,13 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT;
size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST;
std::function<void(const DB::ProxyConfiguration &)> error_report;
std::function<void(const ProxyConfiguration &)> error_report;
void updateSchemeAndRegion();
private:
PocoHTTPClientConfiguration(
std::function<DB::ProxyConfiguration()> per_request_configuration_,
std::function<ProxyConfiguration()> per_request_configuration_,
const String & force_region_,
const RemoteHostFilter & remote_host_filter_,
unsigned int s3_max_redirects_,
@ -70,13 +73,13 @@ private:
bool s3_use_adaptive_timeouts_,
const ThrottlerPtr & get_request_throttler_,
const ThrottlerPtr & put_request_throttler_,
std::function<void(const DB::ProxyConfiguration &)> error_report_
);
std::function<void(const ProxyConfiguration &)> error_report_);
/// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization.
friend ClientFactory;
};
class PocoHTTPResponse : public Aws::Http::Standard::StandardHttpResponse
{
public:
@ -116,10 +119,12 @@ private:
Aws::Utils::Stream::ResponseStream body_stream;
};
class PocoHTTPClient : public Aws::Http::HttpClient
{
public:
explicit PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration);
explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration);
~PocoHTTPClient() override = default;
std::shared_ptr<Aws::Http::HttpResponse> MakeRequest(
@ -166,14 +171,14 @@ protected:
static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request);
void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const;
std::function<DB::ProxyConfiguration()> per_request_configuration;
std::function<void(const DB::ProxyConfiguration &)> error_report;
std::function<ProxyConfiguration()> per_request_configuration;
std::function<void(const ProxyConfiguration &)> error_report;
ConnectionTimeouts timeouts;
const RemoteHostFilter & remote_host_filter;
unsigned int s3_max_redirects;
unsigned int s3_max_redirects = 0;
bool s3_use_adaptive_timeouts = true;
bool enable_s3_requests_logging;
bool for_disk_s3;
bool enable_s3_requests_logging = false;
bool for_disk_s3 = false;
/// Limits get request per second rate for GET, SELECT and all other requests, excluding throttled by put throttler
/// (i.e. throttles GetObject, HeadObject)

View File

@ -15,7 +15,10 @@ namespace DB::S3
std::shared_ptr<Aws::Http::HttpClient>
PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const
{
return std::make_shared<PocoHTTPClient>(static_cast<const PocoHTTPClientConfiguration &>(client_configuration));
if (client_configuration.userAgent.starts_with("ClickHouse"))
return std::make_shared<PocoHTTPClient>(static_cast<const PocoHTTPClientConfiguration &>(client_configuration));
else /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost.
return std::make_shared<PocoHTTPClient>(client_configuration);
}
std::shared_ptr<Aws::Http::HttpRequest> PocoHTTPClientFactory::CreateHttpRequest(

View File

@ -11,11 +11,15 @@
#include <Interpreters/Cache/EvictionCandidates.h>
#include <Interpreters/Context.h>
#include <base/hex.h>
#include <Common/callOnce.h>
#include <Common/Exception.h>
#include <Common/ThreadPool.h>
#include <Common/ElapsedTimeProfileEventIncrement.h>
#include <Core/ServerUUID.h>
#include <exception>
#include <filesystem>
#include <mutex>
namespace fs = std::filesystem;
@ -88,6 +92,7 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s
, bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0)
, boundary_alignment(settings.boundary_alignment)
, load_metadata_threads(settings.load_metadata_threads)
, load_metadata_asynchronously(settings.load_metadata_asynchronously)
, write_cache_per_user_directory(settings.write_cache_per_user_id_directory)
, keep_current_size_to_max_ratio(1 - settings.keep_free_space_size_ratio)
, keep_current_elements_to_max_ratio(1 - settings.keep_free_space_elements_ratio)
@ -136,7 +141,17 @@ const FileCache::UserInfo & FileCache::getInternalUser()
bool FileCache::isInitialized() const
{
return is_initialized.load(std::memory_order_seq_cst);
return is_initialized;
}
void FileCache::throwInitExceptionIfNeeded()
{
if (load_metadata_asynchronously)
return;
std::lock_guard lock(init_mutex);
if (init_exception)
std::rethrow_exception(init_exception);
}
const String & FileCache::getBasePath() const
@ -170,6 +185,35 @@ void FileCache::assertInitialized() const
}
void FileCache::initialize()
{
// Prevent initialize() from running twice. This may be caused by two cache disks being created with the same path (see integration/test_filesystem_cache).
callOnce(initialize_called, [&] {
bool need_to_load_metadata = fs::exists(getBasePath());
try
{
if (!need_to_load_metadata)
fs::create_directories(getBasePath());
status_file = make_unique<StatusFile>(fs::path(getBasePath()) / "status", StatusFile::write_full_info);
}
catch (...)
{
init_exception = std::current_exception();
tryLogCurrentException(__PRETTY_FUNCTION__);
throw;
}
if (load_metadata_asynchronously)
{
load_metadata_main_thread = ThreadFromGlobalPool([this, need_to_load_metadata] { initializeImpl(need_to_load_metadata); });
}
else
{
initializeImpl(need_to_load_metadata);
}
});
}
void FileCache::initializeImpl(bool load_metadata)
{
std::lock_guard lock(init_mutex);
@ -178,16 +222,10 @@ void FileCache::initialize()
try
{
if (fs::exists(getBasePath()))
{
if (load_metadata)
loadMetadata();
}
else
{
fs::create_directories(getBasePath());
}
status_file = make_unique<StatusFile>(fs::path(getBasePath()) / "status", StatusFile::write_full_info);
metadata.startup();
}
catch (...)
{
@ -196,8 +234,6 @@ void FileCache::initialize()
throw;
}
metadata.startup();
if (keep_current_size_to_max_ratio != 1 || keep_current_elements_to_max_ratio != 1)
{
keep_up_free_space_ratio_task = Context::getGlobalContextInstance()->getSchedulePool().createTask(log->name(), [this] { freeSpaceRatioKeepingThreadFunc(); });
@ -205,6 +241,7 @@ void FileCache::initialize()
}
is_initialized = true;
LOG_TEST(log, "Initialized cache from {}", metadata.getBaseDirectory());
}
CachePriorityGuard::Lock FileCache::lockCache() const
@ -1197,7 +1234,6 @@ void FileCache::loadMetadataImpl()
std::vector<ThreadFromGlobalPool> loading_threads;
std::exception_ptr first_exception;
std::mutex set_exception_mutex;
std::atomic<bool> stop_loading = false;
LOG_INFO(log, "Loading filesystem cache with {} threads from {}", load_metadata_threads, metadata.getBaseDirectory());
@ -1207,7 +1243,7 @@ void FileCache::loadMetadataImpl()
{
loading_threads.emplace_back([&]
{
while (!stop_loading)
while (!stop_loading_metadata)
{
try
{
@ -1224,7 +1260,7 @@ void FileCache::loadMetadataImpl()
if (!first_exception)
first_exception = std::current_exception();
}
stop_loading = true;
stop_loading_metadata = true;
return;
}
}
@ -1237,7 +1273,7 @@ void FileCache::loadMetadataImpl()
if (!first_exception)
first_exception = std::current_exception();
}
stop_loading = true;
stop_loading_metadata = true;
break;
}
}
@ -1424,6 +1460,11 @@ FileCache::~FileCache()
void FileCache::deactivateBackgroundOperations()
{
shutdown.store(true);
stop_loading_metadata = true;
if (load_metadata_main_thread.joinable())
load_metadata_main_thread.join();
metadata.shutdown();
if (keep_up_free_space_ratio_task)
keep_up_free_space_ratio_task->deactivate();

View File

@ -8,6 +8,7 @@
#include <IO/ReadSettings.h>
#include <Common/callOnce.h>
#include <Common/ThreadPool.h>
#include <Common/StatusFile.h>
#include <Interpreters/Cache/LRUFileCachePriority.h>
@ -82,6 +83,9 @@ public:
bool isInitialized() const;
/// Throws if `!load_metadata_asynchronously` and there is an exception in `init_exception`
void throwInitExceptionIfNeeded();
const String & getBasePath() const;
static Key createKeyForPath(const String & path);
@ -199,6 +203,9 @@ private:
const size_t bypass_cache_threshold;
const size_t boundary_alignment;
size_t load_metadata_threads;
const bool load_metadata_asynchronously;
std::atomic<bool> stop_loading_metadata = false;
ThreadFromGlobalPool load_metadata_main_thread;
const bool write_cache_per_user_directory;
BackgroundSchedulePool::TaskHolder keep_up_free_space_ratio_task;
@ -210,6 +217,7 @@ private:
std::exception_ptr init_exception;
std::atomic<bool> is_initialized = false;
OnceFlag initialize_called;
mutable std::mutex init_mutex;
std::unique_ptr<StatusFile> status_file;
std::atomic<bool> shutdown = false;
@ -247,6 +255,8 @@ private:
*/
FileCacheQueryLimitPtr query_limit;
void initializeImpl(bool load_metadata);
void assertInitialized() const;
void assertCacheCorrectness();

View File

@ -65,6 +65,9 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin
if (has("load_metadata_threads"))
load_metadata_threads = get_uint("load_metadata_threads");
if (has("load_metadata_asynchronously"))
load_metadata_asynchronously = get_uint("load_metadata_asynchronously");
if (boundary_alignment > max_file_segment_size)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `boundary_alignment` cannot exceed `max_file_segment_size`");

View File

@ -32,6 +32,7 @@ struct FileCacheSettings
size_t background_download_queue_size_limit = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_QUEUE_SIZE_LIMIT;
size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS;
bool load_metadata_asynchronously = false;
bool write_cache_per_user_id_directory = false;

View File

@ -20,6 +20,7 @@ static Block getSampleBlock()
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "max_size"},
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "max_elements"},
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "max_file_segment_size"},
ColumnWithTypeAndName{std::make_shared<DataTypeUInt8>(), "is_initialized"},
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "boundary_alignment"},
ColumnWithTypeAndName{std::make_shared<DataTypeNumber<UInt8>>(), "cache_on_write_operations"},
ColumnWithTypeAndName{std::make_shared<DataTypeNumber<UInt8>>(), "cache_hits_threshold"},
@ -50,6 +51,7 @@ BlockIO InterpreterDescribeCacheQuery::execute()
res_columns[i++]->insert(settings.max_size);
res_columns[i++]->insert(settings.max_elements);
res_columns[i++]->insert(settings.max_file_segment_size);
res_columns[i++]->insert(cache->isInitialized());
res_columns[i++]->insert(settings.boundary_alignment);
res_columns[i++]->insert(settings.cache_on_write_operations);
res_columns[i++]->insert(settings.cache_hits_threshold);

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
namespace DB
{
@ -45,9 +46,7 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel
// It is flattened later, when we process UNION ALL/DISTINCT.
const auto * last_select = children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
{
InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext();
}
InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(/* ignore_setting_constraints= */ false);
}
void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_)
@ -55,10 +54,20 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta
if (!ast)
return;
/// First apply the outermost settings. Then they could be overridden by deeper settings.
if (const auto * query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
{
if (query_with_output->settings_ast)
InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
if (const auto * create_query = ast->as<ASTCreateQuery>(); create_query && create_query->select)
applySettingsFromSelectWithUnion(create_query->select->as<ASTSelectWithUnionQuery &>(), context_);
}
if (const auto * select_query = ast->as<ASTSelectQuery>())
{
if (auto new_settings = select_query->settings())
InterpreterSetQuery(new_settings, context_).executeForCurrentContext();
InterpreterSetQuery(new_settings, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
}
else if (const auto * select_with_union_query = ast->as<ASTSelectWithUnionQuery>())
{
@ -67,28 +76,15 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta
else if (const auto * explain_query = ast->as<ASTExplainQuery>())
{
if (explain_query->settings_ast)
InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext();
InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
applySettingsFromQuery(explain_query->getExplainedQuery(), context_);
}
else if (const auto * query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
{
if (query_with_output->settings_ast)
InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext();
if (const auto * create_query = ast->as<ASTCreateQuery>())
{
if (create_query->select)
{
applySettingsFromSelectWithUnion(create_query->select->as<ASTSelectWithUnionQuery &>(), context_);
}
}
}
else if (auto * insert_query = ast->as<ASTInsertQuery>())
{
context_->setInsertFormat(insert_query->format);
if (insert_query->settings_ast)
InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext();
InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false);
}
}

View File

@ -23,7 +23,7 @@ public:
/** Set setting for current context (query context).
* It is used for interpretation of SETTINGS clause in SELECT query.
*/
void executeForCurrentContext(bool ignore_setting_constraints = false);
void executeForCurrentContext(bool ignore_setting_constraints);
bool supportsTransactions() const override { return true; }

View File

@ -97,7 +97,12 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl()
}
MutableColumnPtr column = ColumnString::create();
column->insert(format({.ctx = getContext(), .query = *create_query, .one_line = false}));
column->insert(format(
{
.ctx = getContext(),
.query = *create_query,
.one_line = false
}));
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{
std::move(column),

View File

@ -27,7 +27,6 @@ class ASTQueryWithTableAndOutput;
class ASTTableIdentifier;
class Context;
// TODO(ilezhankin): refactor and merge |ASTTableIdentifier|
struct StorageID
{
String database_name;

View File

@ -65,7 +65,7 @@ TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, Cu
std::unique_ptr<WriteBufferFromFileBase> TemporaryDataOnDisk::createRawStream(size_t max_file_size)
{
if (file_cache)
if (file_cache && file_cache->isInitialized())
{
auto holder = createCacheFile(max_file_size);
return std::make_unique<WriteBufferToFileSegment>(std::move(holder));
@ -81,7 +81,7 @@ std::unique_ptr<WriteBufferFromFileBase> TemporaryDataOnDisk::createRawStream(si
TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size)
{
if (file_cache)
if (file_cache && file_cache->isInitialized())
{
auto holder = createCacheFile(max_file_size);

View File

@ -786,7 +786,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Verify that AST formatting is consistent:
/// If you format AST, parse it back, and format it again, you get the same string.
String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true);
String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false);
/// The query can become more verbose after formatting, so:
size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0;
@ -811,7 +811,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
chassert(ast2);
String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true);
String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false);
if (formatted1 != formatted2)
throw Exception(ErrorCodes::LOGICAL_ERROR,

View File

@ -25,7 +25,8 @@ inline String format(const SecretHidingFormatSettings & settings)
&& settings.ctx->getSettingsRef().format_display_secrets_in_show_and_select
&& settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect);
return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets);
return settings.query.formatWithPossiblyHidingSensitiveData(
settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names);
}
}

View File

@ -7,6 +7,7 @@
#include <algorithm>
#include <numeric>
#include <thread>
#include <chrono>
#include <Core/ServerUUID.h>
#include <Common/iota.h>
@ -42,6 +43,7 @@
#include <Interpreters/DatabaseCatalog.h>
#include <base/scope_guard.h>
using namespace std::chrono_literals;
namespace fs = std::filesystem;
using namespace DB;
@ -360,9 +362,11 @@ TEST_F(FileCacheTest, LRUPolicy)
settings.max_size = 30;
settings.max_elements = 5;
settings.boundary_alignment = 1;
settings.load_metadata_asynchronously = false;
const size_t file_size = INT_MAX; // the value doesn't really matter because boundary_alignment == 1.
const auto user = FileCache::getCommonUser();
{
std::cerr << "Step 1\n";
@ -817,6 +821,7 @@ TEST_F(FileCacheTest, writeBuffer)
settings.max_elements = 5;
settings.max_file_segment_size = 5;
settings.base_path = cache_base_path;
settings.load_metadata_asynchronously = false;
FileCache cache("6", settings);
cache.initialize();
@ -948,6 +953,7 @@ TEST_F(FileCacheTest, temporaryData)
settings.max_size = 10_KiB;
settings.max_file_segment_size = 1_KiB;
settings.base_path = cache_base_path;
settings.load_metadata_asynchronously = false;
DB::FileCache file_cache("7", settings);
file_cache.initialize();
@ -1076,6 +1082,7 @@ TEST_F(FileCacheTest, CachedReadBuffer)
settings.max_size = 30;
settings.max_elements = 10;
settings.boundary_alignment = 1;
settings.load_metadata_asynchronously = false;
ReadSettings read_settings;
read_settings.enable_filesystem_cache = true;
@ -1095,6 +1102,7 @@ TEST_F(FileCacheTest, CachedReadBuffer)
auto cache = std::make_shared<DB::FileCache>("8", settings);
cache->initialize();
auto key = cache->createKeyForPath(file_path);
const auto user = FileCache::getCommonUser();
@ -1135,6 +1143,7 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize)
settings.max_size = 10_KiB;
settings.max_file_segment_size = 1_KiB;
settings.base_path = cache_base_path;
settings.load_metadata_asynchronously = false;
DB::FileCache file_cache("cache", settings);
file_cache.initialize();
@ -1198,6 +1207,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
settings.max_size = 40;
settings.max_elements = 6;
settings.boundary_alignment = 1;
settings.load_metadata_asynchronously = false;
settings.cache_policy = "SLRU";
settings.slru_size_ratio = 0.5;
@ -1310,6 +1320,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
settings2.boundary_alignment = 1;
settings2.cache_policy = "SLRU";
settings2.slru_size_ratio = 0.5;
settings.load_metadata_asynchronously = false;
auto cache = std::make_shared<DB::FileCache>("slru_2", settings2);
cache->initialize();

View File

@ -66,17 +66,13 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo
{
frame.need_parens = false;
/// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query.
/// We have to always backquote column names to avoid ambiguity with INDEX and other declarations in CREATE query.
format_settings.ostr << backQuote(name);
if (type)
{
format_settings.ostr << ' ';
FormatStateStacked type_frame = frame;
type_frame.indent = 0;
type->formatImpl(format_settings, state, type_frame);
type->formatImpl(format_settings, state, frame);
}
if (null_modifier)

View File

@ -40,12 +40,22 @@ void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & stat
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
if (!settings.one_line && settings.print_pretty_type_names && name == "Tuple")
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, frame);
++frame.indent;
std::string indent_str = settings.one_line ? "" : "\n" + std::string(4 * frame.indent, ' ');
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << ',';
settings.ostr << indent_str;
arguments->children[i]->formatImpl(settings, state, frame);
}
}
else
{
frame.expression_list_prepend_whitespace = false;
arguments->formatImpl(settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_function : "") << ')';

View File

@ -42,7 +42,8 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState
void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = "\n" + std::string(4 * (frame.indent + 1), ' ');
++frame.indent;
std::string indent_str = "\n" + std::string(4 * frame.indent, ' ');
if (frame.expression_list_prepend_whitespace)
{
@ -50,8 +51,6 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
settings.ostr << ' ';
}
++frame.indent;
for (size_t i = 0, size = children.size(); i < size; ++i)
{
if (i && separator)

View File

@ -23,12 +23,8 @@ ASTPtr ASTNameTypePair::clone() const
void ASTNameTypePair::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << indent_str << backQuoteIfNeed(name) << ' ';
settings.ostr << backQuoteIfNeed(name) << ' ';
type->formatImpl(settings, state, frame);
}
}

View File

@ -853,9 +853,10 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
/// Parse numbers (including decimals), strings, arrays and tuples of them.
Pos begin = pos;
const char * data_begin = pos->begin;
const char * data_end = pos->end;
bool is_string_literal = pos->type == StringLiteral;
ASTPtr string_literal;
if (pos->type == Minus)
{
@ -866,10 +867,15 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
data_end = pos->end;
++pos;
}
else if (pos->type == Number || is_string_literal)
else if (pos->type == Number)
{
++pos;
}
else if (pos->type == StringLiteral)
{
if (!ParserStringLiteral().parse(begin, string_literal, expected))
return false;
}
else if (isOneOf<OpeningSquareBracket, OpeningRoundBracket>(pos->type))
{
TokenType last_token = OpeningSquareBracket;
@ -937,20 +943,18 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (ParserToken(DoubleColon).ignore(pos, expected)
&& ParserDataType().parse(pos, type_ast, expected))
{
String s;
size_t data_size = data_end - data_begin;
if (is_string_literal)
if (string_literal)
{
ReadBufferFromMemory buf(data_begin, data_size);
readQuotedStringWithSQLStyle(s, buf);
assert(buf.count() == data_size);
node = createFunctionCast(string_literal, type_ast);
return true;
}
else
s = String(data_begin, data_size);
auto literal = std::make_shared<ASTLiteral>(std::move(s));
node = createFunctionCast(literal, type_ast);
return true;
{
auto literal = std::make_shared<ASTLiteral>(String(data_begin, data_size));
node = createFunctionCast(literal, type_ast);
return true;
}
}
return false;

View File

@ -165,11 +165,12 @@ size_t IAST::checkDepthImpl(size_t max_depth) const
return res;
}
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const
{
WriteBufferFromOwnString buf;
FormatSettings settings(buf, one_line);
settings.show_secrets = show_secrets;
settings.print_pretty_type_names = print_pretty_type_names;
format(settings);
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
}

View File

@ -201,6 +201,7 @@ public:
bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys).
char nl_or_ws; /// Newline or whitespace.
LiteralEscapingStyle literal_escaping_style;
bool print_pretty_type_names;
explicit FormatSettings(
WriteBuffer & ostr_,
@ -209,7 +210,8 @@ public:
bool always_quote_identifiers_ = false,
IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks,
bool show_secrets_ = true,
LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular)
LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular,
bool print_pretty_type_names_ = false)
: ostr(ostr_)
, one_line(one_line_)
, hilite(hilite_)
@ -218,6 +220,7 @@ public:
, show_secrets(show_secrets_)
, nl_or_ws(one_line ? ' ' : '\n')
, literal_escaping_style(literal_escaping_style_)
, print_pretty_type_names(print_pretty_type_names_)
{
}
@ -230,6 +233,7 @@ public:
, show_secrets(other.show_secrets)
, nl_or_ws(other.nl_or_ws)
, literal_escaping_style(other.literal_escaping_style)
, print_pretty_type_names(other.print_pretty_type_names)
{
}
@ -251,7 +255,7 @@ public:
/// The state that is copied when each node is formatted. For example, nesting level.
struct FormatStateStacked
{
UInt8 indent = 0;
UInt16 indent = 0;
bool need_parens = false;
bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element.
bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required)
@ -274,7 +278,7 @@ public:
/// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
/// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const;
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const;
/** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* behaviour is due to the fact such functions are called from Client which knows nothing about
@ -283,12 +287,12 @@ public:
*/
String formatForLogging(size_t max_length = 0) const
{
return formatWithPossiblyHidingSensitiveData(max_length, true, false);
return formatWithPossiblyHidingSensitiveData(max_length, true, false, false);
}
String formatForErrorMessage() const
{
return formatWithPossiblyHidingSensitiveData(0, true, false);
return formatWithPossiblyHidingSensitiveData(0, true, false, false);
}
virtual bool hasSecretParts() const { return childrenHaveSecretParts(); }

View File

@ -25,7 +25,6 @@
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/ParserWatchQuery.h>
#include <Parsers/ParserDescribeCacheQuery.h>
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>
#include <Parsers/Access/ParserShowAccessEntitiesQuery.h>
#include <Parsers/Access/ParserShowAccessQuery.h>
#include <Parsers/Access/ParserShowCreateAccessEntityQuery.h>
@ -152,37 +151,55 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
/// These two sections are allowed in an arbitrary order.
ParserKeyword s_format(Keyword::FORMAT);
if (s_format.ignore(pos, expected))
{
ParserIdentifier format_p;
if (!format_p.parse(pos, query_with_output.format, expected))
return false;
setIdentifierSpecial(query_with_output.format);
query_with_output.children.push_back(query_with_output.format);
}
// SETTINGS key1 = value1, key2 = value2, ...
ParserKeyword s_settings(Keyword::SETTINGS);
if (!query_with_output.settings_ast && s_settings.ignore(pos, expected))
{
ParserSetQuery parser_settings(true);
if (!parser_settings.parse(pos, query_with_output.settings_ast, expected))
return false;
query_with_output.children.push_back(query_with_output.settings_ast);
// SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery)
// Pass them manually, to apply in InterpreterSelectQuery::initSettings()
if (query->as<ASTSelectWithUnionQuery>())
/** Why: let's take the following example:
* SELECT 1 UNION ALL SELECT 2 FORMAT TSV
* Each subquery can be put in parentheses and have its own settings:
* (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV
* And the whole query can have settings:
* (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV SETTINGS e=f
* A single query with output is parsed in the same way as the UNION ALL chain:
* SELECT 1 SETTINGS a=b FORMAT TSV SETTINGS e=f
* So while these forms have a slightly different meaning, they both exist:
* SELECT 1 SETTINGS a=b FORMAT TSV
* SELECT 1 FORMAT TSV SETTINGS e=f
* And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order.
* But while this work:
* (SELECT 1) UNION ALL (SELECT 2) FORMAT TSV SETTINGS d=f
* This does not work automatically, unless we explicitly allow different orders:
* (SELECT 1) UNION ALL (SELECT 2) SETTINGS d=f FORMAT TSV
* Inevitably, we also allow this:
* SELECT 1 SETTINGS a=b SETTINGS d=f FORMAT TSV
* ^^^^^^^^^^^^^^^^^^^^^
* Because this part is consumed into ASTSelectWithUnionQuery
* and the rest into ASTQueryWithOutput.
*/
for (size_t i = 0; i < 2; ++i)
{
if (!query_with_output.format && s_format.ignore(pos, expected))
{
auto settings = query_with_output.settings_ast->clone();
assert_cast<ASTSetQuery *>(settings.get())->print_in_format = false;
QueryWithOutputSettingsPushDownVisitor::Data data{settings};
QueryWithOutputSettingsPushDownVisitor(data).visit(query);
ParserIdentifier format_p;
if (!format_p.parse(pos, query_with_output.format, expected))
return false;
setIdentifierSpecial(query_with_output.format);
query_with_output.children.push_back(query_with_output.format);
}
else if (!query_with_output.settings_ast && s_settings.ignore(pos, expected))
{
// SETTINGS key1 = value1, key2 = value2, ...
ParserSetQuery parser_settings(true);
if (!parser_settings.parse(pos, query_with_output.settings_ast, expected))
return false;
query_with_output.children.push_back(query_with_output.settings_ast);
}
else
break;
}
node = std::move(query);

View File

@ -1,56 +0,0 @@
#include <Common/SettingsChanges.h>
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTSubquery.h>
#include <iterator>
#include <algorithm>
namespace DB
{
bool QueryWithOutputSettingsPushDownMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
{
if (node->as<ASTSelectWithUnionQuery>())
return true;
if (node->as<ASTSubquery>())
return true;
if (child->as<ASTSelectQuery>())
return true;
return false;
}
void QueryWithOutputSettingsPushDownMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * select_query = ast->as<ASTSelectQuery>())
visit(*select_query, ast, data);
}
void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query, ASTPtr &, Data & data)
{
ASTPtr select_settings_ast = select_query.settings();
if (!select_settings_ast)
{
select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, data.settings_ast->clone());
return;
}
SettingsChanges & select_settings = select_settings_ast->as<ASTSetQuery &>().changes;
SettingsChanges & settings = data.settings_ast->as<ASTSetQuery &>().changes;
for (auto & setting : settings)
{
auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting)
{
return select_setting.name == setting.name;
});
if (it == select_settings.end())
select_settings.push_back(setting);
else
it->value = setting.value;
}
}
}

View File

@ -1,39 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTSelectQuery;
struct SettingChange;
class SettingsChanges;
/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query:
/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput)
///
/// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 ->
/// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1
///
/// Otherwise settings after FORMAT will not be applied.
class QueryWithOutputSettingsPushDownMatcher
{
public:
using Visitor = InDepthNodeVisitor<QueryWithOutputSettingsPushDownMatcher, true>;
struct Data
{
const ASTPtr & settings_ast;
};
static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
static void visit(ASTPtr & ast, Data & data);
private:
static void visit(ASTSelectQuery &, ASTPtr &, Data &);
};
using QueryWithOutputSettingsPushDownVisitor = QueryWithOutputSettingsPushDownMatcher::Visitor;
}

View File

@ -1,288 +0,0 @@
#include "ExternalDataSourceConfiguration.h"
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS)
static const std::unordered_set<std::string_view> dictionary_allowed_keys = {
"host", "port", "user", "password", "quota_key", "db",
"database", "table", "schema", "replica",
"update_field", "update_lag", "invalidate_query", "query",
"where", "name", "secure", "uri", "collection"};
template<typename T>
SettingsChanges getSettingsChangesFromConfig(
const BaseSettings<T> & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
SettingsChanges config_settings;
for (const auto & setting : settings.all())
{
const auto & setting_name = setting.getName();
auto setting_value = config.getString(config_prefix + '.' + setting_name, "");
if (!setting_value.empty())
config_settings.emplace_back(setting_name, setting_value);
}
return config_settings;
}
String ExternalDataSourceConfiguration::toString() const
{
WriteBufferFromOwnString configuration_info;
configuration_info << "username: " << username << "\t";
if (addresses.empty())
{
configuration_info << "host: " << host << "\t";
configuration_info << "port: " << port << "\t";
}
else
{
for (const auto & [replica_host, replica_port] : addresses)
{
configuration_info << "host: " << replica_host << "\t";
configuration_info << "port: " << replica_port << "\t";
}
}
return configuration_info.str();
}
void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration & conf)
{
host = conf.host;
port = conf.port;
username = conf.username;
password = conf.password;
quota_key = conf.quota_key;
database = conf.database;
table = conf.table;
schema = conf.schema;
addresses = conf.addresses;
addresses_expr = conf.addresses_expr;
}
static void validateConfigKeys(
const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, HasConfigKeyFunc has_config_key_func)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
dict_config.keys(config_prefix, config_keys);
for (const auto & config_key : config_keys)
{
if (!has_config_key_func(config_key))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key);
}
}
template <typename T>
std::optional<ExternalDataSourceInfo> getExternalDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix,
ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings<T> & settings)
{
validateConfigKeys(dict_config, dict_config_prefix, has_config_key);
ExternalDataSourceConfiguration configuration;
auto collection_name = dict_config.getString(dict_config_prefix + ".name", "");
if (!collection_name.empty())
{
const auto & config = context->getConfigRef();
const auto & collection_prefix = fmt::format("named_collections.{}", collection_name);
validateConfigKeys(dict_config, collection_prefix, has_config_key);
auto config_settings = getSettingsChangesFromConfig(settings, config, collection_prefix);
auto dict_settings = getSettingsChangesFromConfig(settings, dict_config, dict_config_prefix);
/// dictionary config settings override collection settings.
config_settings.insert(config_settings.end(), dict_settings.begin(), dict_settings.end());
if (!config.has(collection_prefix))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name);
configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(collection_prefix + ".host", ""));
configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0));
configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", ""));
configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", ""));
configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", config.getString(collection_prefix + ".quota_key", ""));
configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database",
config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", ""))));
configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", ""));
configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", ""));
if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.table.empty())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Named collection of connection parameters is missing some "
"of the parameters and dictionary parameters are not added");
}
return ExternalDataSourceInfo{.configuration = configuration, .settings_changes = config_settings};
}
return std::nullopt;
}
std::optional<URLBasedDataSourceConfig> getURLBasedDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context)
{
URLBasedDataSourceConfiguration configuration;
auto collection_name = dict_config.getString(dict_config_prefix + ".name", "");
if (!collection_name.empty())
{
const auto & config = context->getConfigRef();
const auto & collection_prefix = fmt::format("named_collections.{}", collection_name);
if (!config.has(collection_prefix))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name);
configuration.url =
dict_config.getString(dict_config_prefix + ".url", config.getString(collection_prefix + ".url", ""));
configuration.endpoint =
dict_config.getString(dict_config_prefix + ".endpoint", config.getString(collection_prefix + ".endpoint", ""));
configuration.format =
dict_config.getString(dict_config_prefix + ".format", config.getString(collection_prefix + ".format", ""));
configuration.compression_method =
dict_config.getString(dict_config_prefix + ".compression", config.getString(collection_prefix + ".compression_method", ""));
configuration.structure =
dict_config.getString(dict_config_prefix + ".structure", config.getString(collection_prefix + ".structure", ""));
configuration.user =
dict_config.getString(dict_config_prefix + ".credentials.user", config.getString(collection_prefix + ".credentials.user", ""));
configuration.password =
dict_config.getString(dict_config_prefix + ".credentials.password", config.getString(collection_prefix + ".credentials.password", ""));
String headers_prefix;
const Poco::Util::AbstractConfiguration *headers_config = nullptr;
if (dict_config.has(dict_config_prefix + ".headers"))
{
headers_prefix = dict_config_prefix + ".headers";
headers_config = &dict_config;
}
else
{
headers_prefix = collection_prefix + ".headers";
headers_config = &config;
}
if (headers_config)
{
Poco::Util::AbstractConfiguration::Keys header_keys;
headers_config->keys(headers_prefix, header_keys);
headers_prefix += ".";
for (const auto & header : header_keys)
{
const auto header_prefix = headers_prefix + header;
configuration.headers.emplace_back(
headers_config->getString(header_prefix + ".name"),
headers_config->getString(header_prefix + ".value"));
}
}
return URLBasedDataSourceConfig{ .configuration = configuration };
}
return std::nullopt;
}
ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key)
{
validateConfigKeys(dict_config, dict_config_prefix, has_config_key);
ExternalDataSourceConfiguration common_configuration;
auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context, has_config_key);
if (named_collection)
{
common_configuration = named_collection->configuration;
}
else
{
common_configuration.host = dict_config.getString(dict_config_prefix + ".host", "");
common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0);
common_configuration.username = dict_config.getString(dict_config_prefix + ".user", "");
common_configuration.password = dict_config.getString(dict_config_prefix + ".password", "");
common_configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", "");
common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", ""));
common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), "");
common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), "");
}
ExternalDataSourcesByPriority configuration
{
.database = common_configuration.database,
.table = common_configuration.table,
.schema = common_configuration.schema,
.replicas_configurations = {}
};
if (dict_config.has(dict_config_prefix + ".replica"))
{
Poco::Util::AbstractConfiguration::Keys config_keys;
dict_config.keys(dict_config_prefix, config_keys);
for (const auto & config_key : config_keys)
{
if (config_key.starts_with("replica"))
{
ExternalDataSourceConfiguration replica_configuration(common_configuration);
String replica_name = dict_config_prefix + "." + config_key;
validateConfigKeys(dict_config, replica_name, has_config_key);
size_t priority = dict_config.getInt(replica_name + ".priority", 0);
replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host);
replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port);
replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username);
replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password);
replica_configuration.quota_key = dict_config.getString(replica_name + ".quota_key", common_configuration.quota_key);
if (replica_configuration.host.empty() || replica_configuration.port == 0
|| replica_configuration.username.empty() || replica_configuration.password.empty())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Named collection of connection parameters is missing some "
"of the parameters and no other dictionary parameters are added");
}
configuration.replicas_configurations[priority].emplace_back(replica_configuration);
}
}
}
else
{
configuration.replicas_configurations[0].emplace_back(common_configuration);
}
return configuration;
}
void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration & conf)
{
url = conf.url;
format = conf.format;
compression_method = conf.compression_method;
structure = conf.structure;
http_method = conf.http_method;
headers = conf.headers;
}
template
std::optional<ExternalDataSourceInfo> getExternalDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix,
ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings<EmptySettingsTraits> & settings);
template
SettingsChanges getSettingsChangesFromConfig(
const BaseSettings<EmptySettingsTraits> & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
}

View File

@ -1,92 +0,0 @@
#pragma once
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <IO/S3Settings.h>
#include <IO/HTTPHeaderEntries.h>
namespace DB
{
#define EMPTY_SETTINGS(M, ALIAS)
DECLARE_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS)
struct EmptySettings : public BaseSettings<EmptySettingsTraits> {};
struct ExternalDataSourceConfiguration
{
String host;
UInt16 port = 0;
String username = "default";
String password;
String quota_key;
String database;
String table;
String schema;
std::vector<std::pair<String, UInt16>> addresses; /// Failover replicas.
String addresses_expr;
String toString() const;
void set(const ExternalDataSourceConfiguration & conf);
};
using StorageSpecificArgs = std::vector<std::pair<String, ASTPtr>>;
struct ExternalDataSourceInfo
{
ExternalDataSourceConfiguration configuration;
SettingsChanges settings_changes;
};
using HasConfigKeyFunc = std::function<bool(const String &)>;
template <typename T = EmptySettingsTraits>
std::optional<ExternalDataSourceInfo> getExternalDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix,
ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings<T> & settings = {});
/// Highest priority is 0, the bigger the number in map, the less the priority.
using ExternalDataSourcesConfigurationByPriority = std::map<size_t, std::vector<ExternalDataSourceConfiguration>>;
struct ExternalDataSourcesByPriority
{
String database;
String table;
String schema;
ExternalDataSourcesConfigurationByPriority replicas_configurations;
};
ExternalDataSourcesByPriority
getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key);
struct URLBasedDataSourceConfiguration
{
String url;
String endpoint;
String format = "auto";
String compression_method = "auto";
String structure = "auto";
String user;
String password;
HTTPHeaderEntries headers;
String http_method;
void set(const URLBasedDataSourceConfiguration & conf);
};
struct URLBasedDataSourceConfig
{
URLBasedDataSourceConfiguration configuration;
};
std::optional<URLBasedDataSourceConfig> getURLBasedDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context);
}

View File

@ -395,6 +395,10 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_
part_stat.prefetch_step_marks = std::max(part_stat.prefetch_step_marks, per_part_infos[i]->min_marks_per_task);
if (part_stat.prefetch_step_marks == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
LOG_DEBUG(
log,
"Part: {}, sum_marks: {}, approx mark size: {}, prefetch_step_bytes: {}, prefetch_step_marks: {}, (ranges: {})",

View File

@ -24,6 +24,7 @@ namespace ErrorCodes
{
extern const int CANNOT_SCHEDULE_TASK;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
MergeTreeReadPool::MergeTreeReadPool(
@ -235,6 +236,10 @@ void MergeTreeReadPool::fillPerThreadInfo(size_t threads, size_t sum_marks)
const auto part_idx = current_parts.back().part_idx;
const auto min_marks_per_task = per_part_infos[part_idx]->min_marks_per_task;
if (min_marks_per_task == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
/// Do not get too few rows from part.
if (marks_in_part >= min_marks_per_task && need_marks < min_marks_per_task)
need_marks = min_marks_per_task;

View File

@ -85,6 +85,7 @@ static size_t calculateMinMarksPerTask(
min_marks_per_task = heuristic_min_marks;
}
}
LOG_TEST(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task);
return min_marks_per_task;
}

View File

@ -8,6 +8,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
@ -38,6 +39,10 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
for (const auto & info : per_part_infos)
min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task);
if (min_marks_per_task == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
extension.all_callback(
InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica));
}

View File

@ -6,6 +6,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrder(
@ -37,6 +38,10 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd
for (const auto & info : per_part_infos)
min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task);
if (min_marks_per_task == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
for (const auto & part : parts_ranges)
request.push_back({part.data_part->info, MarkRanges{}});

View File

@ -1004,6 +1004,10 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init
ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request)
{
if (request.min_number_of_marks == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasHandleRequestMicroseconds);
std::lock_guard lock(mutex);

View File

@ -7,7 +7,6 @@
#include <Interpreters/Context.h>
#include <Common/FailPoint.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/randomSeed.h>
#include <Core/ServerUUID.h>
#include <boost/algorithm/string/replace.hpp>
@ -49,6 +48,20 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage
task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); });
}
void ReplicatedMergeTreeRestartingThread::start(bool schedule)
{
LOG_TRACE(log, "Starting the restating thread, schedule: {}", schedule);
if (schedule)
task->activateAndSchedule();
else
task->activate();
}
void ReplicatedMergeTreeRestartingThread::wakeup()
{
task->schedule();
}
void ReplicatedMergeTreeRestartingThread::run()
{
if (need_stop)

View File

@ -24,16 +24,9 @@ class ReplicatedMergeTreeRestartingThread
public:
explicit ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_);
void start(bool schedule = true)
{
LOG_TRACE(log, "Starting restating thread, schedule: {}", schedule);
if (schedule)
task->activateAndSchedule();
else
task->activate();
}
void start(bool schedule);
void wakeup() { task->schedule(); }
void wakeup();
void shutdown(bool part_of_full_shutdown);

View File

@ -133,7 +133,7 @@ void validateNamedCollection(
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Unexpected key {} in named collection. Required keys: {}, optional keys: {}",
"Unexpected key `{}` in named collection. Required keys: {}, optional keys: {}",
backQuoteIfNeed(key), fmt::join(required_keys, ", "), fmt::join(optional_keys, ", "));
}
}

View File

@ -8,8 +8,6 @@
namespace DB
{
struct ExternalDataSourceConfiguration;
/// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas.
/// This class unites multiple storages with replicas into multiple shards with replicas.
/// A query to external database is passed to one replica on each shard, the result is united.

View File

@ -5194,17 +5194,16 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread)
startBeingLeader();
/// Activate replica in a separate thread if we are not calling from attach thread
restarting_thread.start(/*schedule=*/!from_attach_thread);
if (from_attach_thread)
{
LOG_TRACE(log, "Trying to startup table from right now");
/// Try activating replica in current thread.
/// Try activating replica in the current thread.
restarting_thread.run();
restarting_thread.start(false);
}
else
{
restarting_thread.start(true);
/// Wait while restarting_thread finishing initialization.
/// NOTE It does not mean that replication is actually started after receiving this event.
/// It only means that an attempt to startup replication was made.
@ -5225,7 +5224,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread)
session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]()
{
LOG_TEST(log, "Received event for expired session. Waking up restarting thread");
restarting_thread.start();
restarting_thread.start(true);
});
startBackgroundMovesIfNeeded();
@ -5294,7 +5293,6 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown()
LOG_TRACE(log, "The attach thread is shutdown");
}
restarting_thread.shutdown(/* part_of_full_shutdown */true);
/// Explicitly set the event, because the restarting thread will not set it again
startup_event.set();
@ -6342,7 +6340,7 @@ void StorageReplicatedMergeTree::alter(
"Metadata on replica is not up to date with common metadata in Zookeeper. "
"It means that this replica still not applied some of previous alters."
" Probably too many alters executing concurrently (highly not recommended). "
"You can retry this error");
"You can retry the query");
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if (query_context->getZooKeeperMetadataTransaction())

View File

@ -47,6 +47,9 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
for (const auto & [cache_name, cache_data] : caches)
{
const auto & cache = cache_data->cache;
if (!cache->isInitialized())
continue;
cache->iterate([&](const FileSegment::Info & file_segment)
{
size_t i = 0;

View File

@ -21,6 +21,7 @@ ColumnsDescription StorageSystemFilesystemCacheSettings::getColumnsDescription()
{"path", std::make_shared<DataTypeString>(), "Cache directory"},
{"max_size", std::make_shared<DataTypeUInt64>(), "Cache size limit by the number of bytes"},
{"max_elements", std::make_shared<DataTypeUInt64>(), "Cache size limit by the number of elements"},
{"is_initialized", std::make_shared<DataTypeUInt8>(), "Whether the cache is initialized and ready to be used"},
{"current_size", std::make_shared<DataTypeUInt64>(), "Current cache size by the number of bytes"},
{"current_elements", std::make_shared<DataTypeUInt64>(), "Current cache size by the number of elements"},
{"max_file_segment_size", std::make_shared<DataTypeUInt64>(), "Maximum allowed file segment size"},
@ -56,6 +57,7 @@ void StorageSystemFilesystemCacheSettings::fillData(
res_columns[i++]->insert(settings.base_path);
res_columns[i++]->insert(settings.max_size);
res_columns[i++]->insert(settings.max_elements);
res_columns[i++]->insert(cache->isInitialized());
res_columns[i++]->insert(cache->getUsedCacheSize());
res_columns[i++]->insert(cache->getFileSegmentsNum());
res_columns[i++]->insert(settings.max_file_segment_size);

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