Merge branch 'master' into hanfei/stats_uniq

This commit is contained in:
Han Fei 2024-01-29 19:54:15 +01:00
commit f46065ba88
2612 changed files with 66703 additions and 23161 deletions

View File

@ -8,13 +8,13 @@ on: # yamllint disable-line rule:truthy
schedule:
- cron: '0 */6 * * *'
workflow_dispatch:
workflow_call:
jobs:
KeeperJepsenRelease:
uses: ./.github/workflows/reusable_simple_job.yml
with:
test_name: Jepsen keeper check
runner_type: style-checker
report_required: true
run_command: |
python3 jepsen_check.py keeper
# ServerJepsenRelease:

View File

@ -15,6 +15,8 @@ jobs:
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
@ -33,11 +35,9 @@ jobs:
- name: PrepareRunConfig
id: runconfig
run: |
echo "::group::configure CI run"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --rebuild-all-binaries --outfile ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
echo "::group::CI run configure results"
echo "::group::CI configuration"
python3 -m json.tool ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
@ -255,9 +255,9 @@ jobs:
run_command: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 docker_server.py --release-type head \
--image-repo clickhouse/clickhouse-server --image-path docker/server
--image-repo clickhouse/clickhouse-server --image-path docker/server --allow-build-reuse
python3 docker_server.py --release-type head \
--image-repo clickhouse/clickhouse-keeper --image-path docker/keeper
--image-repo clickhouse/clickhouse-keeper --image-path docker/keeper --allow-build-reuse
############################################################################################
##################################### BUILD REPORTER #######################################
############################################################################################

View File

@ -22,6 +22,8 @@ jobs:
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
@ -44,11 +46,9 @@ jobs:
- name: PrepareRunConfig
id: runconfig
run: |
echo "::group::configure CI run"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --outfile ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
echo "::group::CI run configure results"
echo "::group::CI configuration"
python3 -m json.tool ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
@ -67,6 +67,7 @@ jobs:
DOCKER_TAG=$(echo '${{ toJson(fromJson(steps.runconfig.outputs.CI_DATA).docker_data.images) }}' | tr -d '\n')
export DOCKER_TAG=$DOCKER_TAG
python3 ./tests/ci/style_check.py --no-push
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --post --job-name 'Style check'
BuildDockers:
needs: [RunConfig]
if: ${{ !failure() && !cancelled() }}
@ -796,7 +797,7 @@ jobs:
test_name: Unit tests (asan)
runner_type: fuzzer-unit-tester
data: ${{ needs.RunConfig.outputs.data }}
UnitTestsReleaseClang:
UnitTestsRelease:
needs: [RunConfig, BuilderBinRelease]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
@ -923,7 +924,7 @@ jobs:
- UnitTestsTsan
- UnitTestsMsan
- UnitTestsUBsan
- UnitTestsReleaseClang
- UnitTestsRelease
- CompatibilityCheckX86
- CompatibilityCheckAarch64
- SQLancerTestRelease
@ -966,13 +967,20 @@ jobs:
#############################################################################################
###################################### JEPSEN TESTS #########################################
#############################################################################################
# This is special test NOT INCLUDED in FinishCheck
# When it's skipped, all dependent tasks will be skipped too.
# DO NOT add it there
Jepsen:
# This is special test NOT INCLUDED in FinishCheck
# When it's skipped, all dependent tasks will be skipped too.
# DO NOT add it there
if: ${{ !failure() && !cancelled() && contains(github.event.pull_request.labels.*.name, 'jepsen-test') }}
# we need concurrency as the job uses dedicated instances in the cloud
concurrency:
group: jepsen
if: ${{ !failure() && !cancelled() }}
needs: [RunConfig, BuilderBinRelease]
uses: ./.github/workflows/jepsen.yml
uses: ./.github/workflows/reusable_test.yml
with:
test_name: ClickHouse Keeper Jepsen
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
#############################################################################################
####################################### libFuzzer ###########################################
#############################################################################################

View File

@ -58,6 +58,7 @@ jobs:
- name: Apply sparse checkout for contrib # in order to check that it doesn't break build
# This step is done in GITHUB_WORKSPACE,
# because it's broken in REPO_COPY for some reason
# See also update-submodules.sh
if: ${{ env.BUILD_SPARSE_CHECKOUT == 'true' }}
run: |
rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed'
@ -72,12 +73,15 @@ jobs:
- name: Pre
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(inputs.data) }} --pre --job-name '${{inputs.build_name}}'
- name: Build
- name: Run
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/build_check.py" "$BUILD_NAME"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" \
--infile ${{ toJson(inputs.data) }} \
--job-name "$BUILD_NAME" \
--run
- name: Post
# it still be build report to upload for failed build job
if: always()
if: ${{ !cancelled() }}
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(inputs.data) }} --post --job-name '${{inputs.build_name}}'
- name: Mark as done

View File

@ -34,12 +34,16 @@ name: Simple job
working-directory:
description: sets custom working directory
type: string
default: ""
default: "$GITHUB_WORKSPACE/tests/ci"
git_ref:
description: commit to use, merge commit for pr or head
required: false
type: string
default: ${{ github.event.after }} # no merge commit
report_required:
description: set to true if job report with the commit status required
type: boolean
default: false
secrets:
secret_envs:
description: if given, it's passed to the environments
@ -58,6 +62,8 @@ jobs:
env:
GITHUB_JOB_OVERRIDDEN: ${{inputs.test_name}}
steps:
- name: DebugInfo
uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
@ -79,12 +85,12 @@ jobs:
job_type: test
- name: Run
run: |
if [ -n '${{ inputs.working-directory }}' ]; then
cd "${{ inputs.working-directory }}"
else
cd "$GITHUB_WORKSPACE/tests/ci"
fi
cd "${{ inputs.working-directory }}"
${{ inputs.run_command }}
- name: Post
if: ${{ inputs.report_required }}
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --post --job-name '${{inputs.test_name}}'
- name: Clean
if: always()
uses: ./.github/actions/clean

View File

@ -38,7 +38,7 @@ name: Testing workflow
working-directory:
description: sets custom working directory
type: string
default: ""
default: "$GITHUB_WORKSPACE/tests/ci"
secrets:
secret_envs:
description: if given, it's passed to the environments
@ -96,19 +96,14 @@ jobs:
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(inputs.data) }} --pre --job-name '${{inputs.test_name}}'
- name: Run
run: |
if [ -n "${{ inputs.working-directory }}" ]; then
cd "${{ inputs.working-directory }}"
else
cd "$GITHUB_WORKSPACE/tests/ci"
fi
if [ -n "$(echo '${{ inputs.run_command }}' | tr -d '\n')" ]; then
echo "Running command from workflow input"
${{ inputs.run_command }}
else
echo "Running command from job config"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(inputs.data) }} --run --job-name '${{inputs.test_name}}'
fi
cd "${{ inputs.working-directory }}"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" \
--infile ${{ toJson(inputs.data) }} \
--job-name '${{inputs.test_name}}' \
--run \
--run-command '''${{inputs.run_command}}'''
- name: Post run
if: ${{ !cancelled() }}
run: |
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ toJson(inputs.data) }} --post --job-name '${{inputs.test_name}}'
- name: Mark as done

View File

@ -1,9 +1,18 @@
## To avoid merge commit in CI run (add a leading space to apply):
#no-merge-commit
### CI modificators (add a leading space to apply):
## Running specified job (add a leading space to apply):
## To avoid a merge commit in CI:
#no_merge_commit
## To discard CI cache:
#no_ci_cache
## To run specified set of tests in CI:
#ci_set_<SET_NAME>
#ci_set_reduced
## To run specified job in CI:
#job_<JOB NAME>
#job_stateless_tests_release
#job_package_debug

9
.gitmodules vendored
View File

@ -245,6 +245,12 @@
[submodule "contrib/idxd-config"]
path = contrib/idxd-config
url = https://github.com/intel/idxd-config
[submodule "contrib/QAT-ZSTD-Plugin"]
path = contrib/QAT-ZSTD-Plugin
url = https://github.com/intel/QAT-ZSTD-Plugin
[submodule "contrib/qatlib"]
path = contrib/qatlib
url = https://github.com/intel/qatlib
[submodule "contrib/wyhash"]
path = contrib/wyhash
url = https://github.com/wangyi-fudan/wyhash
@ -360,3 +366,6 @@
[submodule "contrib/sqids-cpp"]
path = contrib/sqids-cpp
url = https://github.com/sqids/sqids-cpp.git
[submodule "contrib/idna"]
path = contrib/idna
url = https://github.com/ada-url/idna.git

View File

@ -22,7 +22,7 @@
* The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for the `OPTIMIZE` is not allowed by default (it can be unlocked with the `allow_experimental_replacing_merge_with_cleanup` setting). [#58267](https://github.com/ClickHouse/ClickHouse/pull/58267) ([Alexander Tokmakov](https://github.com/tavplubix)). This fixes [#57930](https://github.com/ClickHouse/ClickHouse/issues/57930). This closes [#54988](https://github.com/ClickHouse/ClickHouse/issues/54988). This closes [#54570](https://github.com/ClickHouse/ClickHouse/issues/54570). This closes [#50346](https://github.com/ClickHouse/ClickHouse/issues/50346). This closes [#47579](https://github.com/ClickHouse/ClickHouse/issues/47579). The feature has to be removed because it is not good. We have to remove it as quickly as possible, because there is no other option. [#57932](https://github.com/ClickHouse/ClickHouse/pull/57932) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### New Feature
* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/57995). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)).
* Implement Refreshable Materialized Views, requested in [#33919](https://github.com/ClickHouse/ClickHouse/issues/33919). [#56946](https://github.com/ClickHouse/ClickHouse/pull/56946) ([Michael Kolupaev](https://github.com/al13n321), [Michael Guzov](https://github.com/koloshmet)).
* Introduce `PASTE JOIN`, which allows users to join tables without `ON` clause simply by row numbers. Example: `SELECT * FROM (SELECT number AS a FROM numbers(2)) AS t1 PASTE JOIN (SELECT number AS a FROM numbers(2) ORDER BY a DESC) AS t2`. [#57995](https://github.com/ClickHouse/ClickHouse/pull/57995) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
* The `ORDER BY` clause now supports specifying `ALL`, meaning that ClickHouse sorts by all columns in the `SELECT` clause. Example: `SELECT col1, col2 FROM tab WHERE [...] ORDER BY ALL`. [#57875](https://github.com/ClickHouse/ClickHouse/pull/57875) ([zhongyuankai](https://github.com/zhongyuankai)).
* Added a new mutation command `ALTER TABLE <table> APPLY DELETED MASK`, which allows to enforce applying of mask written by lightweight delete and to remove rows marked as deleted from disk. [#57433](https://github.com/ClickHouse/ClickHouse/pull/57433) ([Anton Popov](https://github.com/CurtizJ)).
@ -71,7 +71,7 @@
* Add `skip_unavailable_shards` as a setting for `Distributed` tables that is similar to the corresponding query-level setting. Closes [#43666](https://github.com/ClickHouse/ClickHouse/issues/43666). [#57218](https://github.com/ClickHouse/ClickHouse/pull/57218) ([Gagan Goel](https://github.com/tntnatbry)).
* The function `substring` (aliases: `substr`, `mid`) can now be used with `Enum` types. Previously, the first function argument had to be a value of type `String` or `FixedString`. This improves compatibility with 3rd party tools such as Tableau via MySQL interface. [#57277](https://github.com/ClickHouse/ClickHouse/pull/57277) ([Serge Klochkov](https://github.com/slvrtrn)).
* Function `format` now supports arbitrary argument types (instead of only `String` and `FixedString` arguments). This is important to calculate `SELECT format('The {0} to all questions is {1}', 'answer', 42)`. [#57549](https://github.com/ClickHouse/ClickHouse/pull/57549) ([Robert Schulze](https://github.com/rschu1ze)).
* Allows to use the `date_trunc` function with a case insensitive first argument. Both cases are now supported: `SELECT date_trunc('day', now())` and `SELECT date_trunc('DAY', now())`. [#57624](https://github.com/ClickHouse/ClickHouse/pull/57624) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
* Allows to use the `date_trunc` function with a case-insensitive first argument. Both cases are now supported: `SELECT date_trunc('day', now())` and `SELECT date_trunc('DAY', now())`. [#57624](https://github.com/ClickHouse/ClickHouse/pull/57624) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
* Better hints when a table doesn't exist. [#57342](https://github.com/ClickHouse/ClickHouse/pull/57342) ([Bharat Nallan](https://github.com/bharatnc)).
* Allow to overwrite `max_partition_size_to_drop` and `max_table_size_to_drop` server settings in query time. [#57452](https://github.com/ClickHouse/ClickHouse/pull/57452) ([Jordi Villar](https://github.com/jrdi)).
* Slightly better inference of unnamed tupes in JSON formats. [#57751](https://github.com/ClickHouse/ClickHouse/pull/57751) ([Kruglov Pavel](https://github.com/Avogar)).
@ -375,6 +375,7 @@
* Do not interpret the `send_timeout` set on the client side as the `receive_timeout` on the server side and vise-versa. [#56035](https://github.com/ClickHouse/ClickHouse/pull/56035) ([Azat Khuzhin](https://github.com/azat)).
* Comparison of time intervals with different units will throw an exception. This closes [#55942](https://github.com/ClickHouse/ClickHouse/issues/55942). You might have occasionally rely on the previous behavior when the underlying numeric values were compared regardless of the units. [#56090](https://github.com/ClickHouse/ClickHouse/pull/56090) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Rewrited the experimental `S3Queue` table engine completely: changed the way we keep information in zookeeper which allows to make less zookeeper requests, added caching of zookeeper state in cases when we know the state will not change, improved the polling from s3 process to make it less aggressive, changed the way ttl and max set for trached files is maintained, now it is a background process. Added `system.s3queue` and `system.s3queue_log` tables. Closes [#54998](https://github.com/ClickHouse/ClickHouse/issues/54998). [#54422](https://github.com/ClickHouse/ClickHouse/pull/54422) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Arbitrary paths on HTTP endpoint are no longer interpreted as a request to the `/query` endpoint. [#55521](https://github.com/ClickHouse/ClickHouse/pull/55521) ([Konstantin Bogdanov](https://github.com/thevar1able)).
#### New Feature
* Add function `arrayFold(accumulator, x1, ..., xn -> expression, initial, array1, ..., arrayn)` which applies a lambda function to multiple arrays of the same cardinality and collects the result in an accumulator. [#49794](https://github.com/ClickHouse/ClickHouse/pull/49794) ([Lirikl](https://github.com/Lirikl)).

View File

@ -1,4 +1,4 @@
Copyright 2016-2023 ClickHouse, Inc.
Copyright 2016-2024 ClickHouse, Inc.
Apache License
Version 2.0, January 2004
@ -188,7 +188,7 @@ Copyright 2016-2023 ClickHouse, Inc.
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright 2016-2023 ClickHouse, Inc.
Copyright 2016-2024 ClickHouse, Inc.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.

View File

@ -33,12 +33,7 @@ curl https://clickhouse.com/ | sh
## Upcoming Events
* [**ClickHouse Meetup in Berlin**](https://www.meetup.com/clickhouse-berlin-user-group/events/296488501/) - Nov 30
* [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/296488779/) - Dec 11
* [**ClickHouse Meetup in Sydney**](https://www.meetup.com/clickhouse-sydney-user-group/events/297638812/) - Dec 12
* [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/296488840/) - Dec 12
Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com.
Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `<at>` clickhouse `<dot>` com.
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"

View File

@ -99,7 +99,7 @@ public:
};
}
constexpr DB::UInt64 max_uint_mask = std::numeric_limits<DB::UInt64>::max();
constexpr UInt64 max_uint_mask = std::numeric_limits<UInt64>::max();
namespace std
{
@ -114,8 +114,8 @@ namespace std
{
size_t operator()(const DB::Decimal128 & x) const
{
return std::hash<DB::Int64>()(x.value >> 64)
^ std::hash<DB::Int64>()(x.value & max_uint_mask);
return std::hash<Int64>()(x.value >> 64)
^ std::hash<Int64>()(x.value & max_uint_mask);
}
};
@ -134,8 +134,8 @@ namespace std
size_t operator()(const DB::Decimal256 & x) const
{
// FIXME temp solution
return std::hash<DB::Int64>()(static_cast<DB::Int64>(x.value >> 64 & max_uint_mask))
^ std::hash<DB::Int64>()(static_cast<DB::Int64>(x.value & max_uint_mask));
return std::hash<Int64>()(static_cast<Int64>(x.value >> 64 & max_uint_mask))
^ std::hash<Int64>()(static_cast<Int64>(x.value & max_uint_mask));
}
};
}

View File

@ -3,15 +3,6 @@
#include <cstdint>
#include <string>
using Int8 = int8_t;
using Int16 = int16_t;
using Int32 = int32_t;
using Int64 = int64_t;
#ifndef __cpp_char8_t
using char8_t = unsigned char;
#endif
/// This is needed for more strict aliasing. https://godbolt.org/z/xpJBSb https://stackoverflow.com/a/57453713
using UInt8 = char8_t;
@ -19,24 +10,12 @@ using UInt16 = uint16_t;
using UInt32 = uint32_t;
using UInt64 = uint64_t;
using String = std::string;
namespace DB
{
using UInt8 = ::UInt8;
using UInt16 = ::UInt16;
using UInt32 = ::UInt32;
using UInt64 = ::UInt64;
using Int8 = ::Int8;
using Int16 = ::Int16;
using Int32 = ::Int32;
using Int64 = ::Int64;
using Int8 = int8_t;
using Int16 = int16_t;
using Int32 = int32_t;
using Int64 = int64_t;
using Float32 = float;
using Float64 = double;
using String = std::string;
}

View File

@ -26,6 +26,11 @@
#include "Poco/StreamUtil.h"
namespace DB
{
class ReadBufferFromIStream;
}
namespace Poco
{
@ -120,6 +125,8 @@ protected:
openmode getMode() const { return _mode; }
private:
friend class DB::ReadBufferFromIStream;
virtual int readFromDevice(char_type * /*buffer*/, std::streamsize /*length*/) { return 0; }
virtual int writeToDevice(const char_type * /*buffer*/, std::streamsize /*length*/) { return 0; }

View File

@ -33,7 +33,8 @@ namespace Poco
class Exception;
class Logger;
using LoggerPtr = std::shared_ptr<Logger>;
class Foundation_API Logger : public Channel
/// Logger is a special Channel that acts as the main
@ -870,6 +871,11 @@ public:
/// If the Logger does not yet exist, it is created, based
/// on its parent logger.
static LoggerPtr getShared(const std::string & name);
/// Returns a shared pointer to the Logger with the given name.
/// If the Logger does not yet exist, it is created, based
/// on its parent logger.
static Logger & unsafeGet(const std::string & name);
/// Returns a reference to the Logger with the given name.
/// If the Logger does not yet exist, it is created, based
@ -885,6 +891,11 @@ public:
/// given name. The Logger's Channel and log level as set as
/// specified.
static LoggerPtr createShared(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
/// Creates and returns a shared pointer to a Logger with the
/// given name. The Logger's Channel and log level as set as
/// specified.
static Logger & root();
/// Returns a reference to the root logger, which is the ultimate
/// ancestor of all Loggers.
@ -893,13 +904,6 @@ public:
/// Returns a pointer to the Logger with the given name if it
/// exists, or a null pointer otherwise.
static void destroy(const std::string & name);
/// Destroys the logger with the specified name. Does nothing
/// if the logger is not found.
///
/// After a logger has been destroyed, all references to it
/// become invalid.
static void shutdown();
/// Shuts down the logging framework and releases all
/// Loggers.
@ -929,8 +933,6 @@ public:
static const std::string ROOT; /// The name of the root logger ("").
protected:
typedef std::map<std::string, Logger *> LoggerMap;
Logger(const std::string & name, Channel * pChannel, int level);
~Logger();
@ -938,6 +940,7 @@ protected:
void log(const std::string & text, Message::Priority prio, const char * file, int line);
static std::string format(const std::string & fmt, int argc, std::string argv[]);
static Logger & unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
static Logger & parent(const std::string & name);
static void add(Logger * pLogger);
static Logger * find(const std::string & name);
@ -950,9 +953,6 @@ private:
std::string _name;
Channel * _pChannel;
std::atomic_int _level;
static LoggerMap * _pLoggerMap;
static Mutex _mapMtx;
};

View File

@ -38,15 +38,15 @@ public:
/// Creates the RefCountedObject.
/// The initial reference count is one.
void duplicate() const;
/// Increments the object's reference count.
size_t duplicate() const;
/// Increments the object's reference count, returns reference count before call.
void release() const throw();
size_t release() const throw();
/// Decrements the object's reference count
/// and deletes the object if the count
/// reaches zero.
/// reaches zero, returns reference count before call.
int referenceCount() const;
size_t referenceCount() const;
/// Returns the reference count.
protected:
@ -57,36 +57,40 @@ private:
RefCountedObject(const RefCountedObject &);
RefCountedObject & operator=(const RefCountedObject &);
mutable AtomicCounter _counter;
mutable std::atomic<size_t> _counter;
};
//
// inlines
//
inline int RefCountedObject::referenceCount() const
inline size_t RefCountedObject::referenceCount() const
{
return _counter.value();
return _counter.load(std::memory_order_acquire);
}
inline void RefCountedObject::duplicate() const
inline size_t RefCountedObject::duplicate() const
{
++_counter;
return _counter.fetch_add(1, std::memory_order_acq_rel);
}
inline void RefCountedObject::release() const throw()
inline size_t RefCountedObject::release() const throw()
{
size_t reference_count_before = _counter.fetch_sub(1, std::memory_order_acq_rel);
try
{
if (--_counter == 0)
if (reference_count_before == 1)
delete this;
}
catch (...)
{
poco_unexpected();
}
return reference_count_before;
}

View File

@ -20,12 +20,38 @@
#include "Poco/NumberParser.h"
#include "Poco/String.h"
#include <cassert>
#include <mutex>
namespace
{
std::mutex & getLoggerMutex()
{
auto get_logger_mutex_placeholder_memory = []()
{
static char buffer[sizeof(std::mutex)]{};
return buffer;
};
static std::mutex * logger_mutex = new (get_logger_mutex_placeholder_memory()) std::mutex();
return *logger_mutex;
}
struct LoggerEntry
{
Poco::Logger * logger;
bool owned_by_shared_ptr = false;
};
using LoggerMap = std::unordered_map<std::string, LoggerEntry>;
LoggerMap * _pLoggerMap = nullptr;
}
namespace Poco {
Logger::LoggerMap* Logger::_pLoggerMap = 0;
Mutex Logger::_mapMtx;
const std::string Logger::ROOT;
@ -112,17 +138,17 @@ void Logger::dump(const std::string& msg, const void* buffer, std::size_t length
void Logger::setLevel(const std::string& name, int level)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
std::string::size_type len = name.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
if (len == 0 ||
(it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.')))
(it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.')))
{
it->second->setLevel(level);
it.second.logger->setLevel(level);
}
}
}
@ -131,17 +157,17 @@ void Logger::setLevel(const std::string& name, int level)
void Logger::setChannel(const std::string& name, Channel* pChannel)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
std::string::size_type len = name.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
if (len == 0 ||
(it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.')))
(it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.')))
{
it->second->setChannel(pChannel);
it.second.logger->setChannel(pChannel);
}
}
}
@ -150,17 +176,17 @@ void Logger::setChannel(const std::string& name, Channel* pChannel)
void Logger::setProperty(const std::string& loggerName, const std::string& propertyName, const std::string& value)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
std::string::size_type len = loggerName.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
if (len == 0 ||
(it->first.compare(0, len, loggerName) == 0 && (it->first.length() == len || it->first[len] == '.')))
(it.first.compare(0, len, loggerName) == 0 && (it.first.length() == len || it.first[len] == '.')))
{
it->second->setProperty(propertyName, value);
it.second.logger->setProperty(propertyName, value);
}
}
}
@ -280,11 +306,88 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le
}
namespace
{
struct LoggerDeleter
{
void operator()(Poco::Logger * logger)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
/// If logger infrastructure is destroyed just decrement logger reference count
if (!_pLoggerMap)
{
logger->release();
return;
}
auto it = _pLoggerMap->find(logger->name());
assert(it != _pLoggerMap->end());
/** If reference count is 1, this means this shared pointer owns logger
* and need destroy it.
*/
size_t reference_count_before_release = logger->release();
if (reference_count_before_release == 1)
{
assert(it->second.owned_by_shared_ptr);
_pLoggerMap->erase(it);
}
}
};
inline LoggerPtr makeLoggerPtr(Logger & logger)
{
return std::shared_ptr<Logger>(&logger, LoggerDeleter());
}
}
Logger& Logger::get(const std::string& name)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
return unsafeGet(name);
Logger & logger = unsafeGet(name);
/** If there are already shared pointer created for this logger
* we need to increment Logger reference count and now logger
* is owned by logger infrastructure.
*/
auto it = _pLoggerMap->find(name);
if (it->second.owned_by_shared_ptr)
{
it->second.logger->duplicate();
it->second.owned_by_shared_ptr = false;
}
return logger;
}
LoggerPtr Logger::getShared(const std::string & name)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
bool logger_exists = _pLoggerMap && _pLoggerMap->contains(name);
Logger & logger = unsafeGet(name);
/** If logger already exists, then this shared pointer does not own it.
* If logger does not exists, logger infrastructure could be already destroyed
* or logger was created.
*/
if (logger_exists)
{
logger.duplicate();
}
else if (_pLoggerMap)
{
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
}
return makeLoggerPtr(logger);
}
@ -310,18 +413,24 @@ Logger& Logger::unsafeGet(const std::string& name)
Logger& Logger::create(const std::string& name, Channel* pChannel, int level)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (find(name)) throw ExistsException();
Logger* pLogger = new Logger(name, pChannel, level);
add(pLogger);
return *pLogger;
return unsafeCreate(name, pChannel, level);
}
LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
Logger & logger = unsafeCreate(name, pChannel, level);
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
return makeLoggerPtr(logger);
}
Logger& Logger::root()
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
return unsafeGet(ROOT);
}
@ -329,7 +438,7 @@ Logger& Logger::root()
Logger* Logger::has(const std::string& name)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
return find(name);
}
@ -337,14 +446,18 @@ Logger* Logger::has(const std::string& name)
void Logger::shutdown()
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
for (auto & it : *_pLoggerMap)
{
it->second->release();
if (it.second.owned_by_shared_ptr)
continue;
it.second.logger->release();
}
delete _pLoggerMap;
_pLoggerMap = 0;
}
@ -357,31 +470,15 @@ Logger* Logger::find(const std::string& name)
{
LoggerMap::iterator it = _pLoggerMap->find(name);
if (it != _pLoggerMap->end())
return it->second;
return it->second.logger;
}
return 0;
}
void Logger::destroy(const std::string& name)
{
Mutex::ScopedLock lock(_mapMtx);
if (_pLoggerMap)
{
LoggerMap::iterator it = _pLoggerMap->find(name);
if (it != _pLoggerMap->end())
{
it->second->release();
_pLoggerMap->erase(it);
}
}
}
void Logger::names(std::vector<std::string>& names)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
names.clear();
if (_pLoggerMap)
@ -393,6 +490,14 @@ void Logger::names(std::vector<std::string>& names)
}
}
Logger& Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level)
{
if (find(name)) throw ExistsException();
Logger* pLogger = new Logger(name, pChannel, level);
add(pLogger);
return *pLogger;
}
Logger& Logger::parent(const std::string& name)
{
@ -478,7 +583,8 @@ void Logger::add(Logger* pLogger)
{
if (!_pLoggerMap)
_pLoggerMap = new LoggerMap;
_pLoggerMap->insert(LoggerMap::value_type(pLogger->name(), pLogger));
_pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/});
}

View File

@ -18,6 +18,7 @@
#ifndef POCO_UTIL_NO_XMLCONFIGURATION
#include "Poco/String.h"
#include "Poco/SAX/InputSource.h"
#include "Poco/DOM/DOMParser.h"
#include "Poco/DOM/Element.h"
@ -28,6 +29,8 @@
#include "Poco/NumberParser.h"
#include "Poco/NumberFormatter.h"
#include <unordered_map>
#include <algorithm>
#include <iterator>
namespace Poco {
@ -275,8 +278,9 @@ void XMLConfiguration::enumerate(const std::string& key, Keys& range) const
{
if (pChild->nodeType() == Poco::XML::Node::ELEMENT_NODE)
{
const std::string& nodeName = pChild->nodeName();
std::string nodeName = pChild->nodeName();
size_t& count = keys[nodeName];
replaceInPlace(nodeName, ".", "\\.");
if (count)
range.push_back(nodeName + "[" + NumberFormatter::format(count) + "]");
else
@ -379,7 +383,21 @@ Poco::XML::Node* XMLConfiguration::findNode(std::string::const_iterator& it, con
{
while (it != end && *it == _delim) ++it;
std::string key;
while (it != end && *it != _delim && *it != '[') key += *it++;
while (it != end)
{
if (*it == '\\' && std::distance(it, end) > 1)
{
// Skip backslash, copy only the char after it
std::advance(it, 1);
key += *it++;
continue;
}
if (*it == _delim)
break;
if (*it == '[')
break;
key += *it++;
}
return findNode(it, end, findElement(key, pNode, create), create);
}
}

View File

@ -3,10 +3,10 @@
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.
SET(VERSION_REVISION 54482)
SET(VERSION_MAJOR 23)
SET(VERSION_MINOR 13)
SET(VERSION_MAJOR 24)
SET(VERSION_MINOR 1)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH a2faa65b080a587026c86844f3a20c74d23a86f8)
SET(VERSION_DESCRIBE v23.13.1.1-testing)
SET(VERSION_STRING 23.13.1.1)
SET(VERSION_DESCRIBE v24.1.1.1-testing)
SET(VERSION_STRING 24.1.1.1)
# end of autochange

View File

@ -82,3 +82,4 @@ if (SANITIZE_COVERAGE)
endif()
set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table")
set (WITHOUT_COVERAGE_FLAGS_LIST -fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table)

View File

@ -154,6 +154,7 @@ add_contrib (libpqxx-cmake libpqxx)
add_contrib (libpq-cmake libpq)
add_contrib (nuraft-cmake NuRaft)
add_contrib (fast_float-cmake fast_float)
add_contrib (idna-cmake idna)
add_contrib (datasketches-cpp-cmake datasketches-cpp)
add_contrib (incbin-cmake incbin)
add_contrib (sqids-cpp-cmake sqids-cpp)
@ -171,9 +172,9 @@ add_contrib (s2geometry-cmake s2geometry)
add_contrib (c-ares-cmake c-ares)
if (OS_LINUX AND ARCH_AMD64 AND ENABLE_SSE42)
option (ENABLE_QPL "Enable Intel® Query Processing Library" ${ENABLE_LIBRARIES})
option (ENABLE_QPL "Enable Intel® Query Processing Library (QPL)" ${ENABLE_LIBRARIES})
elseif(ENABLE_QPL)
message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 arch with SSE 4.2 or higher")
message (${RECONFIGURE_MESSAGE_LEVEL} "QPL library is only supported on x86_64 with SSE 4.2 or higher")
endif()
if (ENABLE_QPL)
add_contrib (idxd-config-cmake idxd-config)
@ -182,6 +183,28 @@ else()
message(STATUS "Not using QPL")
endif ()
if (OS_LINUX AND ARCH_AMD64)
option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology Library (QATlib)" ${ENABLE_LIBRARIES})
elseif(ENABLE_QATLIB)
message (${RECONFIGURE_MESSAGE_LEVEL} "QATLib is only supported on x86_64")
endif()
if (ENABLE_QATLIB)
option (ENABLE_QAT_USDM_DRIVER "A User Space DMA-able Memory (USDM) component which allocates/frees DMA-able memory" OFF)
option (ENABLE_QAT_OUT_OF_TREE_BUILD "Using out-of-tree driver, user needs to customize ICP_ROOT variable" OFF)
set(ICP_ROOT "" CACHE STRING "ICP_ROOT variable to define the path of out-of-tree driver package")
if (ENABLE_QAT_OUT_OF_TREE_BUILD)
if (ICP_ROOT STREQUAL "")
message(FATAL_ERROR "Please define the path of out-of-tree driver package with -DICP_ROOT=xxx or disable out-of-tree build with -DENABLE_QAT_OUT_OF_TREE_BUILD=OFF; \
If you want out-of-tree build but have no package available, please download and build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html")
endif ()
else()
add_contrib (qatlib-cmake qatlib) # requires: isa-l
endif ()
add_contrib (QAT-ZSTD-Plugin-cmake QAT-ZSTD-Plugin)
else()
message(STATUS "Not using QATLib")
endif ()
add_contrib (morton-nd-cmake morton-nd)
if (ARCH_S390X)
add_contrib(crc32-s390x-cmake crc32-s390x)

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit b7ea89b817a18dc0eafc1f909d568869f02d2d04
Subproject commit 1278e32bb0d5dc489f947e002bdf8c71b0ddaa63

1
contrib/QAT-ZSTD-Plugin vendored Submodule

@ -0,0 +1 @@
Subproject commit e5a134e12d2ea8a5b0f3b83c5b1c325fda4eb0a8

View File

@ -0,0 +1,85 @@
# Intel® QuickAssist Technology ZSTD Plugin (QAT ZSTD Plugin) is a plugin to Zstandard*(ZSTD*) for accelerating compression by QAT.
# ENABLE_QAT_OUT_OF_TREE_BUILD = 1 means kernel don't have native support, user will build and install driver from external package: https://www.intel.com/content/www/us/en/download/765501.html
# meanwhile, user need to set ICP_ROOT environment variable which point to the root directory of QAT driver source tree.
# ENABLE_QAT_OUT_OF_TREE_BUILD = 0 means kernel has built-in qat driver, QAT-ZSTD-PLUGIN just has dependency on qatlib.
if (ENABLE_QAT_OUT_OF_TREE_BUILD)
message(STATUS "Intel QATZSTD out-of-tree build, ICP_ROOT:${ICP_ROOT}")
set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src")
set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c")
set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib")
set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include")
set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc")
set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include")
set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv")
set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so")
set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so")
if (ENABLE_QAT_USDM_DRIVER)
add_definitions(-DENABLE_USDM_DRV)
endif()
add_library(_qatzstd_plugin ${QATZSTD_SRC})
target_link_libraries (_qatzstd_plugin PUBLIC ${USDM_LIBRARY} ${QAT_S_LIBRARY})
target_include_directories(_qatzstd_plugin
SYSTEM PUBLIC "${QATZSTD_SRC_DIR}"
PRIVATE ${QAT_INCLUDE_DIR}
${QAT_DC_INCLUDE_DIR}
${QAT_AL_INCLUDE_DIR}
${QAT_USDM_INCLUDE_DIR}
${ZSTD_LIBRARY_DIR})
target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC)
add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin)
else () # In-tree build
message(STATUS "Intel QATZSTD in-tree build")
set(QATZSTD_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/QAT-ZSTD-Plugin/src")
set(QATZSTD_SRC "${QATZSTD_SRC_DIR}/qatseqprod.c")
set(ZSTD_LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib")
# please download&build ICP package from: https://www.intel.com/content/www/us/en/download/765501.html
set(ICP_ROOT "${ClickHouse_SOURCE_DIR}/contrib/qatlib")
set(QAT_INCLUDE_DIR "${ICP_ROOT}/quickassist/include")
set(QAT_DC_INCLUDE_DIR "${ICP_ROOT}/quickassist/include/dc")
set(QAT_AL_INCLUDE_DIR "${ICP_ROOT}/quickassist/lookaside/access_layer/include")
set(QAT_USDM_INCLUDE_DIR "${ICP_ROOT}/quickassist/utilities/libusdm_drv")
set(USDM_LIBRARY "${ICP_ROOT}/build/libusdm_drv_s.so")
set(QAT_S_LIBRARY "${ICP_ROOT}/build/libqat_s.so")
set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib")
set(LIBQAT_HEADER_DIR "${CMAKE_CURRENT_BINARY_DIR}/include")
file(MAKE_DIRECTORY
"${LIBQAT_HEADER_DIR}/qat"
)
file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/cpa.h"
DESTINATION "${LIBQAT_HEADER_DIR}/qat/"
)
file(COPY "${LIBQAT_ROOT_DIR}/quickassist/include/dc/cpa_dc.h"
DESTINATION "${LIBQAT_HEADER_DIR}/qat/"
)
file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_poll.h"
DESTINATION "${LIBQAT_HEADER_DIR}/qat/"
)
file(COPY "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include/icp_sal_user.h"
DESTINATION "${LIBQAT_HEADER_DIR}/qat/"
)
file(COPY "${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv/qae_mem.h"
DESTINATION "${LIBQAT_HEADER_DIR}/qat/"
)
if (ENABLE_QAT_USDM_DRIVER)
add_definitions(-DENABLE_USDM_DRV)
endif()
add_library(_qatzstd_plugin ${QATZSTD_SRC})
target_link_libraries (_qatzstd_plugin PUBLIC ch_contrib::qatlib ch_contrib::usdm)
target_include_directories(_qatzstd_plugin PRIVATE
${QAT_INCLUDE_DIR}
${QAT_DC_INCLUDE_DIR}
${QAT_AL_INCLUDE_DIR}
${QAT_USDM_INCLUDE_DIR}
${ZSTD_LIBRARY_DIR}
${LIBQAT_HEADER_DIR})
target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC -DINTREE)
target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $<BUILD_INTERFACE:${QATZSTD_SRC_DIR}> $<INSTALL_INTERFACE:include>)
add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin)
endif ()

2
contrib/avro vendored

@ -1 +1 @@
Subproject commit 2fb8a8a6ec0eab9109b68abf3b4857e8c476b918
Subproject commit d43acc84d3d455b016f847d6666fbc3cd27f16a9

2
contrib/azure vendored

@ -1 +1 @@
Subproject commit 060c54dfb0abe869c065143303a9d3e9c54c29e3
Subproject commit e71395e44f309f97b5a486f5c2c59b82f85dd2d2

View File

@ -44,12 +44,14 @@ set (SRCS_IOSTREAMS
"${LIBRARY_DIR}/libs/iostreams/src/gzip.cpp"
"${LIBRARY_DIR}/libs/iostreams/src/mapped_file.cpp"
"${LIBRARY_DIR}/libs/iostreams/src/zlib.cpp"
"${LIBRARY_DIR}/libs/iostreams/src/zstd.cpp"
)
add_library (_boost_iostreams ${SRCS_IOSTREAMS})
add_library (boost::iostreams ALIAS _boost_iostreams)
target_include_directories (_boost_iostreams PRIVATE ${LIBRARY_DIR})
target_link_libraries (_boost_iostreams PRIVATE ch_contrib::zlib)
target_link_libraries (_boost_iostreams PRIVATE ch_contrib::zstd)
# program_options

View File

@ -1,8 +1,5 @@
if (NOT ENABLE_LIBRARIES)
set(DEFAULT_ENABLE_RUST FALSE)
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64"))
message(STATUS "Rust is not available on aarch64-apple-darwin")
set(DEFAULT_ENABLE_RUST FALSE)
else()
list (APPEND CMAKE_MODULE_PATH "${ClickHouse_SOURCE_DIR}/contrib/corrosion/cmake")
find_package(Rust)
@ -19,7 +16,9 @@ message(STATUS "Checking Rust toolchain for current target")
# See https://doc.rust-lang.org/nightly/rustc/platform-support.html
if((CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") AND (CMAKE_TOOLCHAIN_FILE MATCHES "musl"))
if(CMAKE_TOOLCHAIN_FILE MATCHES "ppc64le")
set(Rust_CARGO_TARGET "powerpc64le-unknown-linux-gnu")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") AND (CMAKE_TOOLCHAIN_FILE MATCHES "musl"))
set(Rust_CARGO_TARGET "x86_64-unknown-linux-musl")
elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64")
set(Rust_CARGO_TARGET "x86_64-unknown-linux-gnu")
@ -29,14 +28,14 @@ elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-aarch64")
set(Rust_CARGO_TARGET "aarch64-unknown-linux-gnu")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64"))
set(Rust_CARGO_TARGET "x86_64-apple-darwin")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64"))
set(Rust_CARGO_TARGET "aarch64-apple-darwin")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "freebsd") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64"))
set(Rust_CARGO_TARGET "x86_64-unknown-freebsd")
elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-riscv64")
set(Rust_CARGO_TARGET "riscv64gc-unknown-linux-gnu")
endif()
if(CMAKE_TOOLCHAIN_FILE MATCHES "ppc64le")
set(Rust_CARGO_TARGET "powerpc64le-unknown-linux-gnu")
else()
message(FATAL_ERROR "Unsupported rust target")
endif()
message(STATUS "Switched Rust target to ${Rust_CARGO_TARGET}")

1
contrib/idna vendored Submodule

@ -0,0 +1 @@
Subproject commit 3c8be01d42b75649f1ac9b697d0ef757eebfe667

View File

@ -0,0 +1,24 @@
option(ENABLE_IDNA "Enable idna support" ${ENABLE_LIBRARIES})
if ((NOT ENABLE_IDNA))
message (STATUS "Not using idna")
return()
endif()
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/idna")
set (SRCS
"${LIBRARY_DIR}/src/idna.cpp"
"${LIBRARY_DIR}/src/mapping.cpp"
"${LIBRARY_DIR}/src/mapping_tables.cpp"
"${LIBRARY_DIR}/src/normalization.cpp"
"${LIBRARY_DIR}/src/normalization_tables.cpp"
"${LIBRARY_DIR}/src/punycode.cpp"
"${LIBRARY_DIR}/src/to_ascii.cpp"
"${LIBRARY_DIR}/src/to_unicode.cpp"
"${LIBRARY_DIR}/src/unicode_transcoding.cpp"
"${LIBRARY_DIR}/src/validity.cpp"
)
add_library (_idna ${SRCS})
target_include_directories(_idna PUBLIC "${LIBRARY_DIR}/include")
add_library (ch_contrib::idna ALIAS _idna)

View File

@ -34,9 +34,9 @@ if (OS_LINUX)
# avoid spurious latencies and additional work associated with
# MADV_DONTNEED. See
# https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation.
set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000")
set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000")
else()
set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000")
set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:0,dirty_decay_ms:5000")
endif()
# CACHE variable is empty to allow changing defaults without the necessity
# to purge cache
@ -161,6 +161,9 @@ target_include_directories(_jemalloc SYSTEM PRIVATE
target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE)
# Because our coverage callbacks call malloc, and recursive call of malloc could not work.
target_compile_options(_jemalloc PRIVATE ${WITHOUT_COVERAGE_FLAGS_LIST})
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")
target_compile_definitions(_jemalloc PRIVATE
-DJEMALLOC_DEBUG=1

View File

@ -33,7 +33,6 @@ set(SRCS
"${LIBCXX_SOURCE_DIR}/src/optional.cpp"
"${LIBCXX_SOURCE_DIR}/src/random.cpp"
"${LIBCXX_SOURCE_DIR}/src/random_shuffle.cpp"
"${LIBCXX_SOURCE_DIR}/src/regex.cpp"
"${LIBCXX_SOURCE_DIR}/src/ryu/d2fixed.cpp"
"${LIBCXX_SOURCE_DIR}/src/ryu/d2s.cpp"
"${LIBCXX_SOURCE_DIR}/src/ryu/f2s.cpp"

View File

@ -1,4 +1,4 @@
option (ENABLE_SSH "Enable support for SSH keys and protocol" ON)
option (ENABLE_SSH "Enable support for SSH keys and protocol" ${ENABLE_LIBRARIES})
if (NOT ENABLE_SSH)
message(STATUS "Not using SSH")

@ -1 +1 @@
Subproject commit 1834e42289c58402c804a87be4d489892b88f3ec
Subproject commit 2568a7cd1297c7c3044b0f3cc0c23a6f6444d856

View File

@ -1,4 +1,4 @@
if(OS_LINUX AND TARGET OpenSSL::SSL)
if((OS_LINUX OR OS_DARWIN) AND TARGET OpenSSL::SSL)
option(ENABLE_MYSQL "Enable MySQL" ${ENABLE_LIBRARIES})
else ()
option(ENABLE_MYSQL "Enable MySQL" FALSE)
@ -73,7 +73,7 @@ set(HAVE_SYS_TYPES_H 1)
set(HAVE_SYS_UN_H 1)
set(HAVE_UNISTD_H 1)
set(HAVE_UTIME_H 1)
set(HAVE_UCONTEXT_H 1)
set(HAVE_UCONTEXT_H 0)
set(HAVE_ALLOCA 1)
set(HAVE_DLERROR 0)
set(HAVE_DLOPEN 0)
@ -116,9 +116,13 @@ CONFIGURE_FILE(${CC_SOURCE_DIR}/include/ma_config.h.in
CONFIGURE_FILE(${CC_SOURCE_DIR}/include/mariadb_version.h.in
${CC_BINARY_DIR}/include-public/mariadb_version.h)
if(WITH_SSL)
if (WITH_SSL)
set(SYSTEM_LIBS ${SYSTEM_LIBS} ${SSL_LIBRARIES})
endif()
endif ()
if (OS_DARWIN)
set(SYSTEM_LIBS ${SYSTEM_LIBS} iconv)
endif ()
function(REGISTER_PLUGIN)
@ -227,15 +231,8 @@ ${CC_SOURCE_DIR}/libmariadb/secure/openssl_crypt.c
${CC_BINARY_DIR}/libmariadb/ma_client_plugin.c
)
if(ICONV_INCLUDE_DIR)
include_directories(BEFORE ${ICONV_INCLUDE_DIR})
endif()
add_definitions(-DLIBICONV_PLUG)
if(WITH_DYNCOL)
set(LIBMARIADB_SOURCES ${LIBMARIADB_SOURCES} ${CC_SOURCE_DIR}/libmariadb/mariadb_dyncol.c)
endif()
set(LIBMARIADB_SOURCES ${LIBMARIADB_SOURCES} ${CC_SOURCE_DIR}/libmariadb/mariadb_async.c ${CC_SOURCE_DIR}/libmariadb/ma_context.c)

1
contrib/qatlib vendored Submodule

@ -0,0 +1 @@
Subproject commit abe15d7bfc083117bfbb4baee0b49ffcd1c03c5c

View File

@ -0,0 +1,213 @@
# Intel® QuickAssist Technology Library (QATlib).
message(STATUS "Intel QATlib ON")
set(LIBQAT_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib")
set(LIBQAT_DIR "${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src")
set(LIBOSAL_DIR "${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src")
set(OPENSSL_DIR "${ClickHouse_SOURCE_DIR}/contrib/openssl")
# Build 3 libraries: _qatmgr, _osal, _qatlib
# Produce ch_contrib::qatlib by linking these libraries.
# _qatmgr
SET(LIBQATMGR_sources ${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_client.c
${LIBQAT_DIR}/qat_direct/vfio/qat_mgr_lib.c
${LIBQAT_DIR}/qat_direct/vfio/qat_log.c
${LIBQAT_DIR}/qat_direct/vfio/vfio_lib.c
${LIBQAT_DIR}/qat_direct/vfio/adf_pfvf_proto.c
${LIBQAT_DIR}/qat_direct/vfio/adf_pfvf_vf_msg.c
${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_pf.c)
add_library(_qatmgr ${LIBQATMGR_sources})
target_include_directories(_qatmgr PRIVATE
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/vfio
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include
${LIBQAT_ROOT_DIR}/quickassist/include
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include
${LIBQAT_ROOT_DIR}/quickassist/qat/drivers/crypto/qat/qat_common
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/common/include
${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include)
target_compile_definitions(_qatmgr PRIVATE -DUSER_SPACE)
target_compile_options(_qatmgr PRIVATE -Wno-error=int-conversion)
# _osal
SET(LIBOSAL_sources
${LIBOSAL_DIR}/linux/user_space/OsalSemaphore.c
${LIBOSAL_DIR}/linux/user_space/OsalThread.c
${LIBOSAL_DIR}/linux/user_space/OsalMutex.c
${LIBOSAL_DIR}/linux/user_space/OsalSpinLock.c
${LIBOSAL_DIR}/linux/user_space/OsalAtomic.c
${LIBOSAL_DIR}/linux/user_space/OsalServices.c
${LIBOSAL_DIR}/linux/user_space/OsalUsrKrnProxy.c
${LIBOSAL_DIR}/linux/user_space/OsalCryptoInterface.c)
add_library(_osal ${LIBOSAL_sources})
target_include_directories(_osal PRIVATE
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include
${OPENSSL_DIR}/include
${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake/linux_x86_64/include
${ClickHouse_SOURCE_DIR}/contrib/qatlib-cmake/include)
target_compile_definitions(_osal PRIVATE -DOSAL_ENSURE_ON -DUSE_OPENSSL)
# _qatlib
SET(LIBQAT_sources
${LIBQAT_DIR}/common/compression/dc_buffers.c
${LIBQAT_DIR}/common/compression/dc_chain.c
${LIBQAT_DIR}/common/compression/dc_datapath.c
${LIBQAT_DIR}/common/compression/dc_dp.c
${LIBQAT_DIR}/common/compression/dc_header_footer.c
${LIBQAT_DIR}/common/compression/dc_header_footer_lz4.c
${LIBQAT_DIR}/common/compression/dc_session.c
${LIBQAT_DIR}/common/compression/dc_stats.c
${LIBQAT_DIR}/common/compression/dc_err_sim.c
${LIBQAT_DIR}/common/compression/dc_ns_datapath.c
${LIBQAT_DIR}/common/compression/dc_ns_header_footer.c
${LIBQAT_DIR}/common/compression/dc_crc32.c
${LIBQAT_DIR}/common/compression/dc_crc64.c
${LIBQAT_DIR}/common/compression/dc_xxhash32.c
${LIBQAT_DIR}/common/compression/icp_sal_dc_err_sim.c
${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_control_path.c
${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_data_path.c
${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_interface_check.c
${LIBQAT_DIR}/common/crypto/asym/diffie_hellman/lac_dh_stats.c
${LIBQAT_DIR}/common/crypto/asym/dsa/lac_dsa.c
${LIBQAT_DIR}/common/crypto/asym/dsa/lac_dsa_interface_check.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec_common.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec_montedwds.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ec_nist_curves.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ecdh.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ecdsa.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_ecsm2.c
${LIBQAT_DIR}/common/crypto/asym/ecc/lac_kpt_ecdsa.c
${LIBQAT_DIR}/common/crypto/asym/large_number/lac_ln.c
${LIBQAT_DIR}/common/crypto/asym/large_number/lac_ln_interface_check.c
${LIBQAT_DIR}/common/crypto/asym/pke_common/lac_pke_mmp.c
${LIBQAT_DIR}/common/crypto/asym/pke_common/lac_pke_qat_comms.c
${LIBQAT_DIR}/common/crypto/asym/pke_common/lac_pke_utils.c
${LIBQAT_DIR}/common/crypto/asym/prime/lac_prime.c
${LIBQAT_DIR}/common/crypto/asym/prime/lac_prime_interface_check.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_control_path.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_decrypt.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_encrypt.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_interface_check.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_keygen.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_rsa_stats.c
${LIBQAT_DIR}/common/crypto/asym/rsa/lac_kpt_rsa_decrypt.c
${LIBQAT_DIR}/common/crypto/sym/drbg/lac_sym_drbg_api.c
${LIBQAT_DIR}/common/crypto/sym/key/lac_sym_key.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_alg_chain.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_api.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_auth_enc.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_cb.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_cipher.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_compile_check.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_dp.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_hash.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_partial.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_queue.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_stats.c
${LIBQAT_DIR}/common/crypto/sym/nrbg/lac_sym_nrbg_api.c
${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat.c
${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_cipher.c
${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_constants_table.c
${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_hash.c
${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_hash_defs_lookup.c
${LIBQAT_DIR}/common/crypto/sym/qat/lac_sym_qat_key.c
${LIBQAT_DIR}/common/crypto/sym/lac_sym_hash_sw_precomputes.c
${LIBQAT_DIR}/common/crypto/kpt/provision/lac_kpt_provision.c
${LIBQAT_DIR}/common/ctrl/sal_compression.c
${LIBQAT_DIR}/common/ctrl/sal_create_services.c
${LIBQAT_DIR}/common/ctrl/sal_ctrl_services.c
${LIBQAT_DIR}/common/ctrl/sal_list.c
${LIBQAT_DIR}/common/ctrl/sal_crypto.c
${LIBQAT_DIR}/common/ctrl/sal_dc_chain.c
${LIBQAT_DIR}/common/ctrl/sal_instances.c
${LIBQAT_DIR}/common/qat_comms/sal_qat_cmn_msg.c
${LIBQAT_DIR}/common/utils/lac_buffer_desc.c
${LIBQAT_DIR}/common/utils/lac_log_message.c
${LIBQAT_DIR}/common/utils/lac_mem.c
${LIBQAT_DIR}/common/utils/lac_mem_pools.c
${LIBQAT_DIR}/common/utils/lac_sw_responses.c
${LIBQAT_DIR}/common/utils/lac_sync.c
${LIBQAT_DIR}/common/utils/sal_service_state.c
${LIBQAT_DIR}/common/utils/sal_statistics.c
${LIBQAT_DIR}/common/utils/sal_misc_error_stats.c
${LIBQAT_DIR}/common/utils/sal_string_parse.c
${LIBQAT_DIR}/common/utils/sal_user_process.c
${LIBQAT_DIR}/common/utils/sal_versions.c
${LIBQAT_DIR}/common/device/sal_dev_info.c
${LIBQAT_DIR}/user/sal_user.c
${LIBQAT_DIR}/user/sal_user_dyn_instance.c
${LIBQAT_DIR}/qat_direct/common/adf_process_proxy.c
${LIBQAT_DIR}/qat_direct/common/adf_user_cfg.c
${LIBQAT_DIR}/qat_direct/common/adf_user_device.c
${LIBQAT_DIR}/qat_direct/common/adf_user_dyn.c
${LIBQAT_DIR}/qat_direct/common/adf_user_ETring_mgr_dp.c
${LIBQAT_DIR}/qat_direct/common/adf_user_init.c
${LIBQAT_DIR}/qat_direct/common/adf_user_ring.c
${LIBQAT_DIR}/qat_direct/common/adf_user_transport_ctrl.c
${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_cfg.c
${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_ring.c
${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_user_bundles.c
${LIBQAT_DIR}/qat_direct/vfio/adf_vfio_user_proxy.c
${LIBQAT_DIR}/common/compression/dc_crc_base.c)
add_library(_qatlib ${LIBQAT_sources})
target_include_directories(_qatlib PRIVATE
${CMAKE_SYSROOT}/usr/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/include
${LIBQAT_ROOT_DIR}/quickassist/utilities/libusdm_drv
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/include
${LIBOSAL_DIR}/linux/user_space/include
${LIBQAT_ROOT_DIR}/quickassist/include
${LIBQAT_ROOT_DIR}/quickassist/include/lac
${LIBQAT_ROOT_DIR}/quickassist/include/dc
${LIBQAT_ROOT_DIR}/quickassist/qat/drivers/crypto/qat/qat_common
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/compression/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/crypto/sym/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/crypto/asym/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/firmware/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/common/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/common/include
${LIBQAT_ROOT_DIR}/quickassist/lookaside/access_layer/src/qat_direct/vfio
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space
${LIBQAT_ROOT_DIR}/quickassist/utilities/osal/src/linux/user_space/include
${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include)
target_link_libraries(_qatlib PRIVATE _qatmgr _osal OpenSSL::SSL ch_contrib::isal)
target_compile_definitions(_qatlib PRIVATE -DUSER_SPACE -DLAC_BYTE_ORDER=__LITTLE_ENDIAN -DOSAL_ENSURE_ON)
target_link_options(_qatlib PRIVATE -pie -z relro -z now -z noexecstack)
target_compile_options(_qatlib PRIVATE -march=native)
add_library (ch_contrib::qatlib ALIAS _qatlib)
# _usdm
set(LIBUSDM_DIR "${ClickHouse_SOURCE_DIR}/contrib/qatlib/quickassist/utilities/libusdm_drv")
set(LIBUSDM_sources
${LIBUSDM_DIR}/user_space/vfio/qae_mem_utils_vfio.c
${LIBUSDM_DIR}/user_space/qae_mem_utils_common.c
${LIBUSDM_DIR}/user_space/vfio/qae_mem_hugepage_utils_vfio.c)
add_library(_usdm ${LIBUSDM_sources})
target_include_directories(_usdm PRIVATE
${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64-musl/include
${LIBUSDM_DIR}
${LIBUSDM_DIR}/include
${LIBUSDM_DIR}/user_space)
add_library (ch_contrib::usdm ALIAS _usdm)

View File

@ -0,0 +1,14 @@
/* This is a workaround for a build conflict issue
1. __GLIBC_PREREQ (referenced in OsalServices.c) is only defined in './sysroot/linux-x86_64/include/features.h'
2. mqueue.h only exist under './sysroot/linux-x86_64-musl/'
This cause target_include_directories for _osal has a conflict between './sysroot/linux-x86_64/include' and './sysroot/linux-x86_64-musl/'
hence create mqueue.h separately under ./qatlib-cmake/include as an alternative.
*/
/* Major and minor version number of the GNU C library package. Use
these macros to test for features in specific releases. */
#define __GLIBC__ 2
#define __GLIBC_MINOR__ 27
#define __GLIBC_PREREQ(maj, min) \
((__GLIBC__ << 16) + __GLIBC_MINOR__ >= ((maj) << 16) + (min))

2
contrib/rocksdb vendored

@ -1 +1 @@
Subproject commit 66e3cbec31400ed3a23deb878c5d7f56f990f0ae
Subproject commit dead55e60b873d5f70f0e9458fbbba2b2180f430

2
contrib/simdjson vendored

@ -1 +1 @@
Subproject commit 1075e8609c4afa253162d441437af929c29e31bb
Subproject commit 6060be2fdf62edf4a8f51a8b0883d57d09397b30

2
contrib/sqids-cpp vendored

@ -1 +1 @@
Subproject commit 3756e537d4d48cc0dd4176801fe19f99601439b0
Subproject commit a471f53672e98d49223f598528a533b07b085c61

View File

@ -6,9 +6,15 @@ SCRIPT_DIR=$(dirname "${SCRIPT_PATH}")
GIT_DIR=$(git -C "$SCRIPT_DIR" rev-parse --show-toplevel)
cd $GIT_DIR
# Exclude from contribs some garbage subdirs that we don't need.
# It reduces the checked out files size about 3 times and therefore speeds up indexing in IDEs and searching.
# NOTE .git/ still contains everything that we don't check out (although, it's compressed)
# See also https://git-scm.com/docs/git-sparse-checkout
contrib/sparse-checkout/setup-sparse-checkout.sh
git submodule init
git submodule sync
# NOTE: do not use --remote for `git submodule update`[1] command, since the submodule references to the specific commit SHA1 in the subproject.
# It may cause unexpected behavior. Instead you need to commit a new SHA1 for a submodule.
#

View File

@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="23.12.1.1368"
ARG VERSION="23.12.2.59"
ARG PACKAGES="clickhouse-keeper"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -3,10 +3,10 @@ compilers and build settings. Correctly configured Docker daemon is single depen
Usage:
Build deb package with `clang-14` in `debug` mode:
Build deb package with `clang-17` in `debug` mode:
```
$ mkdir deb/test_output
$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --debug-build
$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-17 --debug-build
$ ls -l deb/test_output
-rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb
-rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb
@ -17,11 +17,11 @@ $ ls -l deb/test_output
```
Build ClickHouse binary with `clang-14` and `address` sanitizer in `relwithdebuginfo`
Build ClickHouse binary with `clang-17` and `address` sanitizer in `relwithdebuginfo`
mode:
```
$ mkdir $HOME/some_clickhouse
$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-14 --sanitizer=address
$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-17 --sanitizer=address
$ ls -l $HOME/some_clickhouse
-rwxr-xr-x 1 root root 787061952 clickhouse
lrwxrwxrwx 1 root root 10 clickhouse-benchmark -> clickhouse

View File

@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="23.12.1.1368"
ARG VERSION="23.12.2.59"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -30,7 +30,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="23.12.1.1368"
ARG VERSION="23.12.2.59"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image

View File

@ -41,21 +41,18 @@ readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLIC
readarray -t DISKS_METADATA_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.metadata_path' || true)
CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}"
CLICKHOUSE_PASSWORD_FILE="${CLICKHOUSE_PASSWORD_FILE:-}"
if [[ -n "${CLICKHOUSE_PASSWORD_FILE}" && -f "${CLICKHOUSE_PASSWORD_FILE}" ]]; then
CLICKHOUSE_PASSWORD="$(cat "${CLICKHOUSE_PASSWORD_FILE}")"
fi
CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}"
CLICKHOUSE_DB="${CLICKHOUSE_DB:-}"
CLICKHOUSE_ACCESS_MANAGEMENT="${CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT:-0}"
for dir in "$DATA_DIR" \
"$ERROR_LOG_DIR" \
"$LOG_DIR" \
"$TMP_DIR" \
"$USER_PATH" \
"$FORMAT_SCHEMA_PATH" \
"${DISKS_PATHS[@]}" \
"${DISKS_METADATA_PATHS[@]}"
do
function create_directory_and_do_chown() {
local dir=$1
# check if variable not empty
[ -z "$dir" ] && continue
[ -z "$dir" ] && return
# ensure directories exist
if [ "$DO_CHOWN" = "1" ]; then
mkdir="mkdir"
@ -77,6 +74,23 @@ do
chown -R "$USER:$GROUP" "$dir"
fi
fi
}
create_directory_and_do_chown "$DATA_DIR"
# Change working directory to $DATA_DIR in case there're paths relative to $DATA_DIR, also avoids running
# clickhouse-server at root directory.
cd "$DATA_DIR"
for dir in "$ERROR_LOG_DIR" \
"$LOG_DIR" \
"$TMP_DIR" \
"$USER_PATH" \
"$FORMAT_SCHEMA_PATH" \
"${DISKS_PATHS[@]}" \
"${DISKS_METADATA_PATHS[@]}"
do
create_directory_and_do_chown "$dir"
done
# if clickhouse user is defined - create it (user "default" already exists out of box)

View File

@ -22,7 +22,7 @@ RUN apt-get update \
zstd \
--yes --no-install-recommends
RUN pip3 install numpy scipy pandas Jinja2
RUN pip3 install numpy==1.26.3 scipy==1.12.0 pandas==1.5.3 Jinja2==3.1.3
ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz"

View File

@ -242,7 +242,7 @@ quit
--create-query-fuzzer-runs=50 \
--queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \
$NEW_TESTS_OPT \
> >(tail -n 100000 > fuzzer.log) \
> fuzzer.log \
2>&1 &
fuzzer_pid=$!
echo "Fuzzer pid is $fuzzer_pid"
@ -390,6 +390,7 @@ rg --text -F '<Fatal>' server.log > fatal.log ||:
dmesg -T > dmesg.log ||:
zstd --threads=0 server.log
zstd --threads=0 fuzzer.log
cat > report.html <<EOF ||:
<!DOCTYPE html>
@ -413,7 +414,7 @@ p.links a { padding: 5px; margin: 3px; background: #FFF; line-height: 2; white-s
<h1>AST Fuzzer for PR <a href="https://github.com/ClickHouse/ClickHouse/pull/${PR_TO_TEST}">#${PR_TO_TEST}</a> @ ${SHA_TO_TEST}</h1>
<p class="links">
<a href="run.log">run.log</a>
<a href="fuzzer.log">fuzzer.log</a>
<a href="fuzzer.log.zst">fuzzer.log.zst</a>
<a href="server.log.zst">server.log.zst</a>
<a href="main.log">main.log</a>
<a href="dmesg.log">dmesg.log</a>

View File

@ -23,13 +23,15 @@ if [ -f /sys/fs/cgroup/cgroup.controllers ]; then
> /sys/fs/cgroup/cgroup.subtree_control
fi
# In case of test hung it is convenient to use pytest --pdb to debug it,
# and on hung you can simply press Ctrl-C and it will spawn a python pdb,
# but on SIGINT dockerd will exit, so ignore it to preserve the daemon.
trap '' INT
# Binding to an IP address without --tlsverify is deprecated. Startup is intentionally being slowed
# unless --tls=false or --tlsverify=false is set
dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
#
# In case of test hung it is convenient to use pytest --pdb to debug it,
# and on hung you can simply press Ctrl-C and it will spawn a python pdb,
# but on SIGINT dockerd will exit, so we spawn new session to ignore SIGINT by
# docker.
# Note, that if you will run it via runner, it will send SIGINT to docker anyway.
setsid dockerd --host=unix:///var/run/docker.sock --tls=false --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
set +e
reties=0

View File

@ -11,14 +11,6 @@ RUN apt-get update -y \
npm \
&& apt-get clean
COPY s3downloader /s3downloader
ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com"
ENV DATASETS="hits visits"
# The following is already done in clickhouse/stateless-test
# RUN npm install -g azurite
# RUN npm install tslib
COPY create.sql /
COPY run.sh /
CMD ["/bin/bash", "/run.sh"]

View File

@ -0,0 +1,333 @@
ATTACH TABLE datasets.hits_v1 UUID '78ebf6a1-d987-4579-b3ec-00c1a087b1f3'
(
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.Key1" Array(String),
"ParsedParams.Key2" Array(String),
"ParsedParams.Key3" Array(String),
"ParsedParams.Key4" Array(String),
"ParsedParams.Key5" Array(String),
"ParsedParams.ValueDouble" Array(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 disk = disk(type = cache, path = '/var/lib/clickhouse/filesystem_caches/', max_size = '4G',
disk = disk(type = web, endpoint = 'https://clickhouse-datasets-web.s3.us-east-1.amazonaws.com/'));
ATTACH TABLE datasets.visits_v1 UUID '5131f834-711f-4168-98a5-968b691a104b'
(
CounterID UInt32,
StartDate Date,
Sign Int8,
IsNew UInt8,
VisitID UInt64,
UserID UInt64,
StartTime DateTime,
Duration UInt32,
UTCStartTime DateTime,
PageViews Int32,
Hits Int32,
IsBounce UInt8,
Referer String,
StartURL String,
RefererDomain String,
StartURLDomain String,
EndURL String,
LinkURL String,
IsDownload UInt8,
TraficSourceID Int8,
SearchEngineID UInt16,
SearchPhrase String,
AdvEngineID UInt8,
PlaceID Int32,
RefererCategories Array(UInt16),
URLCategories Array(UInt16),
URLRegions Array(UInt32),
RefererRegions Array(UInt32),
IsYandex UInt8,
GoalReachesDepth Int32,
GoalReachesURL Int32,
GoalReachesAny Int32,
SocialSourceNetworkID UInt8,
SocialSourcePage String,
MobilePhoneModel String,
ClientEventTime DateTime,
RegionID UInt32,
ClientIP UInt32,
ClientIP6 FixedString(16),
RemoteIP UInt32,
RemoteIP6 FixedString(16),
IPNetworkID UInt32,
SilverlightVersion3 UInt32,
CodeVersion UInt32,
ResolutionWidth UInt16,
ResolutionHeight UInt16,
UserAgentMajor UInt16,
UserAgentMinor UInt16,
WindowClientWidth UInt16,
WindowClientHeight UInt16,
SilverlightVersion2 UInt8,
SilverlightVersion4 UInt16,
FlashVersion3 UInt16,
FlashVersion4 UInt16,
ClientTimeZone Int16,
OS UInt8,
UserAgent UInt8,
ResolutionDepth UInt8,
FlashMajor UInt8,
FlashMinor UInt8,
NetMajor UInt8,
NetMinor UInt8,
MobilePhone UInt8,
SilverlightVersion1 UInt8,
Age UInt8,
Sex UInt8,
Income UInt8,
JavaEnable UInt8,
CookieEnable UInt8,
JavascriptEnable UInt8,
IsMobile UInt8,
BrowserLanguage UInt16,
BrowserCountry UInt16,
Interests UInt16,
Robotness UInt8,
GeneralInterests Array(UInt16),
Params Array(String),
"Goals.ID" Array(UInt32),
"Goals.Serial" Array(UInt32),
"Goals.EventTime" Array(DateTime),
"Goals.Price" Array(Int64),
"Goals.OrderID" Array(String),
"Goals.CurrencyID" Array(UInt32),
WatchIDs Array(UInt64),
ParamSumPrice Int64,
ParamCurrency FixedString(3),
ParamCurrencyID UInt16,
ClickLogID UInt64,
ClickEventID Int32,
ClickGoodEvent Int32,
ClickEventTime DateTime,
ClickPriorityID Int32,
ClickPhraseID Int32,
ClickPageID Int32,
ClickPlaceID Int32,
ClickTypeID Int32,
ClickResourceID Int32,
ClickCost UInt32,
ClickClientIP UInt32,
ClickDomainID UInt32,
ClickURL String,
ClickAttempt UInt8,
ClickOrderID UInt32,
ClickBannerID UInt32,
ClickMarketCategoryID UInt32,
ClickMarketPP UInt32,
ClickMarketCategoryName String,
ClickMarketPPName String,
ClickAWAPSCampaignName String,
ClickPageName String,
ClickTargetType UInt16,
ClickTargetPhraseID UInt64,
ClickContextType UInt8,
ClickSelectType Int8,
ClickOptions String,
ClickGroupBannerID Int32,
OpenstatServiceName String,
OpenstatCampaignID String,
OpenstatAdID String,
OpenstatSourceID String,
UTMSource String,
UTMMedium String,
UTMCampaign String,
UTMContent String,
UTMTerm String,
FromTag String,
HasGCLID UInt8,
FirstVisit DateTime,
PredLastVisit Date,
LastVisit Date,
TotalVisits UInt32,
"TraficSource.ID" Array(Int8),
"TraficSource.SearchEngineID" Array(UInt16),
"TraficSource.AdvEngineID" Array(UInt8),
"TraficSource.PlaceID" Array(UInt16),
"TraficSource.SocialSourceNetworkID" Array(UInt8),
"TraficSource.Domain" Array(String),
"TraficSource.SearchPhrase" Array(String),
"TraficSource.SocialSourcePage" Array(String),
Attendance FixedString(16),
CLID UInt32,
YCLID UInt64,
NormalizedRefererHash UInt64,
SearchPhraseHash UInt64,
RefererDomainHash UInt64,
NormalizedStartURLHash UInt64,
StartURLDomainHash UInt64,
NormalizedEndURLHash UInt64,
TopLevelDomain UInt64,
URLScheme UInt64,
OpenstatServiceNameHash UInt64,
OpenstatCampaignIDHash UInt64,
OpenstatAdIDHash UInt64,
OpenstatSourceIDHash UInt64,
UTMSourceHash UInt64,
UTMMediumHash UInt64,
UTMCampaignHash UInt64,
UTMContentHash UInt64,
UTMTermHash UInt64,
FromHash UInt64,
WebVisorEnabled UInt8,
WebVisorActivity UInt32,
"ParsedParams.Key1" Array(String),
"ParsedParams.Key2" Array(String),
"ParsedParams.Key3" Array(String),
"ParsedParams.Key4" Array(String),
"ParsedParams.Key5" Array(String),
"ParsedParams.ValueDouble" Array(Float64),
"Market.Type" Array(UInt8),
"Market.GoalID" Array(UInt32),
"Market.OrderID" Array(String),
"Market.OrderPrice" Array(Int64),
"Market.PP" Array(UInt32),
"Market.DirectPlaceID" Array(UInt32),
"Market.DirectOrderID" Array(UInt32),
"Market.DirectBannerID" Array(UInt32),
"Market.GoodID" Array(String),
"Market.GoodName" Array(String),
"Market.GoodQuantity" Array(Int32),
"Market.GoodPrice" Array(Int64),
IslandID FixedString(16)
)
ENGINE = CollapsingMergeTree(Sign)
PARTITION BY toYYYYMM(StartDate)
ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)
SAMPLE BY intHash32(UserID)
SETTINGS disk = disk(type = cache, path = '/var/lib/clickhouse/filesystem_caches/', max_size = '4G',
disk = disk(type = web, endpoint = 'https://clickhouse-datasets-web.s3.us-east-1.amazonaws.com/'));

View File

@ -44,6 +44,9 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR
# It is not needed, we will explicitly create tables on s3.
# We do not have statefull tests with s3 storage run in public repository, but this is needed for another repository.
rm /etc/clickhouse-server/config.d/s3_storage_policy_for_merge_tree_by_default.xml
rm /etc/clickhouse-server/config.d/storage_metadata_with_full_object_key.xml
rm /etc/clickhouse-server/config.d/s3_storage_policy_with_template_object_key.xml
fi
function start()
@ -94,21 +97,9 @@ start
setup_logs_replication
# shellcheck disable=SC2086 # No quotes because I want to split it into words.
/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS
chmod 777 -R /var/lib/clickhouse
clickhouse-client --query "SHOW DATABASES"
clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary"
service clickhouse-server restart
# Wait for server to start accepting connections
for _ in {1..120}; do
clickhouse-client --query "SELECT 1" && break
sleep 1
done
clickhouse-client --query "CREATE DATABASE datasets"
clickhouse-client --multiquery < create.sql
clickhouse-client --query "SHOW TABLES FROM datasets"
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then

View File

@ -1,126 +0,0 @@
#!/usr/bin/env python3
# -*- coding: utf-8 -*-
import os
import sys
import time
import tarfile
import logging
import argparse
import requests
import tempfile
DEFAULT_URL = "https://clickhouse-datasets.s3.amazonaws.com"
AVAILABLE_DATASETS = {
"hits": "hits_v1.tar",
"visits": "visits_v1.tar",
}
RETRIES_COUNT = 5
def _get_temp_file_name():
return os.path.join(
tempfile._get_default_tempdir(), next(tempfile._get_candidate_names())
)
def build_url(base_url, dataset):
return os.path.join(base_url, dataset, "partitions", AVAILABLE_DATASETS[dataset])
def download_with_progress(url, path):
logging.info("Downloading from %s to temp path %s", url, path)
for i in range(RETRIES_COUNT):
try:
with open(path, "wb") as f:
response = requests.get(url, stream=True)
response.raise_for_status()
total_length = response.headers.get("content-length")
if total_length is None or int(total_length) == 0:
logging.info(
"No content-length, will download file without progress"
)
f.write(response.content)
else:
dl = 0
total_length = int(total_length)
logging.info("Content length is %ld bytes", total_length)
for data in response.iter_content(chunk_size=4096):
dl += len(data)
f.write(data)
if sys.stdout.isatty():
done = int(50 * dl / total_length)
percent = int(100 * float(dl) / total_length)
sys.stdout.write(
"\r[{}{}] {}%".format(
"=" * done, " " * (50 - done), percent
)
)
sys.stdout.flush()
break
except Exception as ex:
sys.stdout.write("\n")
time.sleep(3)
logging.info("Exception while downloading %s, retry %s", ex, i + 1)
if os.path.exists(path):
os.remove(path)
else:
raise Exception(
"Cannot download dataset from {}, all retries exceeded".format(url)
)
sys.stdout.write("\n")
logging.info("Downloading finished")
def unpack_to_clickhouse_directory(tar_path, clickhouse_path):
logging.info(
"Will unpack data from temp path %s to clickhouse db %s",
tar_path,
clickhouse_path,
)
with tarfile.open(tar_path, "r") as comp_file:
comp_file.extractall(path=clickhouse_path)
logging.info("Unpack finished")
if __name__ == "__main__":
logging.basicConfig(level=logging.INFO)
parser = argparse.ArgumentParser(
description="Simple tool for dowloading datasets for clickhouse from S3"
)
parser.add_argument(
"--dataset-names",
required=True,
nargs="+",
choices=list(AVAILABLE_DATASETS.keys()),
)
parser.add_argument("--url-prefix", default=DEFAULT_URL)
parser.add_argument("--clickhouse-data-path", default="/var/lib/clickhouse/")
args = parser.parse_args()
datasets = args.dataset_names
logging.info("Will fetch following datasets: %s", ", ".join(datasets))
for dataset in datasets:
logging.info("Processing %s", dataset)
temp_archive_path = _get_temp_file_name()
try:
download_url_for_dataset = build_url(args.url_prefix, dataset)
download_with_progress(download_url_for_dataset, temp_archive_path)
unpack_to_clickhouse_directory(temp_archive_path, args.clickhouse_data_path)
except Exception as ex:
logging.info("Some exception occured %s", str(ex))
raise
finally:
logging.info(
"Will remove downloaded file %s from filesystem if it exists",
temp_archive_path,
)
if os.path.exists(temp_archive_path):
os.remove(temp_archive_path)
logging.info("Processing of %s finished", dataset)
logging.info("Fetch finished, enjoy your tables!")

View File

@ -46,7 +46,7 @@ RUN apt-get update -y \
p7zip-full \
&& apt-get clean
RUN pip3 install numpy scipy pandas Jinja2
RUN pip3 install numpy scipy pandas Jinja2 pyarrow
RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \

View File

@ -99,6 +99,16 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
> /etc/clickhouse-server2/config.d/filesystem_caches_path.xml.tmp
mv /etc/clickhouse-server2/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server2/config.d/filesystem_caches_path.xml
sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \
| sed "s|<custom_cached_disks_base_directory replace=\"replace\">/var/lib/clickhouse/filesystem_caches/</custom_cached_disks_base_directory>|<custom_cached_disks_base_directory replace=\"replace\">/var/lib/clickhouse/filesystem_caches_1/</custom_cached_disks_base_directory>|" \
> /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp
mv /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server1/config.d/filesystem_caches_path.xml
sudo cat /etc/clickhouse-server2/config.d/filesystem_caches_path.xml \
| sed "s|<custom_cached_disks_base_directory replace=\"replace\">/var/lib/clickhouse/filesystem_caches/</custom_cached_disks_base_directory>|<custom_cached_disks_base_directory replace=\"replace\">/var/lib/clickhouse/filesystem_caches_2/</custom_cached_disks_base_directory>|" \
> /etc/clickhouse-server2/config.d/filesystem_caches_path.xml.tmp
mv /etc/clickhouse-server2/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server2/config.d/filesystem_caches_path.xml
mkdir -p /var/run/clickhouse-server1
sudo chown clickhouse:clickhouse /var/run/clickhouse-server1
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \
@ -235,6 +245,23 @@ clickhouse-client -q "system flush logs" ||:
# stop logs replication to make it possible to dump logs tables via clickhouse-local
stop_logs_replication
# Try to get logs while server is running
failed_to_save_logs=0
for table in query_log zookeeper_log trace_log transactions_info_log metric_log
do
err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst; } 2>&1 )
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 )
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
err=$( { clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst; } 2>&1 )
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
fi
done
# Stop server so we can safely read data with clickhouse-local.
# Why do we read data with clickhouse-local?
# Because it's the simplest way to read it when server has crashed.
@ -254,21 +281,25 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR
data_path_config="--config-file=/etc/clickhouse-server/config.xml"
fi
# Compress tables.
#
# NOTE:
# - that due to tests with s3 storage we cannot use /var/lib/clickhouse/data
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log
do
clickhouse-local "$data_path_config" --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
fi
done
# If server crashed dump system logs with clickhouse-local
if [ $failed_to_save_logs -ne 0 ]; then
# Compress tables.
#
# NOTE:
# - that due to tests with s3 storage we cannot use /var/lib/clickhouse/data
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log metric_log
do
clickhouse-local "$data_path_config" --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
fi
done
fi
# Also export trace log in flamegraph-friendly format.
for trace_type in CPU Memory Real

View File

@ -236,6 +236,10 @@ function check_logs_for_critical_errors()
&& echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \
|| echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv
rg -Fa "it is lost forever" /var/log/clickhouse-server/clickhouse-server*.log | grep 'SharedMergeTreePartCheckThread' > /dev/null \
&& echo -e "Lost forever for SharedMergeTree$FAIL" >> /test_output/test_results.tsv \
|| echo -e "No SharedMergeTree lost forever in clickhouse-server.log$OK" >> /test_output/test_results.tsv
# Remove file no_such_key_errors.txt if it's empty
[ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt

View File

@ -23,8 +23,6 @@ RUN apt-get update -y \
COPY run.sh /
ENV DATASETS="hits visits"
ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com"
ENV EXPORT_S3_STORAGE_POLICIES=1
CMD ["/bin/bash", "/run.sh"]

View File

@ -59,12 +59,11 @@ start
setup_logs_replication
# shellcheck disable=SC2086 # No quotes because I want to split it into words.
/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS
chmod 777 -R /var/lib/clickhouse
clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary"
clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test"
clickhouse-client --query "CREATE DATABASE datasets"
clickhouse-client --multiquery < create.sql
clickhouse-client --query "SHOW TABLES FROM datasets"
clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test"
stop
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log
@ -193,6 +192,7 @@ stop
# Let's enable S3 storage by default
export USE_S3_STORAGE_FOR_MERGE_TREE=1
export RANDOMIZE_OBJECT_KEY_TYPE=1
export ZOOKEEPER_FAULT_INJECTION=1
configure

View File

@ -56,6 +56,9 @@ echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/sys
# Install previous release packages
install_packages previous_release_package_folder
# Save old settings from system table for settings changes check
clickhouse-local -q "select * from system.settings format Native" > old_settings.native
# Initial run without S3 to create system.*_log on local file system to make it
# available for dump via clickhouse-local
configure
@ -78,6 +81,7 @@ remove_keeper_config "create_if_not_exists" "[01]"
rm /etc/clickhouse-server/config.d/merge_tree.xml
rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml
rm /etc/clickhouse-server/config.d/zero_copy_destructive_operations.xml
rm /etc/clickhouse-server/config.d/storage_conf_02963.xml
rm /etc/clickhouse-server/users.d/nonconst_timezone.xml
rm /etc/clickhouse-server/users.d/s3_cache_new.xml
rm /etc/clickhouse-server/users.d/replicated_ddl_entry.xml
@ -117,6 +121,7 @@ sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_defau
rm /etc/clickhouse-server/config.d/merge_tree.xml
rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml
rm /etc/clickhouse-server/config.d/zero_copy_destructive_operations.xml
rm /etc/clickhouse-server/config.d/storage_conf_02963.xml
rm /etc/clickhouse-server/users.d/nonconst_timezone.xml
rm /etc/clickhouse-server/users.d/s3_cache_new.xml
rm /etc/clickhouse-server/users.d/replicated_ddl_entry.xml
@ -150,6 +155,63 @@ install_packages package_folder
export ZOOKEEPER_FAULT_INJECTION=1
configure
# Check that all new/changed setting were added in settings changes history.
# Some settings can be different for builds with sanitizers, so we check
# settings changes only for non-sanitizer builds.
IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM system.build_options WHERE name = 'CXX_FLAGS'")
if [ "${IS_SANITIZED}" -eq "0" ]
then
clickhouse-local -q "select * from system.settings format Native" > new_settings.native
clickhouse-local -nmq "
CREATE TABLE old_settings AS file('old_settings.native');
CREATE TABLE new_settings AS file('new_settings.native');
SELECT
name,
new_settings.value AS new_value,
old_settings.value AS old_value
FROM new_settings
LEFT JOIN old_settings ON new_settings.name = old_settings.name
WHERE (new_settings.value != old_settings.value) AND (name NOT IN (
SELECT arrayJoin(tupleElement(changes, 'name'))
FROM system.settings_changes
WHERE version = extract(version(), '^(?:\\d+\\.\\d+)')
))
SETTINGS join_use_nulls = 1
INTO OUTFILE 'changed_settings.txt'
FORMAT PrettyCompactNoEscapes;
SELECT name
FROM new_settings
WHERE (name NOT IN (
SELECT name
FROM old_settings
)) AND (name NOT IN (
SELECT arrayJoin(tupleElement(changes, 'name'))
FROM system.settings_changes
WHERE version = extract(version(), '^(?:\\d+\\.\\d+)')
))
INTO OUTFILE 'new_settings.txt'
FORMAT PrettyCompactNoEscapes;
"
if [ -s changed_settings.txt ]
then
mv changed_settings.txt /test_output/
echo -e "Changed settings are not reflected in settings changes history (see changed_settings.txt)$FAIL$(head_escaped /test_output/changed_settings.txt)" >> /test_output/test_results.tsv
else
echo -e "There are no changed settings or they are reflected in settings changes history$OK" >> /test_output/test_results.tsv
fi
if [ -s new_settings.txt ]
then
mv new_settings.txt /test_output/
echo -e "New settings are not reflected in settings changes history (see new_settings.txt)$FAIL$(head_escaped /test_output/new_settings.txt)" >> /test_output/test_results.tsv
else
echo -e "There are no new settings or they are reflected in settings changes history$OK" >> /test_output/test_results.tsv
fi
fi
# Just in case previous version left some garbage in zk
sudo cat /etc/clickhouse-server/config.d/lost_forever_check.xml \
| sed "s|>1<|>0<|g" \
@ -255,6 +317,8 @@ clickhouse-local --structure "test String, res String, time Nullable(Float32), d
(test like '%Fatal message%') DESC,
(test like '%Error message%') DESC,
(test like '%previous release%') DESC,
(test like '%Changed settings%') DESC,
(test like '%New settings%') DESC,
rowNumberInAllBlocks()
LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv

View File

@ -0,0 +1,51 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.10.6.60-stable (68907bbe643) FIXME as compared to v23.10.5.20-stable (e84001e5c61)
#### Improvement
* Backported in [#58493](https://github.com/ClickHouse/ClickHouse/issues/58493): Fix transfer query to MySQL compatible query. Fixes [#57253](https://github.com/ClickHouse/ClickHouse/issues/57253). Fixes [#52654](https://github.com/ClickHouse/ClickHouse/issues/52654). Fixes [#56729](https://github.com/ClickHouse/ClickHouse/issues/56729). [#56456](https://github.com/ClickHouse/ClickHouse/pull/56456) ([flynn](https://github.com/ucasfl)).
* Backported in [#57659](https://github.com/ClickHouse/ClickHouse/issues/57659): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
#### Build/Testing/Packaging Improvement
* Backported in [#57586](https://github.com/ClickHouse/ClickHouse/issues/57586): Fix issue caught in https://github.com/docker-library/official-images/pull/15846. [#57571](https://github.com/ClickHouse/ClickHouse/pull/57571) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix ALTER COLUMN with ALIAS [#56493](https://github.com/ClickHouse/ClickHouse/pull/56493) ([Nikolay Degterinsky](https://github.com/evillique)).
* Prevent incompatible ALTER of projection columns [#56948](https://github.com/ClickHouse/ClickHouse/pull/56948) ([Amos Bird](https://github.com/amosbird)).
* Fix segfault after ALTER UPDATE with Nullable MATERIALIZED column [#57147](https://github.com/ClickHouse/ClickHouse/pull/57147) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix incorrect JOIN plan optimization with partially materialized normal projection [#57196](https://github.com/ClickHouse/ClickHouse/pull/57196) ([Amos Bird](https://github.com/amosbird)).
* Fix `ReadonlyReplica` metric for all cases [#57267](https://github.com/ClickHouse/ClickHouse/pull/57267) ([Antonio Andelic](https://github.com/antonio2368)).
* Background merges correctly use temporary data storage in the cache [#57275](https://github.com/ClickHouse/ClickHouse/pull/57275) ([vdimir](https://github.com/vdimir)).
* MergeTree mutations reuse source part index granularity [#57352](https://github.com/ClickHouse/ClickHouse/pull/57352) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix function jsonMergePatch for partially const columns [#57379](https://github.com/ClickHouse/ClickHouse/pull/57379) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
* bugfix: correctly parse SYSTEM STOP LISTEN TCP SECURE [#57483](https://github.com/ClickHouse/ClickHouse/pull/57483) ([joelynch](https://github.com/joelynch)).
* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix lost blobs after dropping a replica with broken detached parts [#58333](https://github.com/ClickHouse/ClickHouse/pull/58333) ([Alexander Tokmakov](https://github.com/tavplubix)).
* MergeTreePrefetchedReadPool disable for LIMIT only queries [#58505](https://github.com/ClickHouse/ClickHouse/pull/58505) ([Maksim Kita](https://github.com/kitaisreal)).
#### NO CL CATEGORY
* Backported in [#57916](https://github.com/ClickHouse/ClickHouse/issues/57916):. [#57909](https://github.com/ClickHouse/ClickHouse/pull/57909) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Remove heavy rust stable toolchain [#57905](https://github.com/ClickHouse/ClickHouse/pull/57905) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).
* Fix test_user_valid_until [#58409](https://github.com/ClickHouse/ClickHouse/pull/58409) ([Nikolay Degterinsky](https://github.com/evillique)).

View File

@ -0,0 +1,26 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.11.4.24-stable (e79d840d7fe) FIXME as compared to v23.11.3.23-stable (a14ab450b0e)
#### Bug Fix (user-visible misbehavior in an official stable release)
* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix lost blobs after dropping a replica with broken detached parts [#58333](https://github.com/ClickHouse/ClickHouse/pull/58333) ([Alexander Tokmakov](https://github.com/tavplubix)).
* MergeTreePrefetchedReadPool disable for LIMIT only queries [#58505](https://github.com/ClickHouse/ClickHouse/pull/58505) ([Maksim Kita](https://github.com/kitaisreal)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Handle another case for preprocessing in Keeper [#58308](https://github.com/ClickHouse/ClickHouse/pull/58308) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix test_user_valid_until [#58409](https://github.com/ClickHouse/ClickHouse/pull/58409) ([Nikolay Degterinsky](https://github.com/evillique)).

View File

@ -0,0 +1,32 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.12.2.59-stable (17ab210e761) FIXME as compared to v23.12.1.1368-stable (a2faa65b080)
#### Backward Incompatible Change
* Backported in [#58389](https://github.com/ClickHouse/ClickHouse/issues/58389): The MergeTree setting `clean_deleted_rows` is deprecated, it has no effect anymore. The `CLEANUP` keyword for `OPTIMIZE` is not allowed by default (unless `allow_experimental_replacing_merge_with_cleanup` is enabled). [#58316](https://github.com/ClickHouse/ClickHouse/pull/58316) ([Alexander Tokmakov](https://github.com/tavplubix)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix lost blobs after dropping a replica with broken detached parts [#58333](https://github.com/ClickHouse/ClickHouse/pull/58333) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix segfault when graphite table does not have agg function [#58453](https://github.com/ClickHouse/ClickHouse/pull/58453) ([Duc Canh Le](https://github.com/canhld94)).
* MergeTreePrefetchedReadPool disable for LIMIT only queries [#58505](https://github.com/ClickHouse/ClickHouse/pull/58505) ([Maksim Kita](https://github.com/kitaisreal)).
#### NO CL ENTRY
* NO CL ENTRY: 'Revert "Refreshable materialized views (takeover)"'. [#58296](https://github.com/ClickHouse/ClickHouse/pull/58296) ([Alexander Tokmakov](https://github.com/tavplubix)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Fix an error in the release script - it didn't allow to make 23.12. [#58288](https://github.com/ClickHouse/ClickHouse/pull/58288) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Update version_date.tsv and changelogs after v23.12.1.1368-stable [#58290](https://github.com/ClickHouse/ClickHouse/pull/58290) ([robot-clickhouse](https://github.com/robot-clickhouse)).
* Fix test_storage_s3_queue/test.py::test_drop_table [#58293](https://github.com/ClickHouse/ClickHouse/pull/58293) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Handle another case for preprocessing in Keeper [#58308](https://github.com/ClickHouse/ClickHouse/pull/58308) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix test_user_valid_until [#58409](https://github.com/ClickHouse/ClickHouse/pull/58409) ([Nikolay Degterinsky](https://github.com/evillique)).

View File

@ -0,0 +1,36 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.3.19.32-lts (c4d4ca8ec02) FIXME as compared to v23.3.18.15-lts (7228475d77a)
#### Backward Incompatible Change
* Backported in [#57840](https://github.com/ClickHouse/ClickHouse/issues/57840): Remove function `arrayFold` because it has a bug. This closes [#57816](https://github.com/ClickHouse/ClickHouse/issues/57816). This closes [#57458](https://github.com/ClickHouse/ClickHouse/issues/57458). [#57836](https://github.com/ClickHouse/ClickHouse/pull/57836) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### Improvement
* Backported in [#58489](https://github.com/ClickHouse/ClickHouse/issues/58489): Fix transfer query to MySQL compatible query. Fixes [#57253](https://github.com/ClickHouse/ClickHouse/issues/57253). Fixes [#52654](https://github.com/ClickHouse/ClickHouse/issues/52654). Fixes [#56729](https://github.com/ClickHouse/ClickHouse/issues/56729). [#56456](https://github.com/ClickHouse/ClickHouse/pull/56456) ([flynn](https://github.com/ucasfl)).
* Backported in [#57653](https://github.com/ClickHouse/ClickHouse/issues/57653): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
#### Build/Testing/Packaging Improvement
* Backported in [#57580](https://github.com/ClickHouse/ClickHouse/issues/57580): Fix issue caught in https://github.com/docker-library/official-images/pull/15846. [#57571](https://github.com/ClickHouse/ClickHouse/pull/57571) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Prevent incompatible ALTER of projection columns [#56948](https://github.com/ClickHouse/ClickHouse/pull/56948) ([Amos Bird](https://github.com/amosbird)).
* Fix segfault after ALTER UPDATE with Nullable MATERIALIZED column [#57147](https://github.com/ClickHouse/ClickHouse/pull/57147) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix incorrect JOIN plan optimization with partially materialized normal projection [#57196](https://github.com/ClickHouse/ClickHouse/pull/57196) ([Amos Bird](https://github.com/amosbird)).
* MergeTree mutations reuse source part index granularity [#57352](https://github.com/ClickHouse/ClickHouse/pull/57352) ([Maksim Kita](https://github.com/kitaisreal)).
* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).

View File

@ -0,0 +1,47 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.8.9.54-lts (192a1d231fa) FIXME as compared to v23.8.8.20-lts (5e012a03bf2)
#### Improvement
* Backported in [#57668](https://github.com/ClickHouse/ClickHouse/issues/57668): Output valid JSON/XML on excetpion during HTTP query execution. Add setting `http_write_exception_in_output_format` to enable/disable this behaviour (enabled by default). [#52853](https://github.com/ClickHouse/ClickHouse/pull/52853) ([Kruglov Pavel](https://github.com/Avogar)).
* Backported in [#58491](https://github.com/ClickHouse/ClickHouse/issues/58491): Fix transfer query to MySQL compatible query. Fixes [#57253](https://github.com/ClickHouse/ClickHouse/issues/57253). Fixes [#52654](https://github.com/ClickHouse/ClickHouse/issues/52654). Fixes [#56729](https://github.com/ClickHouse/ClickHouse/issues/56729). [#56456](https://github.com/ClickHouse/ClickHouse/pull/56456) ([flynn](https://github.com/ucasfl)).
* Backported in [#57238](https://github.com/ClickHouse/ClickHouse/issues/57238): Fetching a part waits when that part is fully committed on remote replica. It is better not send part in PreActive state. In case of zero copy this is mandatory restriction. [#56808](https://github.com/ClickHouse/ClickHouse/pull/56808) ([Sema Checherinda](https://github.com/CheSema)).
* Backported in [#57655](https://github.com/ClickHouse/ClickHouse/issues/57655): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)).
#### Build/Testing/Packaging Improvement
* Backported in [#57582](https://github.com/ClickHouse/ClickHouse/issues/57582): Fix issue caught in https://github.com/docker-library/official-images/pull/15846. [#57571](https://github.com/ClickHouse/ClickHouse/pull/57571) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Flatten only true Nested type if flatten_nested=1, not all Array(Tuple) [#56132](https://github.com/ClickHouse/ClickHouse/pull/56132) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix ALTER COLUMN with ALIAS [#56493](https://github.com/ClickHouse/ClickHouse/pull/56493) ([Nikolay Degterinsky](https://github.com/evillique)).
* Prevent incompatible ALTER of projection columns [#56948](https://github.com/ClickHouse/ClickHouse/pull/56948) ([Amos Bird](https://github.com/amosbird)).
* Fix segfault after ALTER UPDATE with Nullable MATERIALIZED column [#57147](https://github.com/ClickHouse/ClickHouse/pull/57147) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix incorrect JOIN plan optimization with partially materialized normal projection [#57196](https://github.com/ClickHouse/ClickHouse/pull/57196) ([Amos Bird](https://github.com/amosbird)).
* Fix `ReadonlyReplica` metric for all cases [#57267](https://github.com/ClickHouse/ClickHouse/pull/57267) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix working with read buffers in StreamingFormatExecutor [#57438](https://github.com/ClickHouse/ClickHouse/pull/57438) ([Kruglov Pavel](https://github.com/Avogar)).
* bugfix: correctly parse SYSTEM STOP LISTEN TCP SECURE [#57483](https://github.com/ClickHouse/ClickHouse/pull/57483) ([joelynch](https://github.com/joelynch)).
* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)).
* Disable system.kafka_consumers by default (due to possible live memory leak) [#57822](https://github.com/ClickHouse/ClickHouse/pull/57822) ([Azat Khuzhin](https://github.com/azat)).
* Fix invalid memory access in BLAKE3 (Rust) [#57876](https://github.com/ClickHouse/ClickHouse/pull/57876) ([Raúl Marín](https://github.com/Algunenano)).
* Normalize function names in CREATE INDEX [#57906](https://github.com/ClickHouse/ClickHouse/pull/57906) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix invalid preprocessing on Keeper [#58069](https://github.com/ClickHouse/ClickHouse/pull/58069) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix Integer overflow in Poco::UTF32Encoding [#58073](https://github.com/ClickHouse/ClickHouse/pull/58073) ([Andrey Fedotov](https://github.com/anfedotoff)).
* Remove parallel parsing for JSONCompactEachRow [#58181](https://github.com/ClickHouse/ClickHouse/pull/58181) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix parallel parsing for JSONCompactEachRow [#58250](https://github.com/ClickHouse/ClickHouse/pull/58250) ([Kruglov Pavel](https://github.com/Avogar)).
#### NO CL ENTRY
* NO CL ENTRY: 'Update PeekableWriteBuffer.cpp'. [#57701](https://github.com/ClickHouse/ClickHouse/pull/57701) ([Kruglov Pavel](https://github.com/Avogar)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Remove heavy rust stable toolchain [#57905](https://github.com/ClickHouse/ClickHouse/pull/57905) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Fix docker image for integration tests (fixes CI) [#57952](https://github.com/ClickHouse/ClickHouse/pull/57952) ([Azat Khuzhin](https://github.com/azat)).

View File

@ -11,6 +11,7 @@ sidebar_label: 2023
* Remove the `status_info` configuration option and dictionaries status from the default Prometheus handler. [#54090](https://github.com/ClickHouse/ClickHouse/pull/54090) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* The experimental parts metadata cache is removed from the codebase. [#54215](https://github.com/ClickHouse/ClickHouse/pull/54215) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Disable setting `input_format_json_try_infer_numbers_from_strings` by default, so we don't try to infer numbers from strings in JSON formats by default to avoid possible parsing errors when sample data contains strings that looks like a number. [#55099](https://github.com/ClickHouse/ClickHouse/pull/55099) ([Kruglov Pavel](https://github.com/Avogar)).
* IPv6 bloom filter indexes created prior to March 2023 are not compatible with current version and have to be rebuilt. [#54200](https://github.com/ClickHouse/ClickHouse/pull/54200) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
#### New Feature
* Added new type of authentication based on SSH keys. It works only for Native TCP protocol. [#41109](https://github.com/ClickHouse/ClickHouse/pull/41109) ([George Gamezardashvili](https://github.com/InfJoker)).

View File

@ -16,7 +16,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
...
) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause'])
) ENGINE = MySQL({host:port, database, table, user, password[, replace_query, on_duplicate_clause] | named_collection[, option=value [,..]]})
SETTINGS
[ connection_pool_size=16, ]
[ connection_max_tries=3, ]
@ -42,23 +42,17 @@ The MySQL Table Engine is currently not available on the ClickHouse builds for M
**Engine Parameters**
- `host:port` — MySQL server address.
- `database` — Remote database name.
- `table` — Remote table name.
- `user` — MySQL user.
- `password` — User password.
- `replace_query` — Flag that converts `INSERT INTO` queries to `REPLACE INTO`. If `replace_query=1`, the query is substituted.
- `on_duplicate_clause` — The `ON DUPLICATE KEY on_duplicate_clause` expression that is added to the `INSERT` query.
Example: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, where `on_duplicate_clause` is `UPDATE c2 = c2 + 1`. See the [MySQL documentation](https://dev.mysql.com/doc/refman/8.0/en/insert-on-duplicate.html) to find which `on_duplicate_clause` you can use with the `ON DUPLICATE KEY` clause.
To specify `on_duplicate_clause` you need to pass `0` to the `replace_query` parameter. If you simultaneously pass `replace_query = 1` and `on_duplicate_clause`, ClickHouse generates an exception.
Arguments also can be passed using [named collections](/docs/en/operations/named-collections.md). In this case `host` and `port` should be specified separately. This approach is recommended for production environment.
Simple `WHERE` clauses such as `=, !=, >, >=, <, <=` are executed on the MySQL server.
The rest of the conditions and the `LIMIT` sampling constraint are executed in ClickHouse only after the query to MySQL finishes.
@ -71,7 +65,7 @@ CREATE TABLE test_replicas (id UInt32, name String, age UInt32, money UInt32) EN
## Usage Example {#usage-example}
Table in MySQL:
Create table in MySQL:
``` text
mysql> CREATE TABLE `test`.`test` (
@ -94,7 +88,7 @@ mysql> select * from test;
1 row in set (0,00 sec)
```
Table in ClickHouse, retrieving data from the MySQL table created above:
Create table in ClickHouse using plain arguments:
``` sql
CREATE TABLE mysql_table
@ -105,6 +99,25 @@ CREATE TABLE mysql_table
ENGINE = MySQL('localhost:3306', 'test', 'test', 'bayonet', '123')
```
Or using [named collections](/docs/en/operations/named-collections.md):
```sql
CREATE NAMED COLLECTION creds AS
host = 'localhost',
port = 3306,
database = 'test',
user = 'bayonet',
password = '123';
CREATE TABLE mysql_table
(
`float_nullable` Nullable(Float32),
`int_id` Int32
)
ENGINE = MySQL(creds, table='test')
```
Retrieving data from MySQL table:
``` sql
SELECT * FROM mysql_table
```

View File

@ -16,7 +16,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 type1 [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
name2 type2 [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
...
) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'[, `schema`]);
) ENGINE = PostgreSQL({host:port, database, table, user, password[, schema, [, on_conflict]] | named_collection[, option=value [,..]]})
```
See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query.
@ -35,31 +35,25 @@ The table structure can differ from the original PostgreSQL table structure:
- `user` — PostgreSQL user.
- `password` — User password.
- `schema` — Non-default table schema. Optional.
- `on conflict ...` — example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient.
- `on_conflict` — Conflict resolution strategy. Example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient.
or via config (since version 21.11):
[Named collections](/docs/en/operations/named-collections.md) (available since version 21.11) are recommended for production environment. Here is an example:
```
<named_collections>
<postgres1>
<host></host>
<port></port>
<user></user>
<password></password>
<table></table>
</postgres1>
<postgres2>
<host></host>
<port></port>
<user></user>
<password></password>
</postgres2>
<postgres_creds>
<host>localhost</host>
<port>5432</port>
<user>postgres</user>
<password>****</password>
<schema>schema1</schema>
</postgres_creds>
</named_collections>
```
Some parameters can be overridden by key value arguments:
``` sql
SELECT * FROM postgresql(postgres1, schema='schema1', table='table1');
SELECT * FROM postgresql(postgres_creds, table='table1');
```
## Implementation Details {#implementation-details}

View File

@ -16,30 +16,32 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name
name1 [type1],
name2 [type2],
...
) ENGINE = Redis(host:port[, db_index[, password[, pool_size]]]) PRIMARY KEY(primary_key_name);
) ENGINE = Redis({host:port[, db_index[, password[, pool_size]]] | named_collection[, option=value [,..]] })
PRIMARY KEY(primary_key_name);
```
**Engine Parameters**
- `host:port` — Redis server address, you can ignore port and default Redis port 6379 will be used.
- `db_index` — Redis db index range from 0 to 15, default is 0.
- `password` — User password, default is blank string.
- `pool_size` — Redis max connection pool size, default is 16.
- `primary_key_name` - any column name in the column list.
- `primary` must be specified, it supports only one column in the primary key. The primary key will be serialized in binary as a Redis key.
:::note Serialization
`PRIMARY KEY` supports only one column. The primary key will be serialized in binary as a Redis key.
Columns other than the primary key will be serialized in binary as Redis value in corresponding order.
:::
- columns other than the primary key will be serialized in binary as Redis value in corresponding order.
Arguments also can be passed using [named collections](/docs/en/operations/named-collections.md). In this case `host` and `port` should be specified separately. This approach is recommended for production environment. At this moment, all parameters passed using named collections to redis are required.
- queries with key equals or in filtering will be optimized to multi keys lookup from Redis. If queries without filtering key full table scan will happen which is a heavy operation.
:::note Filtering
Queries with `key equals` or `in filtering` will be optimized to multi keys lookup from Redis. If queries without filtering key full table scan will happen which is a heavy operation.
:::
## Usage Example {#usage-example}
Create a table in ClickHouse which allows to read data from Redis:
Create a table in ClickHouse using `Redis` engine with plain arguments:
``` sql
CREATE TABLE redis_table
@ -52,6 +54,31 @@ CREATE TABLE redis_table
ENGINE = Redis('redis1:6379') PRIMARY KEY(key);
```
Or using [named collections](/docs/en/operations/named-collections.md):
```
<named_collections>
<redis_creds>
<host>localhost</host>
<port>6379</port>
<password>****</password>
<pool_size>16</pool_size>
<db_index>s0</db_index>
</redis_creds>
</named_collections>
```
```sql
CREATE TABLE redis_table
(
`key` String,
`v1` UInt32,
`v2` String,
`v3` Float32
)
ENGINE = Redis(redis_creds) PRIMARY KEY(key);
```
Insert:
```sql

View File

@ -11,7 +11,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec
``` sql
CREATE TABLE s3_queue_engine_table (name String, value UInt32)
ENGINE = S3Queue(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression])
ENGINE = S3Queue(path, [NOSIGN, | aws_access_key_id, aws_secret_access_key,] format, [compression])
[SETTINGS]
[mode = 'unordered',]
[after_processing = 'keep',]

View File

@ -39,8 +39,8 @@ If you need to update rows frequently, we recommend using the [`ReplacingMergeTr
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY],
name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY],
name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)],
name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)],
...
INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1],
INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2],
@ -56,7 +56,7 @@ ORDER BY expr
[DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ]
[WHERE conditions]
[GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ]
[SETTINGS name=value, ...]
[SETTINGS name = value, ...]
```
For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md).
@ -504,24 +504,25 @@ Indexes of type `set` can be utilized by all functions. The other index types ar
| Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | inverted |
|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------|----------|
| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notEquals(!=, &lt;&gt;)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#equals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notEquals(!=, &lt;&gt;)](/docs/en/sql-reference/functions/comparison-functions.md/#notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [like](/docs/en/sql-reference/functions/string-search-functions.md/#like) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [match](/docs/en/sql-reference/functions/string-search-functions.md/#match) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ |
| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | ✔ |
| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | ✔ |
| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ |
| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ |
| [hasAny](/docs/en/sql-reference/functions/array-functions#function-hasAny) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ |
| [hasAll](/docs/en/sql-reference/functions/array-functions#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ |
| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | ✔ |
| [in](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [notIn](/docs/en/sql-reference/functions/in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ |
| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#less) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#greater) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [empty](/docs/en/sql-reference/functions/array-functions/#empty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [notEmpty](/docs/en/sql-reference/functions/array-functions/#notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | ✗ |
| [has](/docs/en/sql-reference/functions/array-functions/#has) | ✗ | ✗ | ✔ | ✔ | ✔ | ✔ |
| [hasAny](/docs/en/sql-reference/functions/array-functions/#hasany) | ✗ | ✗ | ✔ | ✔ | ✔ | ✗ |
| [hasAll](/docs/en/sql-reference/functions/array-functions/#hasall) | ✗ | ✗ | ✗ | ✗ | ✔ | ✗ |
| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ |
| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | ✔ |
| hasTokenCaseInsensitive (*) | ✗ | ✗ | ✗ | ✔ | ✗ | ✗ |
@ -619,7 +620,7 @@ The `TTL` clause cant be used for key columns.
#### Creating a table with `TTL`:
``` sql
CREATE TABLE example_table
CREATE TABLE tab
(
d DateTime,
a Int TTL d + INTERVAL 1 MONTH,
@ -634,7 +635,7 @@ ORDER BY d;
#### Adding TTL to a column of an existing table
``` sql
ALTER TABLE example_table
ALTER TABLE tab
MODIFY COLUMN
c String TTL d + INTERVAL 1 DAY;
```
@ -642,7 +643,7 @@ ALTER TABLE example_table
#### Altering TTL of the column
``` sql
ALTER TABLE example_table
ALTER TABLE tab
MODIFY COLUMN
c String TTL d + INTERVAL 1 MONTH;
```
@ -680,7 +681,7 @@ If a column is not part of the `GROUP BY` expression and is not set explicitly i
#### Creating a table with `TTL`:
``` sql
CREATE TABLE example_table
CREATE TABLE tab
(
d DateTime,
a Int
@ -696,7 +697,7 @@ TTL d + INTERVAL 1 MONTH DELETE,
#### Altering `TTL` of the table:
``` sql
ALTER TABLE example_table
ALTER TABLE tab
MODIFY TTL d + INTERVAL 1 DAY;
```
@ -1143,6 +1144,8 @@ Optional parameters:
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
- `read_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of read requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk).
- `write_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of write requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk).
- `key_template` — Define the format with which the object keys are generated. By default, Clickhouse takes `root path` from `endpoint` option and adds random generated suffix. That suffix is a dir with 3 random symbols and a file name with 29 random symbols. With that option you have a full control how to the object keys are generated. Some usage scenarios require having random symbols in the prefix or in the middle of object key. For example: `[a-z]{3}-prefix-random/constant-part/random-middle-[a-z]{3}/random-suffix-[a-z]{29}`. The value is parsed with [`re2`](https://github.com/google/re2/wiki/Syntax). Only some subset of the syntax is supported. Check if your preferred format is supported before using that option. Disk isn't initialized if clickhouse is unable to generate a key by the value of `key_template`. It requires enabled feature flag [storage_metadata_write_full_object_key](/docs/en/operations/settings/settings#storage_metadata_write_full_object_key). It forbids declaring the `root path` in `endpoint` option. It requires definition of the option `key_compatibility_prefix`.
- `key_compatibility_prefix` — That option is required when option `key_template` is in use. In order to be able to read the objects keys which were stored in the metadata files with the metadata version lower that `VERSION_FULL_OBJECT_KEY`, the previous `root path` from the `endpoint` option should be set here.
### Configuring the cache
@ -1363,7 +1366,7 @@ In this sample configuration:
The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`.
``` sql
CREATE TABLE example_table
CREATE TABLE tab
(
a Int64 STATISTIC(tdigest, uniq),
b Float64
@ -1375,8 +1378,8 @@ ORDER BY a
We can also manipulate statistics with `ALTER` statements.
```sql
ALTER TABLE example_table ADD STATISTIC b TYPE tdigest, uniq;
ALTER TABLE example_table DROP STATISTIC a;
ALTER TABLE tab ADD STATISTIC b TYPE tdigest, uniq;
ALTER TABLE tab DROP STATISTIC a;
```
These lightweight statistics aggregate information about distribution of values in columns.
@ -1391,3 +1394,43 @@ They can be used for query optimization when we enable `set allow_statistic_opti
- `uniq`
Estimate the number of distinct values of a column.
## Column-level Settings {#column-level-settings}
Certain MergeTree settings can be override at column level:
- `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table.
- `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark.
Example:
```sql
CREATE TABLE tab
(
id Int64,
document String SETTINGS (min_compress_block_size = 16777216, max_compress_block_size = 16777216)
)
ENGINE = MergeTree
ORDER BY id
```
Column-level settings can be modified or removed using [ALTER MODIFY COLUMN](/docs/en/sql-reference/statements/alter/column.md), for example:
- Remove `SETTINGS` from column declaration:
```sql
ALTER TABLE tab MODIFY COLUMN document REMOVE SETTINGS;
```
- Modify a setting:
```sql
ALTER TABLE tab MODIFY COLUMN document MODIFY SETTING min_compress_block_size = 8192;
```
- Reset one or more settings, also removes the setting declaration in the column expression of the table's CREATE query.
```sql
ALTER TABLE tab MODIFY COLUMN document RESET SETTING min_compress_block_size;
```
>>>>>>> master

View File

@ -25,7 +25,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[ORDER BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[SETTINGS name=value, ...]
[SETTINGS name=value, clean_deleted_rows=value, ...]
```
For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md).
@ -88,6 +88,53 @@ SELECT * FROM mySecondReplacingMT FINAL;
└─────┴─────────┴─────────────────────┘
```
### is_deleted
`is_deleted` — Name of a column used during a merge to determine whether the data in this row represents the state or is to be deleted; `1` is a “deleted“ row, `0` is a “state“ row.
Column data type — `UInt8`.
:::note
`is_deleted` can only be enabled when `ver` is used.
The row is deleted when `OPTIMIZE ... FINAL CLEANUP` or `OPTIMIZE ... FINAL` is used, or if the engine setting `clean_deleted_rows` has been set to `Always`.
No matter the operation on the data, the version must be increased. If two inserted rows have the same version number, the last inserted row is the one kept.
:::
Example:
```sql
-- with ver and is_deleted
CREATE OR REPLACE TABLE myThirdReplacingMT
(
`key` Int64,
`someCol` String,
`eventTime` DateTime,
`is_deleted` UInt8
)
ENGINE = ReplacingMergeTree(eventTime, is_deleted)
ORDER BY key;
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 0);
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 1);
select * from myThirdReplacingMT final;
0 rows in set. Elapsed: 0.003 sec.
-- delete rows with is_deleted
OPTIMIZE TABLE myThirdReplacingMT FINAL CLEANUP;
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 00:00:00', 0);
select * from myThirdReplacingMT final;
┌─key─┬─someCol─┬───────────eventTime─┬─is_deleted─┐
│ 1 │ first │ 2020-01-01 00:00:00 │ 0 │
└─────┴─────────┴─────────────────────┴────────────┘
```
## Query clauses
When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.

View File

@ -25,8 +25,7 @@ The steps below will easily work on a local install of ClickHouse too. The only
1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the result:
```sql
DESCRIBE s3Cluster(
'default',
DESCRIBE s3(
'https://clickhouse-public-datasets.s3.amazonaws.com/youtube/original/files/*.zst',
'JSONLines'
);
@ -35,29 +34,29 @@ DESCRIBE s3Cluster(
ClickHouse infers the following schema from the JSON file:
```response
┌─name────────────────┬─type─────────────────────────────────┐
│ id │ Nullable(String) │
│ fetch_date │ Nullable(Int64)
│ upload_date │ Nullable(String) │
│ title │ Nullable(String) │
│ uploader_id │ Nullable(String) │
│ uploader │ Nullable(String) │
│ uploader_sub_count │ Nullable(Int64) │
│ is_age_limit │ Nullable(Bool) │
│ view_count │ Nullable(Int64) │
│ like_count │ Nullable(Int64) │
│ dislike_count │ Nullable(Int64) │
│ is_crawlable │ Nullable(Bool) │
│ is_live_content │ Nullable(Bool) │
│ has_subtitles │ Nullable(Bool) │
│ is_ads_enabled │ Nullable(Bool) │
│ is_comments_enabled │ Nullable(Bool) │
│ description │ Nullable(String) │
│ rich_metadata │ Array(Map(String, Nullable(String)))
│ super_titles │ Array(Map(String, Nullable(String)))
│ uploader_badges │ Nullable(String) │
│ video_badges │ Nullable(String) │
└─────────────────────┴──────────────────────────────────────┘
┌─name────────────────┬─type───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─
│ id │ Nullable(String) │ │ │ │ │
│ fetch_date │ Nullable(String) │ │ │ │ │
│ upload_date │ Nullable(String) │ │ │ │ │
│ title │ Nullable(String) │ │ │ │ │
│ uploader_id │ Nullable(String) │ │ │ │ │
│ uploader │ Nullable(String) │ │ │ │ │
│ uploader_sub_count │ Nullable(Int64) │ │ │ │ │
│ is_age_limit │ Nullable(Bool) │ │ │ │ │
│ view_count │ Nullable(Int64) │ │ │ │ │
│ like_count │ Nullable(Int64) │ │ │ │ │
│ dislike_count │ Nullable(Int64) │ │ │ │ │
│ is_crawlable │ Nullable(Bool) │ │ │ │ │
│ is_live_content │ Nullable(Bool) │ │ │ │ │
│ has_subtitles │ Nullable(Bool) │ │ │ │ │
│ is_ads_enabled │ Nullable(Bool) │ │ │ │ │
│ is_comments_enabled │ Nullable(Bool) │ │ │ │ │
│ description │ Nullable(String) │ │ │ │ │
│ rich_metadata │ Array(Tuple(call Nullable(String), content Nullable(String), subtitle Nullable(String), title Nullable(String), url Nullable(String))) │ │ │ │ │
│ super_titles │ Array(Tuple(text Nullable(String), url Nullable(String))) │ │ │ │ │
│ uploader_badges │ Nullable(String) │ │ │ │ │
│ video_badges │ Nullable(String) │ │ │ │ │
└─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────
```
2. Based on the inferred schema, we cleaned up the data types and added a primary key. Define the following table:
@ -82,13 +81,13 @@ CREATE TABLE youtube
`is_ads_enabled` Bool,
`is_comments_enabled` Bool,
`description` String,
`rich_metadata` Array(Map(String, String)),
`super_titles` Array(Map(String, String)),
`rich_metadata` Array(Tuple(call String, content String, subtitle String, title String, url String)),
`super_titles` Array(Tuple(text String, url String)),
`uploader_badges` String,
`video_badges` String
)
ENGINE = MergeTree
ORDER BY (uploader, upload_date);
ORDER BY (uploader, upload_date)
```
3. The following command streams the records from the S3 files into the `youtube` table.

View File

@ -1262,6 +1262,7 @@ SELECT * FROM json_each_row_nested
- [input_format_import_nested_json](/docs/en/operations/settings/settings-formats.md/#input_format_import_nested_json) - map nested JSON data to nested tables (it works for JSONEachRow format). Default value - `false`.
- [input_format_json_read_bools_as_numbers](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_numbers) - allow to parse bools as numbers in JSON input formats. Default value - `true`.
- [input_format_json_read_bools_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_bools_as_strings) - allow to parse bools as strings in JSON input formats. Default value - `true`.
- [input_format_json_read_numbers_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_numbers_as_strings) - allow to parse numbers as strings in JSON input formats. Default value - `true`.
- [input_format_json_read_arrays_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_arrays_as_strings) - allow to parse JSON arrays as strings in JSON input formats. Default value - `true`.
- [input_format_json_read_objects_as_strings](/docs/en/operations/settings/settings-formats.md/#input_format_json_read_objects_as_strings) - allow to parse JSON objects as strings in JSON input formats. Default value - `true`.
@ -2355,6 +2356,8 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam
### Arrow format settings {#parquet-format-settings}
- [output_format_arrow_low_cardinality_as_dictionary](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_low_cardinality_as_dictionary) - enable output ClickHouse LowCardinality type as Dictionary Arrow type. Default value - `false`.
- [output_format_arrow_use_64_bit_indexes_for_dictionary](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_use_64_bit_indexes_for_dictionary) - use 64-bit integer type for Dictionary indexes. Default value - `false`.
- [output_format_arrow_use_signed_indexes_for_dictionary](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_use_signed_indexes_for_dictionary) - use signed integer type for Dictionary indexes. Default value - `true`.
- [output_format_arrow_string_as_string](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`.
- [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`.
- [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`.

View File

@ -614,6 +614,26 @@ DESC format(JSONEachRow, $$
└───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
##### input_format_json_read_bools_as_strings
Enabling this setting allows reading Bool values as strings.
This setting is enabled by default.
**Example:**
```sql
SET input_format_json_read_bools_as_strings = 1;
DESC format(JSONEachRow, $$
{"value" : true}
{"value" : "Hello, World"}
$$)
```
```response
┌─name──┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
│ value │ Nullable(String) │ │ │ │ │ │
└───────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
##### input_format_json_read_arrays_as_strings
Enabling this setting allows reading JSON array values as strings.

View File

@ -0,0 +1,207 @@
---
slug: /en/operations/allocation-profiling
sidebar_label: "Allocation profiling"
title: "Allocation profiling"
---
import Tabs from '@theme/Tabs';
import TabItem from '@theme/TabItem';
# Allocation profiling
ClickHouse uses [jemalloc](https://github.com/jemalloc/jemalloc) as its global allocator that comes with some tools for allocation sampling and profiling.
To make allocation profiling more convenient, `SYSTEM` commands are provided along 4LW commands in Keeper.
## Sampling allocations and flushing heap profiles
If we want to sample and profile allocations in `jemalloc`, we need to start ClickHouse/Keeper with profiling enabled using environment variable `MALLOC_CONF`.
```sh
MALLOC_CONF=background_thread:true,prof:true
```
`jemalloc` will sample allocation and store the information internally.
We can tell `jemalloc` to flush current profile by running:
<Tabs groupId="binary">
<TabItem value="clickhouse" label="ClickHouse">
SYSTEM JEMALLOC FLUSH PROFILE
</TabItem>
<TabItem value="keeper" label="Keeper">
echo jmfp | nc localhost 9181
</TabItem>
</Tabs>
By default, heap profile file will be generated in `/tmp/jemalloc_clickhouse._pid_._seqnum_.heap` where `_pid_` is the PID of ClickHouse and `_seqnum_` is the global sequence number for the current heap profile.
For Keeper, the default file is `/tmp/jemalloc_keeper._pid_._seqnum_.heap` following the same rules.
A different location can be defined by appending the `MALLOC_CONF` environment variable with `prof_prefix` option.
For example, if we want to generate profiles in `/data` folder where the prefix for filename will be `my_current_profile` we can run ClickHouse/Keeper with following environment variable:
```sh
MALLOC_CONF=background_thread:true,prof:true,prof_prefix:/data/my_current_profile
```
Generated file will append to prefix PID and sequence number.
## Analyzing heap profiles
After we generated heap profiles, we need to analyze them.
For that, we need to use `jemalloc`'s tool called [jeprof](https://github.com/jemalloc/jemalloc/blob/dev/bin/jeprof.in) which can be installed in multiple ways:
- installing `jemalloc` using system's package manager
- cloning [jemalloc repo](https://github.com/jemalloc/jemalloc) and running autogen.sh from the root folder that will provide you with `jeprof` script inside the `bin` folder
:::note
`jeprof` uses `addr2line` to generate stacktraces which can be really slow.
If thats the case, we recommend installing an [alternative implementation](https://github.com/gimli-rs/addr2line) of the tool.
```
git clone https://github.com/gimli-rs/addr2line
cd addr2line
cargo b --examples -r
cp ./target/release/examples/addr2line path/to/current/addr2line
```
:::
There are many different formats to generate from the heap profile using `jeprof`.
We recommend to run `jeprof --help` to check usage and many different options the tool provides.
In general, `jeprof` command will look like this:
```sh
jeprof path/to/binary path/to/heap/profile --output_format [ > output_file]
```
If we want to compare which allocations happened between 2 profiles we can set the base argument:
```sh
jeprof path/to/binary --base path/to/first/heap/profile path/to/second/heap/profile --output_format [ > output_file]
```
For example:
- if we want to generate a text file with each procedure written per line:
```sh
jeprof path/to/binary path/to/heap/profile --text > result.txt
```
- if we want to generate a PDF file with call-graph:
```sh
jeprof path/to/binary path/to/heap/profile --pdf > result.pdf
```
### Generating flame graph
`jeprof` allows us to generate collapsed stacks for building flame graphs.
We need to use `--collapsed` argument:
```sh
jeprof path/to/binary path/to/heap/profile --collapsed > result.collapsed
```
After that, we can use many different tools to visualize collapsed stacks.
Most popular would be [FlameGraph](https://github.com/brendangregg/FlameGraph) which contains a script called `flamegraph.pl`:
```sh
cat result.collapsed | /path/to/FlameGraph/flamegraph.pl --color=mem --title="Allocation Flame Graph" --width 2400 > result.svg
```
Another interesting tool is [speedscope](https://www.speedscope.app/) that allows you to analyze collected stacks in a more interactive way.
## Controlling allocation profiler during runtime
If ClickHouse/Keeper were started with enabled profiler, they support additional commands for disabling/enabling allocation profiling during runtime.
Using those commands, it's easier to profile only specific intervals.
Disable profiler:
<Tabs groupId="binary">
<TabItem value="clickhouse" label="ClickHouse">
SYSTEM JEMALLOC DISABLE PROFILE
</TabItem>
<TabItem value="keeper" label="Keeper">
echo jmdp | nc localhost 9181
</TabItem>
</Tabs>
Enable profiler:
<Tabs groupId="binary">
<TabItem value="clickhouse" label="ClickHouse">
SYSTEM JEMALLOC ENABLE PROFILE
</TabItem>
<TabItem value="keeper" label="Keeper">
echo jmep | nc localhost 9181
</TabItem>
</Tabs>
It's also possible to control the initial state of the profiler by setting `prof_active` option which is enabled by default.
For example, if we don't want to sample allocations during startup but only after we enable the profiler, we can start ClickHouse/Keeper with following environment variable:
```sh
MALLOC_CONF=background_thread:true,prof:true,prof_active:false
```
and enable profiler at a later point.
## Additional options for profiler
`jemalloc` has many different options available related to profiler which can be controlled by modifying `MALLOC_CONF` environment variable.
For example, interval between allocation samples can be controlled with `lg_prof_sample`.
If you want to dump heap profile every N bytes you can enable it using `lg_prof_interval`.
We recommend to check `jemalloc`s [reference page](https://jemalloc.net/jemalloc.3.html) for such options.
## Other resources
ClickHouse/Keeper expose `jemalloc` related metrics in many different ways.
:::warning Warning
It's important to be aware that none of these metrics are synchronized with each other and values may drift.
:::
### System table `asynchronous_metrics`
```sql
SELECT *
FROM system.asynchronous_metrics
WHERE metric ILIKE '%jemalloc%'
FORMAT Vertical
```
[Reference](/en/operations/system-tables/asynchronous_metrics)
### System table `jemalloc_bins`
Contains information about memory allocations done via jemalloc allocator in different size classes (bins) aggregated from all arenas.
[Reference](/en/operations/system-tables/jemalloc_bins)
### Prometheus
All `jemalloc` related metrics from `asynchronous_metrics` are also exposed using Prometheus endpoint in both ClickHouse and Keeper.
[Reference](/en/operations/server-configuration-parameters/settings#prometheus)
### `jmst` 4LW command in Keeper
Keeper supports `jmst` 4LW command which returns [basic allocator statistics](https://github.com/jemalloc/jemalloc/wiki/Use-Case%3A-Basic-Allocator-Statistics).
Example:
```sh
echo jmst | nc localhost 9181
```

View File

@ -65,6 +65,20 @@ With Cluster Discovery, rather than defining each node explicitly, you simply sp
<cluster_name>
<discovery>
<path>/clickhouse/discovery/cluster_name</path>
<!-- # Optional configuration parameters: -->
<!-- ## Authentication credentials to access all other nodes in cluster: -->
<!-- <user>user1</user> -->
<!-- <password>pass123</password> -->
<!-- ### Alternatively to password, interserver secret may be used: -->
<!-- <secret>secret123</secret> -->
<!-- ## Shard for current node (see below): -->
<!-- <shard>1</shard> -->
<!-- ## Observer mode (see below): -->
<!-- <observer/> -->
</discovery>
</cluster_name>
</remote_servers>

View File

@ -296,7 +296,6 @@ host = '127.0.0.1',
port = 5432,
database = 'test',
schema = 'test_schema',
connection_pool_size = 8
```
Example of configuration:
@ -310,7 +309,6 @@ Example of configuration:
<port>5432</port>
<database>test</database>
<schema>test_schema</schema>
<connection_pool_size>8</connection_pool_size>
</mypg>
</named_collections>
</clickhouse>
@ -445,4 +443,3 @@ SELECT dictGet('dict', 'b', 1);
│ a │
└─────────────────────────┘
```

View File

@ -29,10 +29,6 @@ Transactionally inconsistent caching is traditionally provided by client tools o
the same caching logic and configuration is often duplicated. With ClickHouse's query cache, the caching logic moves to the server side.
This reduces maintenance effort and avoids redundancy.
:::note
Security consideration: The cached query result is tied to the user executing it. Authorization checks are performed when the query is executed. This means that if there are any alterations to the user's role or permissions between the time the query is cached and when the cache is accessed, the result will not reflect these changes. We recommend using different users to distinguish between different levels of access, instead of actively toggling roles for a single user between queries, as this practice may lead to unexpected query results.
:::
## Configuration Settings and Usage
Setting [use_query_cache](settings/settings.md#use-query-cache) can be used to control whether a specific query or all queries of the

View File

@ -2866,3 +2866,10 @@ This also allows a mix of resolver types can be used.
### disable_tunneling_for_https_requests_over_http_proxy {#disable_tunneling_for_https_requests_over_http_proxy}
By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests over `HTTP` proxy. This setting can be used to disable it.
## max_materialized_views_count_for_table {#max_materialized_views_count_for_table}
A limit on the number of materialized views attached to a table.
Note that only directly dependent views are considered here, and the creation of one view on top of another view is not considered.
Default value: `0`.

View File

@ -1,5 +1,5 @@
---
sidebar_label: Settings Overview
title: "Settings Overview"
sidebar_position: 1
slug: /en/operations/settings/
pagination_next: en/operations/settings/settings
@ -16,11 +16,34 @@ There are two main groups of ClickHouse settings:
- Global server settings
- Query-level settings
The main distinction between global server settings and query-level settings is that
global server settings must be set in configuration files while query-level settings
can be set in configuration files or with SQL queries.
The main distinction between global server settings and query-level settings is that global server settings must be set in configuration files, while query-level settings can be set in configuration files or with SQL queries.
Read about [global server settings](/docs/en/operations/server-configuration-parameters/settings.md) to learn more about configuring your ClickHouse server at the global server level.
Read about [query-level settings](/docs/en/operations/settings/settings-query-level.md) to learn more about configuring your ClickHouse server at the query-level.
Read about [query-level settings](/docs/en/operations/settings/settings-query-level.md) to learn more about configuring your ClickHouse server at the query level.
## See non-default settings
To view which settings have been changed from their default value:
```sql
SELECT name, value FROM system.settings WHERE changed
```
If you haven't changed any settings from their default value, then ClickHouse will return nothing.
To check the value of a particular setting, specify the `name` of the setting in your query:
```sql
SELECT name, value FROM system.settings WHERE name = 'max_threads'
```
This command should return something like:
```response
┌─name────────┬─value─────┐
│ max_threads │ 'auto(8)' │
└─────────────┴───────────┘
1 row in set. Elapsed: 0.002 sec.
```

View File

@ -852,6 +852,16 @@ If the file name for column is too long (more than `max_file_name_length` bytes)
The maximal length of the file name to keep it as is without hashing. Takes effect only if setting `replace_long_file_name_to_hash` is enabled. The value of this setting does not include the length of file extension. So, it is recommended to set it below the maximum filename length (usually 255 bytes) with some gap to avoid filesystem errors. Default value: 127.
## clean_deleted_rows
Enable/disable automatic deletion of rows flagged as `is_deleted` when perform `OPTIMIZE ... FINAL` on a table using the ReplacingMergeTree engine. When disabled, the `CLEANUP` keyword has to be added to the `OPTIMIZE ... FINAL` to have the same behaviour.
Possible values:
- `Always` or `Never`.
Default value: `Never`
## allow_experimental_block_number_column
Persists virtual column `_block_number` on merges.

View File

@ -0,0 +1,176 @@
# The MySQL Binlog Client
The MySQL Binlog Client provides a mechanism in ClickHouse to share the binlog from a MySQL instance among multiple [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) databases. This avoids consuming unnecessary bandwidth and CPU when replicating more than one schema/database.
The implementation is resilient against crashes and disk issues. The executed GTID sets of the binlog itself and the consuming databases have persisted only after the data they describe has been safely persisted as well. The implementation also tolerates re-doing aborted operations (at-least-once delivery).
# Settings
## use_binlog_client
Forces to reuse existing MySQL binlog connection or creates new one if does not exist. The connection is defined by `user:pass@host:port`.
Default value: 0
**Example**
```sql
-- create MaterializedMySQL databases that read the events from the binlog client
CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1
CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1
CREATE DATABASE db3 ENGINE = MaterializedMySQL('host:port', 'db3', 'user2', 'password2') SETTINGS use_binlog_client=1
```
Databases `db1` and `db2` will use the same binlog connection, since they use the same `user:pass@host:port`. Database `db3` will use separate binlog connection.
## max_bytes_in_binlog_queue
Defines the limit of bytes in the events binlog queue. If bytes in the queue increases this limit, it will stop reading new events from MySQL until the space for new events will be freed. This introduces the memory limits. Very high value could consume all available memory. Very low value could make the databases to wait for new events.
Default value: 67108864
**Example**
```sql
CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=33554432
CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1
```
If database `db1` is unable to consume binlog events fast enough and the size of the events queue exceeds `33554432` bytes, reading of new events from MySQL is postponed until `db1`
consumes the events and releases some space.
NOTE: This will impact to `db2`, and it will be waiting for new events too, since they share the same connection.
## max_milliseconds_to_wait_in_binlog_queue
Defines the max milliseconds to wait when `max_bytes_in_binlog_queue` exceeded. After that it will detach the database from current binlog connection and will retry establish new one to prevent other databases to wait for this database.
Default value: 10000
**Example**
```sql
CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=33554432, max_milliseconds_to_wait_in_binlog_queue=1000
CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1
```
If the event queue of database `db1` is full, the binlog connection will be waiting in `1000`ms and if the database is not able to consume the events, it will be detached from the connection to create another one.
NOTE: If the database `db1` has been detached from the shared connection and created new one, after the binlog connections for `db1` and `db2` have the same positions they will be merged to one. And `db1` and `db2` will use the same connection again.
## max_bytes_in_binlog_dispatcher_buffer
Defines the max bytes in the binlog dispatcher's buffer before it is flushed to attached binlog. The events from MySQL binlog connection are buffered before sending to attached databases. It increases the events throughput from the binlog to databases.
Default value: 1048576
## max_flush_milliseconds_in_binlog_dispatcher
Defines the max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlog. If there are no events received from MySQL binlog connection for a while, after some time buffered events should be sent to the attached databases.
Default value: 1000
# Design
## The Binlog Events Dispatcher
Currently each MaterializedMySQL database opens its own connection to MySQL to subscribe to binlog events. There is a need to have only one connection and _dispatch_ the binlog events to all databases that replicate from the same MySQL instance.
## Each MaterializedMySQL Database Has Its Own Event Queue
To prevent slowing down other instances there should be an _event queue_ per MaterializedMySQL database to handle the events independently of the speed of other instances. The dispatcher reads an event from the binlog, and sends it to every MaterializedMySQL database that needs it. Each database handles its events in separate threads.
## Catching up
If several databases have the same binlog position, they can use the same dispatcher. If a newly created database (or one that has been detached for some time) requests events that have been already processed, we need to create another communication _channel_ to the binlog. We do this by creating another temporary dispatcher for such databases. When the new dispatcher _catches up with_ the old one, the new/temporary dispatcher is not needed anymore and all databases getting events from this dispatcher can be moved to the old one.
## Memory Limit
There is a _memory limit_ to control event queue memory consumption per MySQL Client. If a database is not able to handle events fast enough, and the event queue is getting full, we have the following options:
1. The dispatcher is blocked until the slowest database frees up space for new events. All other databases are waiting for the slowest one. (Preferred)
2. The dispatcher is _never_ blocked, but suspends incremental sync for the slow database and continues dispatching events to remained databases.
## Performance
A lot of CPU can be saved by not processing every event in every database. The binlog contains events for all databases, it is wasteful to distribute row events to a database that it will not process it, especially if there are a lot of databases. This requires some sort of per-database binlog filtering and buffering.
Currently all events are sent to all MaterializedMySQL databases but parsing the event which consumes CPU is up to the database.
# Detailed Design
1. If a client (e.g. database) wants to read a stream of the events from MySQL binlog, it creates a connection to remote binlog by host/user/password and _executed GTID set_ params.
2. If another client wants to read the events from the binlog but for different _executed GTID set_, it is **not** possible to reuse existing connection to MySQL, then need to create another connection to the same remote binlog. (_This is how it is implemented today_).
3. When these 2 connections get the same binlog positions, they read the same events. It is logical to drop duplicate connection and move all its users out. And now one connection dispatches binlog events to several clients. Obviously only connections to the same binlog should be merged.
## Classes
1. One connection can send (or dispatch) events to several clients and might be called `BinlogEventsDispatcher`.
2. Several dispatchers grouped by _user:password@host:port_ in `BinlogClient`. Since they point to the same binlog.
3. The clients should communicate only with public API from `BinlogClient`. The result of using `BinlogClient` is an object that implements `IBinlog` to read events from. This implementation of `IBinlog` must be compatible with old implementation `MySQLFlavor` -> when replacing old implementation by new one, the behavior must not be changed.
## SQL
```sql
-- create MaterializedMySQL databases that read the events from the binlog client
CREATE DATABASE db1_client1 ENGINE = MaterializedMySQL('host:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=1024;
CREATE DATABASE db2_client1 ENGINE = MaterializedMySQL('host:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1;
CREATE DATABASE db3_client1 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1;
CREATE DATABASE db4_client2 ENGINE = MaterializedMySQL('host2:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1;
CREATE DATABASE db5_client3 ENGINE = MaterializedMySQL('host:port', 'db', 'user1', 'password') SETTINGS use_binlog_client=1;
CREATE DATABASE db6_old ENGINE = MaterializedMySQL('host:port', 'db', 'user1', 'password') SETTINGS use_binlog_client=0;
```
Databases `db1_client1`, `db2_client1` and `db3_client1` share one instance of `BinlogClient` since they have the same params. `BinlogClient` will create 3 connections to MySQL server thus 3 instances of `BinlogEventsDispatcher`, but if these connections would have the same binlog position, they should be merged to one connection. Means all clients will be moved to one dispatcher and others will be closed. Databases `db4_client2` and `db5_client3` would use 2 different independent `BinlogClient` instances. Database `db6_old` will use old implementation. NOTE: By default `use_binlog_client` is disabled. Setting `max_bytes_in_binlog_queue` defines the max allowed bytes in the binlog queue. By default, it is `1073741824` bytes. If number of bytes exceeds this limit, the dispatching will be stopped until the space will be freed for new events.
## Binlog Table Structure
To see the status of the all `BinlogClient` instances there is `system.mysql_binlogs` system table. It shows the list of all created and _alive_ `IBinlog` instances with information about its `BinlogEventsDispatcher` and `BinlogClient`.
Example:
```
SELECT * FROM system.mysql_binlogs FORMAT Vertical
Row 1:
──────
binlog_client_name: root@127.0.0.1:3306
name: test_Clickhouse1
mysql_binlog_name: binlog.001154
mysql_binlog_pos: 7142294
mysql_binlog_timestamp: 1660082447
mysql_binlog_executed_gtid_set: a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30523304
dispatcher_name: Applier
dispatcher_mysql_binlog_name: binlog.001154
dispatcher_mysql_binlog_pos: 7142294
dispatcher_mysql_binlog_timestamp: 1660082447
dispatcher_mysql_binlog_executed_gtid_set: a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30523304
size: 0
bytes: 0
max_bytes: 0
```
### Tests
Unit tests:
```
$ ./unit_tests_dbms --gtest_filter=MySQLBinlog.*
```
Integration tests:
```
$ pytest -s -vv test_materialized_mysql_database/test.py::test_binlog_client
```
Dumps events from the file
```
$ ./utils/check-mysql-binlog/check-mysql-binlog --binlog binlog.001392
```
Dumps events from the server
```
$ ./utils/check-mysql-binlog/check-mysql-binlog --host 127.0.0.1 --port 3306 --user root --password pass --gtid a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30462856
```

View File

@ -172,7 +172,7 @@ If you set `timeout_before_checking_execution_speed `to 0, ClickHouse will use c
## timeout_overflow_mode {#timeout-overflow-mode}
What to do if the query is run longer than `max_execution_time`: `throw` or `break`. By default, `throw`.
What to do if the query is run longer than `max_execution_time` or the estimated running time is longer than `max_estimated_execution_time`: `throw` or `break`. By default, `throw`.
# max_execution_time_leaf
@ -214,6 +214,10 @@ A maximum number of execution bytes per second. Checked on every data block when
Checks that execution speed is not too slow (no less than min_execution_speed), after the specified time in seconds has expired.
## max_estimated_execution_time {#max_estimated_execution_time}
Maximum query estimate execution time in seconds. Checked on every data block when timeout_before_checking_execution_speed expires.
## max_columns_to_read {#max-columns-to-read}
A maximum number of columns that can be read from a table in a single query. If a query requires reading a greater number of columns, it throws an exception.

View File

@ -377,6 +377,12 @@ Allow parsing bools as numbers in JSON input formats.
Enabled by default.
## input_format_json_read_bools_as_strings {#input_format_json_read_bools_as_strings}
Allow parsing bools as strings in JSON input formats.
Enabled by default.
## input_format_json_read_numbers_as_strings {#input_format_json_read_numbers_as_strings}
Allow parsing numbers as strings in JSON input formats.
@ -1263,6 +1269,28 @@ Possible values:
Default value: `0`.
### output_format_arrow_use_signed_indexes_for_dictionary {#output_format_arrow_use_signed_indexes_for_dictionary}
Use signed integer types instead of unsigned in `DICTIONARY` type of the [Arrow](../../interfaces/formats.md/#data-format-arrow) format during [LowCardinality](../../sql-reference/data-types/lowcardinality.md) output when `output_format_arrow_low_cardinality_as_dictionary` is enabled.
Possible values:
- 0 — Unsigned integer types are used for indexes in `DICTIONARY` type.
- 1 — Signed integer types are used for indexes in `DICTIONARY` type.
Default value: `1`.
### output_format_arrow_use_64_bit_indexes_for_dictionary {#output_format_arrow_use_64_bit_indexes_for_dictionary}
Use 64-bit integer type in `DICTIONARY` type of the [Arrow](../../interfaces/formats.md/#data-format-arrow) format during [LowCardinality](../../sql-reference/data-types/lowcardinality.md) output when `output_format_arrow_low_cardinality_as_dictionary` is enabled.
Possible values:
- 0 — Type for indexes in `DICTIONARY` type is determined automatically.
- 1 — 64-bit integer type is used for indexes in `DICTIONARY` type.
Default value: `0`.
### output_format_arrow_string_as_string {#output_format_arrow_string_as_string}
Use Arrow String type instead of Binary for String columns.
@ -1569,7 +1597,13 @@ Result:
Use ANSI escape sequences to paint colors in Pretty formats.
Enabled by default.
possible values:
- `0` — Disabled. Pretty formats do not use ANSI escape sequences.
- `1` — Enabled. Pretty formats will use ANSI escape sequences except for `NoEscapes` formats.
- `auto` - Enabled if `stdout` is a terminal except for `NoEscapes` formats.
Default value is `auto`.
### output_format_pretty_grid_charset {#output_format_pretty_grid_charset}

View File

@ -4,7 +4,7 @@ sidebar_position: 63
sidebar_label: User Settings
---
# User Settings
# Users and Roles Settings
The `users` section of the `user.xml` configuration file contains user settings.
@ -187,3 +187,34 @@ The following configuration forces that user `user1` can only see the rows of `t
```
The `filter` can be any expression resulting in a [UInt8](../../sql-reference/data-types/int-uint.md)-type value. It usually contains comparisons and logical operators. Rows from `database_name.table1` where filter results to 0 are not returned for this user. The filtering is incompatible with `PREWHERE` operations and disables `WHERE→PREWHERE` optimization.
## Roles
You can create any predefined roles using the `roles` section of the `user.xml` configuration file.
Structure of the `roles` section:
```xml
<roles>
<test_role>
<grants>
<query>GRANT SHOW ON *.*</query>
<query>REVOKE SHOW ON system.*</query>
<query>GRANT CREATE ON *.* WITH GRANT OPTION</query>
</grants>
</test_role>
</roles>
```
These roles can also be granted to users from the `users` section:
```xml
<users>
<user_name>
...
<grants>
<query>GRANT test_role</query>
</grants>
</user_name>
<users>
```

View File

@ -2040,6 +2040,32 @@ SELECT * FROM test_table
└───┘
```
## update_insert_deduplication_token_in_dependent_materialized_views {#update-insert-deduplication-token-in-dependent-materialized-views}
Allows to update `insert_deduplication_token` with table identifier during insert in dependent materialized views, if setting `deduplicate_blocks_in_dependent_materialized_views` is enabled and `insert_deduplication_token` is set.
Possible values:
0 — Disabled.
1 — Enabled.
Default value: 0.
Usage:
If setting `deduplicate_blocks_in_dependent_materialized_views` is enabled, `insert_deduplication_token` is passed to dependent materialized views. But in complex INSERT flows it is possible that we want to avoid deduplication for dependent materialized views.
Example:
```
landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1
| |
└--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘
```
In this example we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will be inserted into `ds_2_1`. Without `update_insert_deduplication_token_in_dependent_materialized_views` setting enabled, those two different blocks will be deduplicated, because different blocks from `mv_2_1` and `mv_2_2` will have the same `insert_deduplication_token`.
If setting `update_insert_deduplication_token_in_dependent_materialized_views` is enabled, during each insert into dependent materialized views `insert_deduplication_token` is updated with table identifier, so block from `mv_2_1` and block from `mv_2_2` will have different `insert_deduplication_token` and will not be deduplicated.
## insert_keeper_max_retries
The setting sets the maximum number of retries for ClickHouse Keeper (or ZooKeeper) requests during insert into replicated MergeTree. Only Keeper requests which failed due to network error, Keeper session timeout, or request timeout are considered for retries.
@ -3847,6 +3873,8 @@ Possible values:
- `none` — Is similar to throw, but distributed DDL query returns no result set.
- `null_status_on_timeout` — Returns `NULL` as execution status in some rows of result set instead of throwing `TIMEOUT_EXCEEDED` if query is not finished on the corresponding hosts.
- `never_throw` — Do not throw `TIMEOUT_EXCEEDED` and do not rethrow exceptions if query has failed on some hosts.
- `null_status_on_timeout_only_active` — similar to `null_status_on_timeout`, but doesn't wait for inactive replicas of the `Replicated` database
- `throw_only_active` — similar to `throw`, but doesn't wait for inactive replicas of the `Replicated` database
Default value: `throw`.
@ -4771,6 +4799,45 @@ Type: Int64
Default: 0
## enable_deflate_qpl_codec {#enable_deflate_qpl_codec}
If turned on, the DEFLATE_QPL codec may be used to compress columns.
Possible values:
- 0 - Disabled
- 1 - Enabled
Type: Bool
## enable_zstd_qat_codec {#enable_zstd_qat_codec}
If turned on, the ZSTD_QAT codec may be used to compress columns.
Possible values:
- 0 - Disabled
- 1 - Enabled
Type: Bool
## output_format_compression_level
Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when writing to table functions `file`, `url`, `hdfs`, `s3`, or `azureBlobStorage`.
Possible values: from `1` to `22`
Default: `3`
## output_format_compression_zstd_window_log
Can be used when the output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (power of `2`) and enables a long-range mode for zstd compression. This can help to achieve a better compression ratio.
Possible values: non-negative numbers. Note that if the value is too small or too big, `zstdlib` will throw an exception. Typical values are from `20` (window size = `1MB`) to `30` (window size = `1GB`).
Default: `0`
## rewrite_count_distinct_if_with_count_distinct_implementation
Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#count_distinct_implementation) setting.
@ -5135,6 +5202,95 @@ When set to `false` than all attempts are made with identical timeouts.
Default value: `true`.
## allow_experimental_variant_type {#allow_experimental_variant_type}
Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md).
Default value: `false`.
## use_variant_as_common_type {#use_variant_as_common_type}
Allows to use `Variant` type as a result type for [if](../../sql-reference/functions/conditional-functions.md/#if)/[multiIf](../../sql-reference/functions/conditional-functions.md/#multiif)/[array](../../sql-reference/functions/array-functions.md)/[map](../../sql-reference/functions/tuple-map-functions.md) functions when there is no common type for argument types.
Example:
```sql
SET use_variant_as_common_type = 1;
SELECT toTypeName(if(number % 2, number, range(number))) as variant_type FROM numbers(1);
SELECT if(number % 2, number, range(number)) as variant FROM numbers(5);
```
```text
┌─variant_type───────────────────┐
│ Variant(Array(UInt64), UInt64) │
└────────────────────────────────┘
┌─variant───┐
│ [] │
│ 1 │
│ [0,1] │
│ 3 │
│ [0,1,2,3] │
└───────────┘
```
```sql
SET use_variant_as_common_type = 1;
SELECT toTypeName(multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL)) AS variant_type FROM numbers(1);
SELECT multiIf((number % 4) = 0, 42, (number % 4) = 1, [1, 2, 3], (number % 4) = 2, 'Hello, World!', NULL) AS variant FROM numbers(4);
```
```text
─variant_type─────────────────────────┐
│ Variant(Array(UInt8), String, UInt8) │
└──────────────────────────────────────┘
┌─variant───────┐
│ 42 │
│ [1,2,3] │
│ Hello, World! │
│ ᴺᵁᴸᴸ │
└───────────────┘
```
```sql
SET use_variant_as_common_type = 1;
SELECT toTypeName(array(range(number), number, 'str_' || toString(number))) as array_of_variants_type from numbers(1);
SELECT array(range(number), number, 'str_' || toString(number)) as array_of_variants FROM numbers(3);
```
```text
┌─array_of_variants_type────────────────────────┐
│ Array(Variant(Array(UInt64), String, UInt64)) │
└───────────────────────────────────────────────┘
┌─array_of_variants─┐
│ [[],0,'str_0'] │
│ [[0],1,'str_1'] │
│ [[0,1],2,'str_2'] │
└───────────────────┘
```
```sql
SET use_variant_as_common_type = 1;
SELECT toTypeName(map('a', range(number), 'b', number, 'c', 'str_' || toString(number))) as map_of_variants_type from numbers(1);
SELECT map('a', range(number), 'b', number, 'c', 'str_' || toString(number)) as map_of_variants FROM numbers(3);
```
```text
┌─map_of_variants_type────────────────────────────────┐
│ Map(String, Variant(Array(UInt64), String, UInt64)) │
└─────────────────────────────────────────────────────┘
┌─map_of_variants───────────────┐
│ {'a':[],'b':0,'c':'str_0'} │
│ {'a':[0],'b':1,'c':'str_1'} │
│ {'a':[0,1],'b':2,'c':'str_2'} │
└───────────────────────────────┘
```
Default value: `false`.
## max_partition_size_to_drop
Restriction on dropping partitions in query time.
@ -5156,3 +5312,13 @@ The value 0 means that you can delete all tables without any restrictions.
:::note
This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop)
:::
## iceberg_engine_ignore_schema_evolution {#iceberg_engine_ignore_schema_evolution}
Allow to ignore schema evolution in Iceberg table engine and read all data using schema specified by the user on table creation or latest schema parsed from metadata on table creation.
:::note
Enabling this setting can lead to incorrect result as in case of evolved schema all data files will be read using the same schema.
:::
Default value: 'false'.

View File

@ -196,7 +196,7 @@ These settings should be defined in the disk configuration section.
- `max_elements` - a limit for a number of cache files. Default: `10000000`.
- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `1`.
- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `16`.
File Cache **query/profile settings**:

View File

@ -10,7 +10,7 @@ Columns:
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
- `name` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value.
**Example**

View File

@ -0,0 +1,26 @@
---
slug: /en/operations/system-tables/database_engines
---
# database_engines
Contains the list of database engines supported by the server.
This table contains the following columns (the column type is shown in brackets):
- `name` (String) — The name of database engine.
Example:
``` sql
SELECT *
FROM system.database_engines
WHERE name in ('Atomic', 'Lazy', 'Ordinary')
```
``` text
┌─name─────┐
│ Ordinary │
│ Atomic │
│ Lazy │
└──────────┘
```

View File

@ -0,0 +1,14 @@
---
slug: /en/operations/system-tables/dropped_tables_parts
---
# dropped_tables_parts {#system_tables-dropped_tables_parts}
Contains information about parts of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) dropped tables from [system.dropped_tables](./dropped_tables.md)
The schema of this table is the same as [system.parts](./parts.md)
**See Also**
- [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md)
- [system.parts](./parts.md)
- [system.dropped_tables](./dropped_tables.md)

View File

@ -9,11 +9,15 @@ Columns:
- `name` ([String](../../sql-reference/data-types/string.md)) — name of the error (`errorCodeToName`).
- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — code number of the error.
- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — the number of times this error has been happened.
- `last_error_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — time when the last error happened.
- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — the number of times this error happened.
- `last_error_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — the time when the last error happened.
- `last_error_message` ([String](../../sql-reference/data-types/string.md)) — message for the last error.
- `last_error_trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — A [stack trace](https://en.wikipedia.org/wiki/Stack_trace) which represents a list of physical addresses where the called methods are stored.
- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — remote exception (i.e. received during one of the distributed query).
- `last_error_trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — A [stack trace](https://en.wikipedia.org/wiki/Stack_trace) that represents a list of physical addresses where the called methods are stored.
- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — remote exception (i.e. received during one of the distributed queries).
:::note
Counters for some errors may increase during successful query execution. It's not recommended to use this table for server monitoring purposes unless you are sure that corresponding error can not be a false positive.
:::
**Example**

View File

@ -287,7 +287,7 @@ Number of threads in the HashedDictionary thread pool running a task.
### IOPrefetchThreads
Number of threads in the IO prefertch thread pool.
Number of threads in the IO prefetch thread pool.
### IOPrefetchThreadsActive

View File

@ -42,7 +42,7 @@ Columns:
- `'ExceptionWhileProcessing' = 4` — Exception during the query execution.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Query starting date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time with microseconds precision.
- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Query starting time with microseconds precision.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds.

View File

@ -49,7 +49,7 @@ Columns:
- `last_attempt_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last attempted.
- `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of postponed tasks.
- `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of times the action was postponed.
- `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — The reason why the task was postponed.

View File

@ -14,6 +14,11 @@ Columns:
- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml`
- `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description.
- `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type.
- `changeable_without_restart` ([Enum8](../../sql-reference/data-types/enum.md)) — Whether the setting can be changed at server runtime. Values:
- `'No' `
- `'IncreaseOnly'`
- `'DecreaseOnly'`
- `'Yes'`
- `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete.
**Example**
@ -27,22 +32,21 @@ WHERE name LIKE '%thread_pool%'
```
``` text
┌─name────────────────────────────────────────_─value─_─default─_─changed─_─description──────────────────────────────────────────────────────────────────────────────────────────────────────
───────────────────────────────────_─type───_─is_obsolete─┐
│ max_thread_pool_size │ 10000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │
│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │
│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │
│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │
│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │
│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │
│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │
│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │
│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │
│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │
│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │
│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │
└─────────────────────────────────────────────┴───────┴─────────┴─────────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────
───────────────────────────────────┴────────┴─────────────┘
┌─name────────────────────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┬─changeable_without_restart─┬─is_obsolete─┐
│ max_thread_pool_size │ 10000 │ 10000 │ 0 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ No │ 0 │
│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ No │ 0 │
│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ No │ 0 │
│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ No │ 0 │
│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ No │ 0 │
│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ No │ 0 │
│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ No │ 0 │
│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ No │ 0 │
│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ No │ 0 │
│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ No │ 0 │
│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ No │ 0 │
│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ No │ 0 │
└─────────────────────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┴────────────────────────────┴─────────────┘
```
Using of `WHERE changed` can be useful, for example, when you want to check

View File

@ -10,7 +10,7 @@ Columns:
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
- `event_date` (Date) — Date of the entry.
- `event_time` (DateTime) — Time of the entry.
- `event_time_microseconds` (DateTime) — Time of the entry with microseconds precision.
- `event_time_microseconds` (DateTime64) — Time of the entry with microseconds precision.
- `microseconds` (UInt32) — Microseconds of the entry.
- `thread_name` (String) — Name of the thread from which the logging was done.
- `thread_id` (UInt64) — OS thread ID.

View File

@ -0,0 +1,43 @@
---
slug: /en/operations/system-tables/view_refreshes
---
# view_refreshes
Information about [Refreshable Materialized Views](../../sql-reference/statements/create/view.md#refreshable-materialized-view). Contains all refreshable materialized views, regardless of whether there's a refresh in progress or not.
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in.
- `view` ([String](../../sql-reference/data-types/string.md)) — Table name.
- `status` ([String](../../sql-reference/data-types/string.md)) — Current state of the refresh.
- `last_refresh_result` ([String](../../sql-reference/data-types/string.md)) — Outcome of the latest refresh attempt.
- `last_refresh_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the last refresh attempt. `NULL` if no refresh attempts happened since server startup or table creation.
- `last_success_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the last successful refresh. `NULL` if no successful refreshes happened since server startup or table creation.
- `duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — How long the last refresh attempt took.
- `next_refresh_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time at which the next refresh is scheduled to start.
- `remaining_dependencies` ([Array(String)](../../sql-reference/data-types/array.md)) — If the view has [refresh dependencies](../../sql-reference/statements/create/view.md#refresh-dependencies), this array contains the subset of those dependencies that are not satisfied for the current refresh yet. If `status = 'WaitingForDependencies'`, a refresh is ready to start as soon as these dependencies are fulfilled.
- `exception` ([String](../../sql-reference/data-types/string.md)) — if `last_refresh_result = 'Exception'`, i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace.
- `refresh_count` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of successful refreshes since last server restart or table creation.
- `progress` ([Float64](../../sql-reference/data-types/float.md)) — Progress of the current refresh, between 0 and 1.
- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of rows read by the current refresh so far.
- `total_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Estimated total number of rows that need to be read by the current refresh.
(There are additional columns related to current refresh progress, but they are currently unreliable.)
**Example**
```sql
SELECT
database,
view,
status,
last_refresh_result,
last_refresh_time,
next_refresh_time
FROM system.view_refreshes
┌─database─┬─view───────────────────────┬─status────┬─last_refresh_result─┬───last_refresh_time─┬───next_refresh_time─┐
│ default │ hello_documentation_reader │ Scheduled │ Finished │ 2023-12-01 01:24:00 │ 2023-12-01 01:25:00 │
└──────────┴────────────────────────────┴───────────┴─────────────────────┴─────────────────────┴─────────────────────┘
```

View File

@ -11,6 +11,8 @@ Keys:
- `--query` — Format queries of any length and complexity.
- `--hilite` — Add syntax highlight with ANSI terminal escape sequences.
- `--oneline` — Format in single line.
- `--max_line_length` — Format in single line queries with length less than specified.
- `--comments` — Keep comments in the output.
- `--quiet` or `-q` — Just check syntax, no output on success.
- `--multiquery` or `-n` — Allow multiple queries in the same file.
- `--obfuscate` — Obfuscate instead of formatting.
@ -27,7 +29,7 @@ $ clickhouse-format --query "select number from numbers(10) where number%2 order
Result:
```sql
```bash
SELECT number
FROM numbers(10)
WHERE number % 2
@ -49,22 +51,20 @@ SELECT sum(number) FROM numbers(5)
3. Multiqueries:
```bash
$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);"
$ clickhouse-format -n <<< "SELECT min(number) FROM numbers(5); SELECT max(number) FROM numbers(5);"
```
Result:
```sql
SELECT *
FROM
(
SELECT 1 AS x
UNION ALL
SELECT 1
UNION DISTINCT
SELECT 3
)
```
SELECT min(number)
FROM numbers(5)
;
SELECT max(number)
FROM numbers(5)
;
```
4. Obfuscating:
@ -75,7 +75,7 @@ $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWE
Result:
```sql
```
SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END;
```
@ -87,7 +87,7 @@ $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWE
Result:
```sql
```
SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END;
```
@ -99,7 +99,7 @@ $ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELE
Result:
```sql
```
SELECT * \
FROM \
( \

View File

@ -24,7 +24,7 @@ A client application to interact with clickhouse-keeper by its native protocol.
## Example {#clickhouse-keeper-client-example}
```bash
./clickhouse-keeper-client -h localhost:9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30
./clickhouse-keeper-client -h localhost -p 9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30
Connected to ZooKeeper at [::1]:9181 with session_id 137
/ :) ls
keeper foo bar

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