Merge branch 'master' into alter_on_top_of_mutations_merge_tree

This commit is contained in:
alesapin 2020-03-19 16:05:17 +03:00
commit ce8eb9293a
140 changed files with 914 additions and 827 deletions

View File

@ -186,11 +186,13 @@ if (COMPILER_GCC OR COMPILER_CLANG)
endif ()
option(WITH_COVERAGE "Build with coverage." 0)
if(WITH_COVERAGE AND COMPILER_CLANG)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping")
# If we want to disable coverage for specific translation units
set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping")
endif()
if(WITH_COVERAGE AND COMPILER_GCC)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-arcs -ftest-coverage")
set(COVERAGE_OPTION "-lgcov")

View File

@ -8,6 +8,7 @@ ClickHouse is an open-source column-oriented database management system that all
* [Tutorial](https://clickhouse.tech/docs/en/getting_started/tutorial/) shows how to set up and query small ClickHouse cluster.
* [Documentation](https://clickhouse.tech/docs/en/) provides more in-depth information.
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/enQtOTUzMjM4ODQwNTc5LWJmMjE3Yjc2YmI1ZDBlZmI4ZTc3OWY3ZTIwYTljYzY4MzBlODM3YzBjZTc1YmYyODRlZTJkYTgzYzBiNTA2Yjk) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time.
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events.
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.

View File

@ -6,6 +6,7 @@ set (SRCS
DateLUT.cpp
DateLUTImpl.cpp
demangle.cpp
getFQDNOrHostName.cpp
getMemoryAmount.cpp
getThreadId.cpp
JSON.cpp
@ -20,15 +21,9 @@ set (SRCS
)
if (ENABLE_REPLXX)
set (SRCS ${SRCS}
ReplxxLineReader.cpp
ReplxxLineReader.h
)
list (APPEND SRCS ReplxxLineReader.cpp)
elseif (ENABLE_READLINE)
set (SRCS ${SRCS}
ReadlineLineReader.cpp
ReadlineLineReader.h
)
list (APPEND SRCS ReadlineLineReader.cpp)
endif ()
if (USE_DEBUG_HELPERS)
@ -38,6 +33,12 @@ endif ()
add_library (common ${SRCS})
if (WITH_COVERAGE)
target_compile_definitions(common PUBLIC WITH_COVERAGE=1)
else ()
target_compile_definitions(common PUBLIC WITH_COVERAGE=0)
endif ()
target_include_directories(common PUBLIC .. ${CMAKE_CURRENT_BINARY_DIR}/..)
if(CCTZ_INCLUDE_DIR)
@ -56,8 +57,6 @@ if(CCTZ_LIBRARY)
target_link_libraries(common PRIVATE ${CCTZ_LIBRARY})
endif()
target_link_libraries(common PUBLIC replxx)
# allow explicitly fallback to readline
if (NOT ENABLE_REPLXX AND ENABLE_READLINE)
message (STATUS "Attempt to fallback to readline explicitly")
@ -82,11 +81,13 @@ endif ()
target_link_libraries (common
PUBLIC
${Poco_Net_LIBRARY}
${Poco_Util_LIBRARY}
${Poco_Foundation_LIBRARY}
${CITYHASH_LIBRARIES}
${Boost_SYSTEM_LIBRARY}
FastMemcpy
replxx
)
if (ENABLE_TESTS)

View File

@ -1,4 +1,4 @@
#include <common/argsToConfig.h>
#include "argsToConfig.h"
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>

View File

@ -1,4 +1,5 @@
#pragma once
#include <Poco/Util/Application.h>
namespace Poco::Util

View File

@ -4,4 +4,3 @@
#cmakedefine01 USE_JEMALLOC
#cmakedefine01 UNBUNDLED
#cmakedefine01 WITH_COVERAGE

View File

@ -1,16 +1,17 @@
#include <common/coverage.h>
#include <common/config_common.h>
#include "coverage.h"
#if WITH_COVERAGE
#include <unistd.h>
#include <mutex>
# include <mutex>
#if defined(__clang__)
# include <unistd.h>
# if defined(__clang__)
extern "C" void __llvm_profile_dump();
#elif defined(__GNUC__) || defined(__GNUG__)
# elif defined(__GNUC__) || defined(__GNUG__)
extern "C" void __gcov_exit();
#endif
# endif
#endif
@ -21,11 +22,11 @@ void dumpCoverageReportIfPossible()
static std::mutex mutex;
std::lock_guard lock(mutex);
#if defined(__clang__)
# if defined(__clang__)
__llvm_profile_dump();
#elif defined(__GNUC__) || defined(__GNUG__)
# elif defined(__GNUC__) || defined(__GNUG__)
__gcov_exit();
#endif
# endif
#endif
}

View File

@ -1,5 +1,5 @@
#include <Poco/Net/DNS.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
namespace

12
base/common/ya.make Normal file
View File

@ -0,0 +1,12 @@
LIBRARY()
PEERDIR(
contrib/libs/poco/Util
)
SRCS(
argsToConfig.cpp
coverage.cpp
)
END()

View File

@ -2,7 +2,7 @@
#include <daemon/BaseDaemon.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <Poco/Util/Application.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <mutex>
#include <iomanip>

View File

@ -1 +1,3 @@
RECURSE(
common
)

View File

@ -19,7 +19,7 @@
#include <Common/Exception.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/isLocalAddress.h>
#include <Common/typeid_cast.h>
#include <Common/ClickHouseRevision.h>

View File

@ -6,7 +6,7 @@
#include <thread>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <common/getMemoryAmount.h>
#include <Common/StringUtils/StringUtils.h>

View File

@ -12,7 +12,7 @@
#include <Core/ExternalTable.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <Common/config.h>

View File

@ -1,7 +1,7 @@
#pragma once
#include <Common/config.h>
#include <Poco/Net/TCPServerConnection.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/CurrentMetrics.h>
#include <Core/MySQLProtocol.h>
#include "IServer.h"

View File

@ -27,7 +27,7 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include "config_core.h"
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/getExecutablePath.h>

View File

@ -2,7 +2,7 @@
#include <Poco/Net/TCPServerConnection.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/CurrentMetrics.h>
#include <Common/Stopwatch.h>
#include <Core/Protocol.h>

View File

@ -4,7 +4,7 @@
#include <Poco/Net/DNS.h>
#include <Common/BitHelpers.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/isLocalAddress.h>
#include <Common/ProfileEvents.h>
#include <Core/Settings.h>

View File

@ -1,7 +1,7 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Core/Field.h>

View File

@ -4,7 +4,7 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Core/Defines.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/ClickHouseRevision.h>
#include <Common/config_version.h>
#include <unistd.h>

View File

@ -20,7 +20,7 @@
#include <Access/AccessRightsElement.h>
#include <Common/DNSResolver.h>
#include <Common/Macros.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
#include <Common/randomSeed.h>

View File

@ -50,10 +50,13 @@ Block PartLogElement::createBlock()
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "bytes_uncompressed"}, // Result bytes
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "read_rows"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "read_bytes"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "peak_memory_usage"},
/// Is there an error during the execution or commit
{ColumnUInt16::create(), std::make_shared<DataTypeUInt16>(), "error"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "exception"},
};
}
@ -87,10 +90,12 @@ void PartLogElement::appendToBlock(Block & block) const
columns[i++]->insert(bytes_uncompressed);
columns[i++]->insert(rows_read);
columns[i++]->insert(bytes_read_uncompressed);
columns[i++]->insert(peak_memory_usage);
columns[i++]->insert(error);
columns[i++]->insert(exception);
block.setColumns(std::move(columns));
}

View File

@ -40,11 +40,13 @@ struct PartLogElement
UInt64 bytes_uncompressed = 0;
UInt64 rows_read = 0;
UInt64 bytes_read_uncompressed = 0;
UInt64 peak_memory_usage = 0;
/// Was the operation successful?
UInt16 error = 0;
String exception;
static std::string name() { return "PartLog"; }
static Block createBlock();

View File

@ -3373,6 +3373,7 @@ try
part_log_elem.rows = (*merge_entry)->rows_written;
part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
part_log_elem.peak_memory_usage = (*merge_entry)->memory_tracker.getPeak();
}
part_log->add(part_log_elem);

View File

@ -53,6 +53,11 @@ public:
}
bool hasSortingKey() const { return part->storage.hasSortingKey(); }
Names getSortingKeyColumns() const override { return part->storage.getSortingKeyColumns(); }
protected:
StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_)
: IStorage(getIDFromPart(part_), part_->storage.getVirtuals())

View File

@ -2,6 +2,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Functions/IFunction.h>
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
namespace DB
{
@ -31,14 +32,28 @@ ReadInOrderOptimizer::ReadInOrderOptimizer(
InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage) const
{
const MergeTreeData * merge_tree = dynamic_cast<const MergeTreeData *>(storage.get());
if (!merge_tree || !merge_tree->hasSortingKey())
Names sorting_key_columns;
if (const auto * merge_tree = dynamic_cast<const MergeTreeData *>(storage.get()))
{
if (!merge_tree->hasSortingKey())
return {};
sorting_key_columns = merge_tree->getSortingKeyColumns();
}
else if (const auto * part = dynamic_cast<const StorageFromMergeTreeDataPart *>(storage.get()))
{
if (!part->hasSortingKey())
return {};
sorting_key_columns = part->getSortingKeyColumns();
}
else /// Inapplicable storage type
{
return {};
}
SortDescription order_key_prefix_descr;
int read_direction = required_sort_description.at(0).direction;
const auto & sorting_key_columns = merge_tree->getSortingKeyColumns();
size_t prefix_size = std::min(required_sort_description.size(), sorting_key_columns.size());
for (size_t i = 0; i < prefix_size; ++i)

View File

@ -19,47 +19,49 @@
Мы запускаем этот запрос и наблюдаем, с какой скоростью он выполняется. Через несколько секунд, когда скорость стабилизируется, прерываем выполнение.
В качестве скорости выполнения запроса указывается количество обработанных исходных (прочитанных из таблицы) данных в единицу времени.
Например, в таблице numbers читаемые нами данные - это числа типа UInt64 (8 байт). Если мы обрабатываем миллиард таких чисел в секунду, то отобразится скорость - 8 GB/sec. -->
<query>SELECT count() FROM zeros(100000000) WHERE NOT ignore(rand())</query>
<query>SELECT count() FROM zeros_mt(1600000000) WHERE NOT ignore(rand())</query>
<query>SELECT count() FROM zeros( 100000000) WHERE NOT ignore(rand())</query>
<query>SELECT count() FROM zeros_mt(1600000000) WHERE NOT ignore(rand())</query>
<!-- 10. Некриптографическая хэш-функция для целых чисел 64bit -> 64bit. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(intHash64(number))</query>
<query>SELECT count() FROM numbers( 100000000) WHERE NOT ignore(intHash64(number))</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(intHash64(number))</query>
<!-- 11. Некриптографическая хэш-функция для целых чисел 64bit -> 32bit. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(intHash32(number))</query>
<query>SELECT count() FROM numbers( 100000000) WHERE NOT ignore(intHash32(number))</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(intHash32(number))</query>
<!-- 12. Преобразование целого числа в строку в десятичном виде. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(toString(number))</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(toString(number))</query>
<query>SELECT count() FROM numbers( 10000000) WHERE NOT ignore(toString(number))</query>
<query>SELECT count() FROM numbers_mt(160000000) WHERE NOT ignore(toString(number))</query>
<!-- 13. Преобразование целого числа в строку путём копирования куска памяти. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(reinterpretAsString(number))</query>
<query>SELECT count() FROM numbers( 100000000) WHERE NOT ignore(reinterpretAsString(number))</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(reinterpretAsString(number))</query>
<!-- 26. Целочисленное деление на константу. Используется библиотека libdivide. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(number / 7)</query>
<query>SELECT count() FROM numbers( 100000000) WHERE NOT ignore(number / 7)</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(number / 7)</query>
<!-- 27. Целочисленное деление на константу. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(number % 7)</query>
<query>SELECT count() FROM numbers( 100000000) WHERE NOT ignore(number % 7)</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(number % 7)</query>
<!-- 28. Целочисленное деление на константу. -->
<query>SELECT count() FROM numbers(100000000) WHERE NOT ignore(number % 34908756)</query>
<query>SELECT count() FROM numbers( 100000000) WHERE NOT ignore(number % 34908756)</query>
<query>SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(number % 34908756)</query>
<!-- 29. Lookup-таблица, помещающаяся в L2-кэш. -->
<query>SELECT number % 1000 AS k, count() FROM numbers(100000000) GROUP BY k</query>
<query>SELECT number % 1000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k</query>
<query>SELECT number % 1000 AS k, count() FROM numbers( 100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<!-- 30. Хэш-таблица, помещающаяся в L3-кэш. -->
<query>SELECT number % 100000 AS k, count() FROM numbers(100000000) GROUP BY k</query>
<query>SELECT number % 100000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k</query>
<query>SELECT number % 100000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 100000 AS k, count() FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<!-- 31. Хэш-таблица, наверное помещающаяся в L3-кэш. -->
<query>SELECT number % 1000000 AS k, count() FROM numbers(100000000) GROUP BY k</query>
<query>SELECT number % 1000000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k</query>
<query>SELECT number % 1000000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000000 AS k, count() FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<!-- 32. Хэш-таблица, не помещающаяся в L3-кэш. -->
<query>SELECT number % 10000000 AS k, count() FROM numbers(100000000) GROUP BY k</query>
<query>SELECT number % 10000000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k</query>
<query>SELECT number % 10000000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10000000 AS k, count() FROM numbers_mt(80000000) GROUP BY k FORMAT Null</query>
<!-- 33. Хэш-таблица, требующая кучу оперативки. Возможны интересные эффекты. -->
<query>SELECT number % 500000000 AS k, count() FROM numbers(100000000) GROUP BY k</query>
<query>SELECT number % 500000000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k</query>
<!-- For this HT size, a single-threaded query that makes sense would be too slow (tens of seconds).
<query>SELECT number % 100000000 AS k, count() FROM numbers( 100000000) GROUP BY k FORMAT Null</query>
-->
<query>SELECT number % toUInt32(1e8) AS k, count() FROM numbers_mt(toUInt32(1e8)) GROUP BY k FORMAT Null</query>
<!-- 35. Кэш-промахи, осуществляемые из многих процессорных ядер. -->
<!-- <query>SELECT number % (intDiv(100000000, {THREADS})) AS k, count() FROM numbers_mt(1600000000) GROUP BY k</query> -->
<!-- 46. Запрос, требующий много бесполезных копирований. -->
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(materialize('xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx') AS s, concat(s,s,s,s,s,s,s,s,s,s) AS t, concat(t,t,t,t,t,t,t,t,t,t) AS u) SETTINGS max_block_size = 1000</query>
<query>SELECT count() FROM zeros(1000000) WHERE NOT ignore(materialize('xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx') AS s, concat(s,s,s,s,s,s,s,s,s,s) AS t, concat(t,t,t,t,t,t,t,t,t,t) AS u) SETTINGS max_block_size = 1000</query>
</test>

View File

@ -0,0 +1,25 @@
DROP TABLE IF EXISTS table_with_pk;
CREATE TABLE table_with_pk
(
key UInt8,
value String
)
ENGINE = MergeTree
ORDER BY key;
INSERT INTO table_with_pk SELECT number, toString(number % 10) FROM numbers(10000000);
ALTER TABLE table_with_pk DELETE WHERE key % 77 = 0 SETTINGS mutations_sync = 1;
SYSTEM FLUSH LOGS;
-- Memory usage for all mutations must be almost constant and less than
-- read_bytes.
SELECT
DISTINCT read_bytes >= peak_memory_usage
FROM
system.part_log2
WHERE event_type = 'MutatePart' AND table = 'table_with_pk' AND database = currentDatabase();
DROP TABLE IF EXISTS table_with_pk;

View File

@ -274,6 +274,7 @@ if unstable_queries:
error_tests += slow_average_tests
if error_tests:
status = 'failure'
message_array.append(str(error_tests) + ' errors')
if message_array:

View File

@ -1,4 +1,4 @@
# Array(T) { #data_type-array}
# Array(T) {#data_type-array}
Array of `T`-type items.

View File

@ -1,4 +1,4 @@
# DateTime { #data_type-datetime}
# DateTime {#data_type-datetime}
Allows to store an instant in time, that can be expressed as a calendar date and a time of a day.

View File

@ -1,4 +1,4 @@
# DateTime64 { #data_type-datetime64}
# DateTime64 {#data_type-datetime64}
Allows to store an instant in time, that can be expressed as a calendar date and a time of a day, with defined sub-second precision

View File

@ -27,7 +27,7 @@ SELECT 1 - 0.9
- Floating-point calculations might result in numbers such as infinity (`Inf`) and "not-a-number" (`NaN`). This should be taken into account when processing the results of calculations.
- When parsing floating-point numbers from text, the result might not be the nearest machine-representable number.
## NaN and Inf { #data_type-float-nan-inf}
## NaN and Inf {#data_type-float-nan-inf}
In contrast to standard SQL, ClickHouse supports the following categories of floating-point numbers:

View File

@ -1,4 +1,4 @@
# Data Types { #data_types}
# Data Types {#data_types}
ClickHouse can store various kinds of data in table cells.

View File

@ -1,4 +1,4 @@
# AggregateFunction(name, types_of_arguments...) { #data_type-aggregatefunction}
# AggregateFunction(name, types_of_arguments...) {#data_type-aggregatefunction}
The intermediate state of an aggregate function. To get it, use aggregate functions with the `-State` suffix. To get aggregated data in the future, you must use the same aggregate functions with the `-Merge`suffix.

View File

@ -1,4 +1,4 @@
# Nullable(TypeName) { #data_type-nullable}
# Nullable(TypeName) {#data_type-nullable}
Allows to store special marker ([NULL](../query_language/syntax.md)) that denotes "missing value" alongside normal values allowed by `TypeName`. For example, a `Nullable(Int8)` type column can store `Int8` type values, and the rows that don't have a value will store `NULL`.

View File

@ -1,4 +1,4 @@
# Interval { #data-type-interval}
# Interval {#data-type-interval}
The family of data types representing time and date intervals. The resulting types of the [INTERVAL](../../query_language/operators.md#operator-interval) operator.
@ -32,7 +32,7 @@ SELECT toTypeName(INTERVAL 4 DAY)
└──────────────────────────────┘
```
## Usage Remarks { #data-type-interval-usage-remarks}
## Usage Remarks {#data-type-interval-usage-remarks}
You can use `Interval`-type values in arithmetical operations with [Date](../../data_types/date.md) and [DateTime](../../data_types/datetime.md)-type values. For example, you can add 4 days to the current time:

View File

@ -1,6 +1,5 @@
# Set
Used for the right half of an [IN](../../query_language/select.md##select-in-operators) expression.
Used for the right half of an [IN](../../query_language/select.md#select-in-operators) expression.
[Original article](https://clickhouse.tech/docs/en/data_types/special_data_types/set/) <!--hide-->

View File

@ -1,4 +1,4 @@
# UUID { #uuid-data-type}
# UUID {#uuid-data-type}
A universally unique identifier (UUID) is a 16-byte number used to identify records. For detailed information about the UUID, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier).

View File

@ -2,16 +2,16 @@
## Why Not Use Something Like MapReduce?
We can refer to systems like MapReduce as distributed computing systems in which the reduce operation is based on distributed sorting. The most common open source solution in this class is [Apache Hadoop](http://hadoop.apache.org). Yandex uses their in-house solution, YT.
We can refer to systems like MapReduce as distributed computing systems in which the reduce operation is based on distributed sorting. The most common open-source solution in this class is [Apache Hadoop](http://hadoop.apache.org). Yandex uses its in-house solution, YT.
These systems aren't appropriate for online queries due to their high latency. In other words, they can't be used as the back-end for a web interface.
These types of systems aren't useful for real-time data updates.
Distributed sorting isn't the best way to perform reduce operations if the result of the operation and all the intermediate results (if there are any) are located in the RAM of a single server, which is usually the case for online queries. In such a case, a hash table is the optimal way to perform reduce operations. A common approach to optimizing map-reduce tasks is pre-aggregation (partial reduce) using a hash table in RAM. The user performs this optimization manually.
Distributed sorting isn't the best way to perform reduce operations if the result of the operation and all the intermediate results (if there are any) are located in the RAM of a single server, which is usually the case for online queries. In such a case, a hash table is an optimal way to perform reduce operations. A common approach to optimizing map-reduce tasks is pre-aggregation (partial reduce) using a hash table in RAM. The user performs this optimization manually.
Distributed sorting is one of the main causes of reduced performance when running simple map-reduce tasks.
Most MapReduce implementations allow you to execute arbitrary code on a cluster. But a declarative query language is better suited to OLAP in order to run experiments quickly. For example, Hadoop has Hive and Pig. Also consider Cloudera Impala or Shark (outdated) for Spark, as well as Spark SQL, Presto, and Apache Drill. Performance when running such tasks is highly sub-optimal compared to specialized systems, but relatively high latency makes it unrealistic to use these systems as the backend for a web interface.
Most MapReduce implementations allow you to execute arbitrary code on a cluster. But a declarative query language is better suited to OLAP to run experiments quickly. For example, Hadoop has Hive and Pig. Also consider Cloudera Impala or Shark (outdated) for Spark, as well as Spark SQL, Presto, and Apache Drill. Performance when running such tasks is highly sub-optimal compared to specialized systems, but relatively high latency makes it unrealistic to use these systems as the backend for a web interface.
## What If I Have a Problem with Encodings When Using Oracle Through ODBC? { #oracle-odbc-encodings}
## What If I Have a Problem with Encodings When Using Oracle Through ODBC? {#oracle-odbc-encodings}
If you use Oracle through the ODBC driver as a source of external dictionaries, you need to set the correct value for the `NLS_LANG` environment variable in `/etc/default/clickhouse`. For more information, see the [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html).
@ -21,7 +21,7 @@ If you use Oracle through the ODBC driver as a source of external dictionaries,
NLS_LANG=RUSSIAN_RUSSIA.UTF8
```
## How Do I Export Data from ClickHouse to a File? { #how-to-export-to-file}
## How Do I Export Data from ClickHouse to a File? {#how-to-export-to-file}
### Using INTO OUTFILE Clause

View File

@ -2,7 +2,7 @@
See <https://amplab.cs.berkeley.edu/benchmark/>
Sign up for a free account at <https://aws.amazon.com>. You will need a credit card, email and phone number.Get a new access key at <https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential>
Sign up for a free account at <https://aws.amazon.com>. You will need a credit card, email and phone number. Get a new access key at <https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential>
Run the following in the console:

View File

@ -272,7 +272,7 @@ SELECT formatReadableSize(sum(bytes)) FROM system.parts WHERE table = 'trips_mer
└────────────────────────────────┘
```
Among other things, you can run the OPTIMIZE query on MergeTree. But it's not required, since everything will be fine without it.
Among other things, you can run the OPTIMIZE query on MergeTree. But it's not required since everything will be fine without it.
## Download of Prepared Partitions
@ -285,8 +285,7 @@ $ clickhouse-client --query "select count(*) from datasets.trips_mergetree"
```
!!!info
If you will run queries described below, you have to use full table name,
`datasets.trips_mergetree`.
If you will run the queries described below, you have to use the full table name, `datasets.trips_mergetree`.
## Results on Single Server
@ -330,7 +329,7 @@ The following server was used:
Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,128 GiB RAM,8x6 TB HD on hardware RAID-5
Execution time is the best of three runsBut starting from the second run, queries read data from the file system cache. No further caching occurs: the data is read out and processed in each run.
Execution time is the best of three runs. But starting from the second run, queries read data from the file system cache. No further caching occurs: the data is read out and processed in each run.
Creating a table on three servers:
@ -363,7 +362,7 @@ Q4: 1.241 seconds.
No surprises here, since the queries are scaled linearly.
We also have results from a cluster of 140 servers:
We also have the results from a cluster of 140 servers:
Q1: 0.028 sec.
Q2: 0.043 sec.

View File

@ -1,4 +1,3 @@
# OnTime
This dataset can be obtained in two ways:
@ -158,8 +157,7 @@ $ clickhouse-client --query "select count(*) from datasets.ontime"
```
!!!info
If you will run queries described below, you have to use full table name,
`datasets.ontime`.
If you will run the queries described below, you have to use the full table name, `datasets.ontime`.
## Queries
@ -195,7 +193,7 @@ GROUP BY DayOfWeek
ORDER BY c DESC;
```
Q3. The number of delays by airport for 2000-2008
Q3. The number of delays by the airport for 2000-2008
```sql
SELECT Origin, count(*) AS c

View File

@ -1,6 +1,6 @@
# Getting Started
If you are new to ClickHouse and want to get a hands-on feeling of it's performance, first of all you need to go through the [installation process](install.md). After that you can:
If you are new to ClickHouse and want to get a hands-on feeling of its performance, first of all, you need to go through the [installation process](install.md). After that you can:
* [Go through detailed tutorial](tutorial.md)
* [Experiment with example datasets](example_datasets/ontime.md)

View File

@ -14,7 +14,7 @@ To run ClickHouse on processors that do not support SSE 4.2 or have AArch64 or P
## Available Installation Options
### From DEB Packages { #install-from-deb-packages}
### From DEB Packages {#install-from-deb-packages}
It is recommended to use official pre-compiled `deb` packages for Debian or Ubuntu.
@ -40,15 +40,15 @@ You can also download and install packages manually from here: <https://repo.yan
#### Packages
- `clickhouse-common-static` — Installs ClickHouse compiled binary files.
- `clickhouse-server` — Creates symbolic link for `clickhouse-server`. Installs server configuration.
- `clickhouse-client` — Creates symbolic link for `clickhouse-client` and other client-related tools. Installs client configurations.
- `clickhouse-server` — Creates a symbolic link for `clickhouse-server`. Installs server configuration.
- `clickhouse-client` — Creates a symbolic link for `clickhouse-client` and other client-related tools. Installs client configurations.
- `clickhouse-common-static-dbg` — Installs ClickHouse compiled binary files with debug info.
### From RPM Packages
It is recommended to use official pre-compiled `rpm` packages for CentOS, RedHat and all other rpm-based Linux distributions.
First you need to add the official repository:
First, you need to add the official repository:
```bash
sudo yum install yum-utils
@ -58,7 +58,7 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_6
If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). The `prestable` tag is sometimes available too.
Then run these commands to actually install packages:
Then run these commands to install packages:
```bash
sudo yum install clickhouse-server clickhouse-client
@ -66,11 +66,11 @@ sudo yum install clickhouse-server clickhouse-client
You can also download and install packages manually from here: <https://repo.clickhouse.tech/rpm/stable/x86_64>.
### From tgz archives { #from-tgz-archives}
### From tgz archives {#from-tgz-archives}
It is recommended to use official pre-compiled `tgz` archives for all Linux distributions, where installation of `deb` or `rpm` packages is not possible.
Required version can be dowloaded with `curl` or `wget` from repository <https://repo.yandex.ru/clickhouse/tgz/>.
The required version can be downloaded with `curl` or `wget` from repository <https://repo.yandex.ru/clickhouse/tgz/>.
After that downloaded archives should be unpacked and installed with installation scripts. Example for the latest version:
```bash
export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1`
@ -93,7 +93,7 @@ tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
```
For production environments it's recommended to use latest `stable`-version. You can find it's number on github page https://github.com/ClickHouse/ClickHouse/tags with postfix `-stable`.
For production environments, it's recommended to use the latest `stable`-version. You can find its number on GitHub page https://github.com/ClickHouse/ClickHouse/tags with postfix `-stable`.
### From Docker Image
@ -116,7 +116,7 @@ You'll need to create a data and metadata folders and `chown` them for the desir
/opt/clickhouse/metadata/default/
```
On Gentoo you can just use `emerge clickhouse` to install ClickHouse from sources.
On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sources.
## Launch
@ -156,7 +156,7 @@ After launching server, you can use the command-line client to connect to it:
$ clickhouse-client
```
By default it connects to `localhost:9000` on behalf of the user `default` without a password. It can also be used to connect to a remote server using `--host` argument.
By default, it connects to `localhost:9000` on behalf of the user `default` without a password. It can also be used to connect to a remote server using `--host` argument.
The terminal must use UTF-8 encoding.
For more information, see the section ["Command-line client"](../interfaces/cli.md).
@ -183,7 +183,7 @@ SELECT 1
**Congratulations, the system works!**
To continue experimenting, you can download one of test data sets or go through [tutorial](https://clickhouse.tech/tutorial.html).
To continue experimenting, you can download one of the test data sets or go through [tutorial](https://clickhouse.tech/tutorial.html).
[Original article](https://clickhouse.tech/docs/en/getting_started/install/) <!--hide-->

View File

@ -2,7 +2,7 @@
## What to Expect from This Tutorial?
By going through this tutorial you'll learn how to set up basic ClickHouse cluster, it'll be small, but fault tolerant and scalable. We will use one of example datasets to fill it with data and execute some demo queries.
By going through this tutorial you'll learn how to set up basic ClickHouse cluster, it'll be small, but fault-tolerant and scalable. We will use one of the example datasets to fill it with data and execute some demo queries.
## Single Node Setup
@ -25,9 +25,9 @@ What do we have in the packages that got installed:
* `clickhouse-common` package contains a ClickHouse executable file.
* `clickhouse-server` package contains configuration files to run ClickHouse as a server.
Server config files are located in `/etc/clickhouse-server/`. Before going further please notice the `<path>` element in `config.xml`. Path determines the location for data storage, so it should be located on volume with large disk capacity, the default value is `/var/lib/clickhouse/`. If you want to adjust the configuration it's not really handy to directly edit `config.xml` file, considering it might get rewritten on future package updates. Recommended way to override the config elements is to create [files in config.d directory](../operations/configuration_files.md) which serve as "patches" to config.xml.
Server config files are located in `/etc/clickhouse-server/`. Before going further please notice the `<path>` element in `config.xml`. Path determines the location for data storage, so it should be located on volume with large disk capacity, the default value is `/var/lib/clickhouse/`. If you want to adjust the configuration it's not handy to directly edit `config.xml` file, considering it might get rewritten on future package updates. The recommended way to override the config elements is to create [files in config.d directory](../operations/configuration_files.md) which serve as "patches" to config.xml.
As you might have noticed, `clickhouse-server` is not launched automatically after package installation. It won't be automatically restarted after updates either. The way you start the server depends on your init system, usually it's:
As you might have noticed, `clickhouse-server` is not launched automatically after package installation. It won't be automatically restarted after updates either. The way you start the server depends on your init system, usually, it's:
``` bash
sudo service clickhouse-server start
@ -38,7 +38,7 @@ or
sudo /etc/init.d/clickhouse-server start
```
The default location for server logs is `/var/log/clickhouse-server/`. Server will be ready to handle client connections once `Ready for connections` message was logged.
The default location for server logs is `/var/log/clickhouse-server/`. The server will be ready to handle client connections once `Ready for connections` message was logged.
Once the `clickhouse-server` is up and running, we can use `clickhouse-client` to connect to the server and run some test queries like `SELECT "Hello, world!";`.
@ -71,7 +71,7 @@ clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv
## Import Sample Dataset
Now it's time to fill our ClickHouse server with some sample data. In this tutorial we'll use anonymized data of Yandex.Metrica, the first service that run ClickHouse in production way before it became open-source (more on that in [history section](../introduction/history.md)). There are [multiple ways to import Yandex.Metrica dataset](example_datasets/metrica.md) and for the sake of the tutorial we'll go with the most realistic one.
Now it's time to fill our ClickHouse server with some sample data. In this tutorial, we'll use anonymized data of Yandex.Metrica, the first service that runs ClickHouse in production way before it became open-source (more on that in [history section](../introduction/history.md)). There are [multiple ways to import Yandex.Metrica dataset](example_datasets/metrica.md) and for the sake of the tutorial, we'll go with the most realistic one.
### Download and Extract Table Data
@ -96,7 +96,7 @@ Syntax for creating tables is way more complicated compared to databases (see [r
2. Table schema, i.e. list of columns and their [data types](../data_types/index.md).
3. [Table engine](../operations/table_engines/index.md) and it's settings, which determines all the details on how queries to this table will be physically executed.
Yandex.Metrica is a web analytics service and sample dataset doesn't cover it's full functionality, so there are only two tables to create:
Yandex.Metrica is a web analytics service and sample dataset doesn't cover its full functionality, so there are only two tables to create:
* `hits` is a table with each action done by all users on all websites covered by the service.
* `visits` is a table that contains pre-built sessions instead of individual actions.
@ -444,7 +444,7 @@ SAMPLE BY intHash32(UserID)
SETTINGS index_granularity = 8192
```
You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want.
You can execute those queries using the interactive mode of `clickhouse-client` (just launch it in a terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want.
As we can see, `hits_v1` uses the [basic MergeTree engine](../operations/table_engines/mergetree.md), while the `visits_v1` uses the [Collapsing](../operations/table_engines/collapsingmergetree.md) variant.
@ -470,7 +470,7 @@ FORMAT TSV
max_insert_block_size 1048576 0 "The maximum block size for insertion, if we control the creation of blocks for insertion."
```
Optionally you can [OPTIMIZE](../query_language/misc/#misc_operations-optimize) the tables after import. Tables that are configured with MergeTree-family engine always do merges of data parts in background to optimize data storage (or at least check if it makes sense). These queries will just force table engine to do storage optimization right now instead of some time later:
Optionally you can [OPTIMIZE](../query_language/misc/#misc_operations-optimize) the tables after import. Tables that are configured with MergeTree-family engine always do merges of data parts in background to optimize data storage (or at least check if it makes sense). These queries will just force the table engine to do storage optimization right now instead of some time later:
``` bash
clickhouse-client --query "OPTIMIZE TABLE tutorial.hits_v1 FINAL"
clickhouse-client --query "OPTIMIZE TABLE tutorial.visits_v1 FINAL"
@ -517,7 +517,7 @@ ClickHouse cluster is a homogenous cluster. Steps to set up:
[Distributed table](../operations/table_engines/distributed.md) is actually a kind of "view" to local tables of ClickHouse cluster. SELECT query from a distributed table will be executed using resources of all cluster's shards. You may specify configs for multiple clusters and create multiple distributed tables providing views to different clusters.
Example config for cluster with three shards, one replica each:
Example config for a cluster with three shards, one replica each:
``` xml
<remote_servers>
<perftest_3shards_1replicas>
@ -543,7 +543,7 @@ Example config for cluster with three shards, one replica each:
</remote_servers>
```
For further demonstration let's create new local table with exactly the same `CREATE TABLE` query that we used for `hits_v1`, but different table name:
For further demonstration let's create a new local table with the same `CREATE TABLE` query that we used for `hits_v1`, but different table name:
``` sql
CREATE TABLE tutorial.hits_local (...) ENGINE = MergeTree() ...
```
@ -554,9 +554,9 @@ CREATE TABLE tutorial.hits_all AS tutorial.hits_local
ENGINE = Distributed(perftest_3shards_1replicas, tutorial, hits_local, rand());
```
Common practice is to create similar Distributed tables on all machines of the cluster. This would allow to run distributed queries on any machine of the cluster. Also there's an alternative option to create temporary distributed table for a given SELECT query using [remote](../query_language/table_functions/remote.md) table function.
A common practice is to create similar Distributed tables on all machines of the cluster. This would allow running distributed queries on any machine of the cluster. Also there's an alternative option to create temporary distributed table for a given SELECT query using [remote](../query_language/table_functions/remote.md) table function.
Let's run [INSERT SELECT](../query_language/insert_into.md) into Distributed table to spread the table to multiple servers.
Let's run [INSERT SELECT](../query_language/insert_into.md) into the Distributed table to spread the table to multiple servers.
``` sql
INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1;
@ -567,11 +567,11 @@ INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1;
As you could expect computationally heavy queries are executed N times faster being launched on 3 servers instead of one.
In this case we have used a cluster with 3 shards each contains a single replica.
In this case, we have used a cluster with 3 shards each contains a single replica.
To provide resilience in production environment we recommend that each shard should contain 2-3 replicas distributed between multiple data-centers. Note that ClickHouse supports unlimited number of replicas.
To provide resilience in a production environment we recommend that each shard should contain 2-3 replicas distributed between multiple datacenters. Note that ClickHouse supports an unlimited number of replicas.
Example config for cluster of one shard containing three replicas:
Example config for a cluster of one shard containing three replicas:
``` xml
<remote_servers>
...
@ -597,10 +597,10 @@ Example config for cluster of one shard containing three replicas:
To enable native replication <a href="http://zookeeper.apache.org/" rel="external nofollow">ZooKeeper</a> is required. ClickHouse will take care of data consistency on all replicas and run restore procedure after failure
automatically. It's recommended to deploy ZooKeeper cluster to separate servers.
ZooKeeper is not a strict requirement: in some simple cases you can duplicate the data by writing it into all the replicas from your application code. This approach is **not** recommended, in this case ClickHouse won't be able to
ZooKeeper is not a strict requirement: in some simple cases, you can duplicate the data by writing it into all the replicas from your application code. This approach is **not** recommended, in this case, ClickHouse won't be able to
guarantee data consistency on all replicas. This remains the responsibility of your application.
ZooKeeper locations need to be specified in configuration file:
ZooKeeper locations need to be specified in the configuration file:
``` xml
<zookeeper>
<node>
@ -618,7 +618,7 @@ ZooKeeper locations need to be specified in configuration file:
</zookeeper>
```
Also we need to set macros for identifying each shard and replica, it will be used on table creation:
Also, we need to set macros for identifying each shard and replica, it will be used on table creation:
``` xml
<macros>
<shard>01</shard>
@ -626,7 +626,7 @@ Also we need to set macros for identifying each shard and replica, it will be us
</macros>
```
If there are no replicas at the moment on replicated table creation, a new first replica will be instantiated. If there are already live replicas, new replica will clone the data from existing ones. You have an option to create all replicated tables first and that insert data to it. Another option is to create some replicas and add the others after or during data insertion.
If there are no replicas at the moment on replicated table creation, a new first replica will be instantiated. If there are already live replicas, the new replica will clone the data from existing ones. You have an option to create all replicated tables first and that insert data to it. Another option is to create some replicas and add the others after or during data insertion.
``` sql
CREATE TABLE tutorial.hits_replica (...)
@ -642,4 +642,6 @@ Here we use [ReplicatedMergeTree](../operations/table_engines/replication.md) ta
``` sql
INSERT INTO tutorial.hits_replica SELECT * FROM tutorial.hits_local;
```
Replication operates in multi-master mode. Data can be loaded into any replica and it will be synced with other instances automatically. Replication is asynchronous so at a given moment of time not all replicas may contain recently inserted data. To allow data insertion at least one replica should be up. Others will sync up data and repair consistency once they will become active again. Please notice that such approach allows for the low possibility of loss of just appended data.
Replication operates in multi-master mode. Data can be loaded into any replica and it will be synced with other instances automatically. Replication is asynchronous so at a given moment, not all replicas may contain recently inserted data. To allow data insertion at least one replica should be up. Others will sync up data and repair consistency once they will become active again. Please notice that such an approach allows for the low possibility of a loss of just appended data.
[Original article](https://clickhouse.tech/docs/en/getting_started/tutorial/) <!--hide-->

View File

@ -1,4 +1,4 @@
# Applying a Catboost Model in ClickHouse { #applying-catboost-model-in-clickhouse}
# Applying a Catboost Model in ClickHouse {#applying-catboost-model-in-clickhouse}
[CatBoost](https://catboost.ai) is a free and open-source gradient boosting library developed at [Yandex](https://yandex.com/company/) for machine learning.
@ -13,7 +13,7 @@ To apply a CatBoost model in ClickHouse:
For more information about training CatBoost models, see [Training and applying models](https://catboost.ai/docs/features/training.html#training).
## Prerequisites { #prerequisites}
## Prerequisites {#prerequisites}
If you don't have the [Docker](https://docs.docker.com/install/) yet, install it.
@ -44,11 +44,11 @@ yandex/tutorial-catboost-clickhouse latest 622e4d17945b 22
$ docker run -it -p 8888:8888 yandex/tutorial-catboost-clickhouse
```
## 1. Create a Table { #create-table}
## 1. Create a Table {#create-table}
To create a ClickHouse table for the train sample:
To create a ClickHouse table for the training sample:
**1.** Start ClickHouse console client in interactive mode:
**1.** Start ClickHouse console client in the interactive mode:
```bash
$ clickhouse client
@ -83,7 +83,7 @@ ENGINE = MergeTree ORDER BY date
:) exit
```
## 2. Insert the Data to the Table { #insert-data-to-table}
## 2. Insert the Data to the Table {#insert-data-to-table}
To insert the data:
@ -93,7 +93,7 @@ To insert the data:
$ clickhouse client --host 127.0.0.1 --query 'INSERT INTO amazon_train FORMAT CSVWithNames' < ~/amazon/train.csv
```
**2.** Start ClickHouse console client in interactive mode:
**2.** Start ClickHouse console client in the interactive mode:
```bash
$ clickhouse client
@ -112,7 +112,7 @@ FROM amazon_train
+---------+
```
## 3. Integrate CatBoost into ClickHouse { #integrate-catboost-into-clickhouse}
## 3. Integrate CatBoost into ClickHouse {#integrate-catboost-into-clickhouse}
!!! note "Note"
**Optional step.** The Docker image contains everything you need to run CatBoost and ClickHouse.
@ -154,7 +154,7 @@ The fastest way to evaluate a CatBoost model is compile `libcatboostmodel.<so|dl
<models_config>/home/catboost/models/*_model.xml</models_config>
```
## 4. Run the Model Inference from SQL { #run-model-inference}
## 4. Run the Model Inference from SQL {#run-model-inference}
For test model run the ClickHouse client `$ clickhouse client`.
@ -180,7 +180,7 @@ LIMIT 10
!!! note "Note"
Function [modelEvaluate](../query_language/functions/other_functions.md#function-modelevaluate) returns tuple with per-class raw predictions for multiclass models.
Let's predict probability:
Let's predict the probability:
```sql
:) SELECT
@ -228,3 +228,5 @@ FROM
!!! note "Note"
More info about [avg()](../query_language/agg_functions/reference.md#agg_function-avg) and [log()](../query_language/functions/math_functions.md) functions.
[Original article](https://clickhouse.tech/docs/en/guides/apply_catboost_model/) <!--hide-->

View File

@ -2,4 +2,6 @@
Detailed step-by-step instructions that will help you solve various tasks using ClickHouse.
- [Applying a CatBoost Model in ClickHouse](apply_catboost_model.md)
- [Applying a CatBoost Model in ClickHouse](apply_catboost_model.md)
[Original article](https://clickhouse.tech/docs/en/guides/) <!--hide-->

View File

@ -19,7 +19,7 @@ Different client and server versions are compatible with one another, but some f
ClickHouse client version is older than ClickHouse server. It may lack support for new features.
```
## Usage { #cli_usage}
## Usage {#cli_usage}
The client can be used in interactive and non-interactive (batch) mode.
To use batch mode, specify the 'query' parameter, or send data to 'stdin' (it verifies that 'stdin' is not a terminal), or both.
@ -40,7 +40,7 @@ $ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FOR
In batch mode, the default data format is TabSeparated. You can set the format in the FORMAT clause of the query.
By default, you can only process a single query in batch mode. To make multiple queries from a "script," use the --multiquery parameter. This works for all queries except INSERT. Query results are output consecutively without additional separators.
By default, you can only process a single query in batch mode. To make multiple queries from a "script," use the `--multiquery` parameter. This works for all queries except INSERT. Query results are output consecutively without additional separators.
Similarly, to process a large number of queries, you can run 'clickhouse-client' for each query. Note that it may take tens of milliseconds to launch the 'clickhouse-client' program.
In interactive mode, you get a command line where you can enter queries.
@ -67,11 +67,11 @@ When processing a query, the client shows:
3. The result in the specified format.
4. The number of lines in the result, the time passed, and the average speed of query processing.
You can cancel a long query by pressing Ctrl+C. However, you will still need to wait a little for the server to abort the request. It is not possible to cancel a query at certain stages. If you don't wait and press Ctrl+C a second time, the client will exit.
You can cancel a long query by pressing Ctrl+C. However, you will still need to wait for a little for the server to abort the request. It is not possible to cancel a query at certain stages. If you don't wait and press Ctrl+C a second time, the client will exit.
The command-line client allows passing external data (external temporary tables) for querying. For more information, see the section "External data for query processing".
### Queries with Parameters { #cli-queries-with-parameters}
### Queries with Parameters {#cli-queries-with-parameters}
You can create a query with parameters and pass values to them from client application. This allows to avoid formatting query with specific dynamic values on client side. For example:
@ -79,7 +79,7 @@ You can create a query with parameters and pass values to them from client appli
$ clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}"
```
#### Query Syntax { #cli-queries-with-parameters-syntax}
#### Query Syntax {#cli-queries-with-parameters-syntax}
Format a query as usual, then place the values that you want to pass from the app parameters to the query in braces in the following format:
@ -96,7 +96,7 @@ Format a query as usual, then place the values that you want to pass from the ap
$ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}"
```
## Configuring { #interfaces_cli_configuration}
## Configuring {#interfaces_cli_configuration}
You can pass parameters to `clickhouse-client` (all parameters have a default value) using:

View File

@ -1,4 +1,4 @@
# Formats for Input and Output Data { #formats}
# Formats for Input and Output Data {#formats}
ClickHouse can accept and return data in various formats. A format supported for input can be used to parse the data provided to `INSERT`s, to perform `SELECT`s from a file-backed table such as File, URL or HDFS, or to read an external dictionary. A format supported for output can be used to arrange the
results of a `SELECT`, and to perform `INSERT`s into a file-backed table.
@ -42,9 +42,9 @@ The supported formats are:
You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section.
## TabSeparated { #tabseparated}
## TabSeparated {#tabseparated}
In TabSeparated format, data is written by row. Each row contains values separated by tabs. Each value is follow by a tab, except the last value in the row, which is followed by a line feed. Strictly Unix line feeds are assumed everywhere. The last row also must contain a line feed at the end. Values are written in text format, without enclosing quotation marks, and with special characters escaped.
In TabSeparated format, data is written by row. Each row contains values separated by tabs. Each value is followed by a tab, except the last value in the row, which is followed by a line feed. Strictly Unix line feeds are assumed everywhere. The last row also must contain a line feed at the end. Values are written in text format, without enclosing quotation marks, and with special characters escaped.
This format is also available under the name `TSV`.
@ -80,13 +80,13 @@ During formatting, accuracy may be lost on floating-point numbers.
During parsing, it is not strictly required to read the nearest machine-representable number.
Dates are written in YYYY-MM-DD format and parsed in the same format, but with any characters as separators.
Dates with times are written in the format YYYY-MM-DD hh:mm:ss and parsed in the same format, but with any characters as separators.
This all occurs in the system time zone at the time the client or server starts (depending on which one formats data). For dates with times, daylight saving time is not specified. So if a dump has times during daylight saving time, the dump does not unequivocally match the data, and parsing will select one of the two times.
Dates with times are written in the format `YYYY-MM-DD hh:mm:ss` and parsed in the same format, but with any characters as separators.
This all occurs in the system time zone at the time the client or server starts (depending on which of them formats data). For dates with times, daylight saving time is not specified. So if a dump has times during daylight saving time, the dump does not unequivocally match the data, and parsing will select one of the two times.
During a read operation, incorrect dates and dates with times can be parsed with natural overflow or as null dates and times, without an error message.
As an exception, parsing dates with times is also supported in Unix timestamp format, if it consists of exactly 10 decimal digits. The result is not time zone-dependent. The formats YYYY-MM-DD hh:mm:ss and NNNNNNNNNN are differentiated automatically.
Strings are output with backslash-escaped special characters. The following escape sequences are used for output: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. Parsing also supports the sequences `\a`, `\v`, and `\xHH` (hex escape sequences) and any `\c` sequences, where `c` is any character (these sequences are converted to `c`). Thus, reading data supports formats where a line feed can be written as `\n` or `\`, or as a line feed. For example, the string `Hello world` with a line feed between the words instead of a space can be parsed in any of the following variations:
Strings are output with backslash-escaped special characters. The following escape sequences are used for output: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. Parsing also supports the sequences `\a`, `\v`, and `\xHH` (hex escape sequences) and any `\c` sequences, where `c` is any character (these sequences are converted to `c`). Thus, reading data supports formats where a line feed can be written as `\n` or `\`, or as a line feed. For example, the string `Hello world` with a line feed between the words instead of space can be parsed in any of the following variations:
```text
Hello\nworld
@ -127,17 +127,17 @@ INSERT INTO nestedt Values ( 1, [1], ['a'])
SELECT * FROM nestedt FORMAT TSV
```
```text
1 [1] ['a']
1 [1] ['a']
```
## TabSeparatedRaw { #tabseparatedraw}
## TabSeparatedRaw {#tabseparatedraw}
Differs from `TabSeparated` format in that the rows are written without escaping.
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
This format is also available under the name `TSVRaw`.
## TabSeparatedWithNames { #tabseparatedwithnames}
## TabSeparatedWithNames {#tabseparatedwithnames}
Differs from the `TabSeparated` format in that the column names are written in the first row.
During parsing, the first row is completely ignored. You can't use column names to determine their position or to check their correctness.
@ -145,16 +145,16 @@ During parsing, the first row is completely ignored. You can't use column names
This format is also available under the name `TSVWithNames`.
## TabSeparatedWithNamesAndTypes { #tabseparatedwithnamesandtypes}
## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes}
Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row.
During parsing, the first and second rows are completely ignored.
This format is also available under the name `TSVWithNamesAndTypes`.
## Template { #format-template}
## Template {#format-template}
This format allows to specify a custom format string with placeholders for values with specified escaping rule.
This format allows specifying a custom format string with placeholders for values with a specified escaping rule.
It uses settings `format_template_resultset`, `format_template_row`, `format_template_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further)
@ -172,7 +172,7 @@ Setting `format_template_row` specifies path to file, which contains format stri
- `Raw` (without escaping, similarly to `TSVRaw`)
- `None` (no escaping rule, see further)
If escaping rule is omitted, then`None` will be used. `XML` and `Raw` are suitable only for output.
If an escaping rule is omitted, then `None` will be used. `XML` and `Raw` are suitable only for output.
So, for the following format string:
@ -184,21 +184,21 @@ Setting `format_template_row` specifies path to file, which contains format stri
The `format_template_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default)
Setting `format_template_resultset` specifies path to file, which contains format string for resultset. Format string for resultset has the same syntax as format string for row and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
Setting `format_template_resultset` specifies the path to file, which contains a format string for resultset. Format string for resultset has the same syntax as a format string for row and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
- `data` is the rows with data in `format_template_row` format, separated by `format_template_rows_between_delimiter`. This placeholder must be the first placeholder in the format string.
- `totals` is the row with total values in `format_template_row` format (when using WITH TOTALS)
- `min` is the row with minimum values in `format_template_row` format (when extremes is set to 1)
- `max` is the row with maximum values in `format_template_row` format (when extremes is set to 1)
- `min` is the row with minimum values in `format_template_row` format (when extremes are set to 1)
- `max` is the row with maximum values in `format_template_row` format (when extremes are set to 1)
- `rows` is the total number of output rows
- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT.
- `time` is the request execution time in seconds
- `rows_read` is the number of rows have been read
- `bytes_read` is the number of bytes (uncompressed) have been read
- `rows_read` is the number of rows has been read
- `bytes_read` is the number of bytes (uncompressed) has been read
The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified.
If the `format_template_resultset` setting is an empty string, `${data}` is used as default value.
For insert queries format allows to skip some columns or some fields if prefix or suffix (see example).
For insert queries format allows skipping some columns or some fields if prefix or suffix (see example).
Select example:
```sql
@ -268,7 +268,7 @@ Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Du
`PageViews`, `UserID`, `Duration` and `Sign` inside placeholders are names of columns in the table. Values after `Useless field` in rows and after `\nTotal rows: ` in suffix will be ignored.
All delimiters in the input data must be strictly equal to delimiters in specified format strings.
## TemplateIgnoreSpaces { #templateignorespaces}
## TemplateIgnoreSpaces {#templateignorespaces}
This format is suitable only for input.
Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters.
@ -286,7 +286,7 @@ format_template_resultset = '/some/path/resultset.format', format_template_row =
{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}
```
## TSKV { #tskv}
## TSKV {#tskv}
Similar to TabSeparated, but outputs a value in name=value format. Names are escaped the same way as in TabSeparated format, and the = symbol is also escaped.
@ -310,7 +310,7 @@ SELECT * FROM t_null FORMAT TSKV
```
```text
x=1 y=\N
x=1 y=\N
```
When there is a large number of small columns, this format is ineffective, and there is generally no reason to use it. Nevertheless, it is no worse than JSONEachRow in terms of efficiency.
@ -319,11 +319,11 @@ Both data output and parsing are supported in this format. For parsing, any orde
Parsing allows the presence of the additional field `tskv` without the equal sign or a value. This field is ignored.
## CSV { #csv}
## CSV {#csv}
Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)).
When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
When formatting, rows are enclosed in double-quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double-quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first, the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double-quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
```bash
$ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
@ -345,12 +345,12 @@ The CSV format supports the output of totals and extremes the same way as `TabSe
Also prints the header row, similar to `TabSeparatedWithNames`.
## CustomSeparated { #format-customseparated}
## CustomSeparated {#format-customseparated}
Similar to [Template](#format-template), but it prints or reads all columns and uses escaping rule from setting `format_custom_escaping_rule` and delimiters from settings `format_custom_field_delimiter`, `format_custom_row_before_delimiter`, `format_custom_row_after_delimiter`, `format_custom_row_between_delimiter`, `format_custom_result_before_delimiter` and `format_custom_result_after_delimiter`, not from format strings.
There is also `CustomSeparatedIgnoreSpaces` format, which is similar to `TemplateIgnoreSpaces`.
## JSON { #json}
## JSON {#json}
Outputs data in JSON format. Besides data tables, it also outputs column names and types, along with some additional information: the total number of output rows, and the number of rows that could have been output if there weren't a LIMIT. Example:
@ -422,7 +422,7 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA
}
```
The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character <20> so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double quotes by default. To remove the quotes, you can set the configuration parameter [output_format_json_quote_64bit_integers](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) to 0.
The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character <20> so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double-quotes by default. To remove the quotes, you can set the configuration parameter [output_format_json_quote_64bit_integers](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) to 0.
`rows` The total number of output rows.
@ -431,7 +431,7 @@ If the query contains GROUP BY, rows_before_limit_at_least is the exact number o
`totals` Total values (when using WITH TOTALS).
`extremes` Extreme values (when extremes is set to 1).
`extremes` Extreme values (when extremes are set to 1).
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
@ -439,7 +439,7 @@ ClickHouse supports [NULL](../query_language/syntax.md), which is displayed as `
See also the [JSONEachRow](#jsoneachrow) format.
## JSONCompact { #jsoncompact}
## JSONCompact {#jsoncompact}
Differs from JSON only in that data rows are output in arrays, not in objects.
@ -485,7 +485,7 @@ Example:
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
See also the `JSONEachRow` format.
## JSONEachRow { #jsoneachrow}
## JSONEachRow {#jsoneachrow}
When using this format, ClickHouse outputs rows as separated, newline-delimited JSON objects, but the data as a whole is not valid JSON.
@ -555,7 +555,7 @@ Unlike the [JSON](#json) format, there is no substitution of invalid UTF-8 seque
!!! note "Note"
Any set of bytes can be output in the strings. Use the `JSONEachRow` format if you are sure that the data in the table can be formatted as JSON without losing any information.
### Usage of Nested Structures { #jsoneachrow-nested}
### Usage of Nested Structures {#jsoneachrow-nested}
If you have a table with [Nested](../data_types/nested_data_structures/nested.md) data type columns, you can insert JSON data with the same structure. Enable this feature with the [input_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) setting.
@ -609,20 +609,20 @@ SELECT * FROM json_each_row_nested
└───────────────┴────────┘
```
## Native { #native}
## Native {#native}
The most efficient format. Data is written and read by blocks in binary format. For each block, the number of rows, number of columns, column names and types, and parts of columns in this block are recorded one after another. In other words, this format is "columnar" it doesn't convert columns to rows. This is the format used in the native interface for interaction between servers, for using the command-line client, and for C++ clients.
You can use this format to quickly generate dumps that can only be read by the ClickHouse DBMS. It doesn't make sense to work with this format yourself.
## Null { #null}
## Null {#null}
Nothing is output. However, the query is processed, and when using the command-line client, data is transmitted to the client. This is used for tests, including productivity testing.
Nothing is output. However, the query is processed, and when using the command-line client, data is transmitted to the client. This is used for tests, including performance testing.
Obviously, this format is only appropriate for output, not for parsing.
## Pretty { #pretty}
## Pretty {#pretty}
Outputs data as Unicode-art tables, also using ANSI-escape sequences for setting colors in the terminal.
Outputs data as Unicode-art tables, also using ANSI-escape sequences for setting colours in the terminal.
A full grid of the table is drawn, and each row occupies two lines in the terminal.
Each result block is output as a separate table. This is necessary so that blocks can be output without buffering results (buffering would be necessary in order to pre-calculate the visible width of all the values).
@ -648,7 +648,7 @@ SELECT 'String with \'quotes\' and \t character' AS Escaping_test
```text
┌─Escaping_test────────────────────────┐
│ String with 'quotes' and character │
│ String with 'quotes' and character │
└──────────────────────────────────────┘
```
@ -684,16 +684,16 @@ Extremes:
└────────────┴─────────┘
```
## PrettyCompact { #prettycompact}
## PrettyCompact {#prettycompact}
Differs from [Pretty](#pretty) in that the grid is drawn between rows and the result is more compact.
This format is used by default in the command-line client in interactive mode.
## PrettyCompactMonoBlock { #prettycompactmonoblock}
## PrettyCompactMonoBlock {#prettycompactmonoblock}
Differs from [PrettyCompact](#prettycompact) in that up to 10,000 rows are buffered, then output as a single table, not by blocks.
## PrettyNoEscapes { #prettynoescapes}
## PrettyNoEscapes {#prettynoescapes}
Differs from Pretty in that ANSI-escape sequences aren't used. This is necessary for displaying this format in a browser, as well as for using the 'watch' command-line utility.
@ -713,16 +713,16 @@ The same as the previous setting.
The same as the previous setting.
## PrettySpace { #prettyspace}
## PrettySpace {#prettyspace}
Differs from [PrettyCompact](#prettycompact) in that whitespace (space characters) is used instead of the grid.
## RowBinary { #rowbinary}
## RowBinary {#rowbinary}
Formats and parses data by row in binary format. Rows and values are listed consecutively, without separators.
This format is less efficient than the Native format, since it is row-based.
This format is less efficient than the Native format since it is row-based.
Integers use fixed-length little endian representation. For example, UInt64 uses 8 bytes.
Integers use fixed-length little-endian representation. For example, UInt64 uses 8 bytes.
DateTime is represented as UInt32 containing the Unix timestamp as the value.
Date is represented as a UInt16 object that contains the number of days since 1970-01-01 as the value.
String is represented as a varint length (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128)), followed by the bytes of the string.
@ -732,7 +732,7 @@ Array is represented as a varint length (unsigned [LEB128](https://en.wikipedia.
For [NULL](../query_language/syntax.md#null-literal) support, an additional byte containing 1 or 0 is added before each [Nullable](../data_types/nullable.md) value. If 1, then the value is `NULL` and this byte is interpreted as a separate value. If 0, the value after the byte is not `NULL`.
## RowBinaryWithNamesAndTypes { #rowbinarywithnamesandtypes}
## RowBinaryWithNamesAndTypes {#rowbinarywithnamesandtypes}
Similar to [RowBinary](#rowbinary), but with added header:
@ -740,9 +740,9 @@ Similar to [RowBinary](#rowbinary), but with added header:
* N `String`s specifying column names
* N `String`s specifying column types
## Values { #data-format-values}
## Values {#data-format-values}
Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren't inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../query_language/syntax.md) is represented as `NULL`.
Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren't inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../query_language/syntax.md) is represented as `NULL`.
The minimum set of characters that you need to escape when passing data in Values format: single quotes and backslashes.
@ -750,9 +750,9 @@ This is the format that is used in `INSERT INTO t VALUES ...`, but you can also
See also: [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) and [input_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) settings.
## Vertical { #vertical}
## Vertical {#vertical}
Prints each value on a separate line with the column name specified. This format is convenient for printing just one or a few rows, if each row consists of a large number of columns.
Prints each value on a separate line with the column name specified. This format is convenient for printing just one or a few rows if each row consists of a large number of columns.
[NULL](../query_language/syntax.md) is output as `ᴺᵁᴸᴸ`.
@ -777,17 +777,17 @@ SELECT 'string with \'quotes\' and \t with some special \n characters' AS test F
```text
Row 1:
──────
test: string with 'quotes' and with some special
test: string with 'quotes' and with some special
characters
```
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
## VerticalRaw { #verticalraw}
## VerticalRaw {#verticalraw}
Similar to [Vertical](#vertical), but with escaping disabled. This format is only suitable for outputting query results, not for parsing (receiving data and inserting it in the table).
## XML { #xml}
## XML {#xml}
XML format is suitable only for output, not for parsing. Example:
@ -860,7 +860,7 @@ In string values, the characters `<` and `&` are escaped as `<` and `&`.
Arrays are output as `<array><elem>Hello</elem><elem>World</elem>...</array>`,and tuples as `<tuple><elem>Hello</elem><elem>World</elem>...</tuple>`.
## CapnProto { #capnproto}
## CapnProto {#capnproto}
Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, but not like JSON or MessagePack.
@ -883,7 +883,7 @@ Deserialization is effective and usually doesn't increase the system load.
See also [Format Schema](#formatschema).
## Protobuf { #protobuf}
## Protobuf {#protobuf}
Protobuf - is a [Protocol Buffers](https://developers.google.com/protocol-buffers/) format.
@ -950,7 +950,7 @@ ClickHouse inputs and outputs protobuf messages in the `length-delimited` format
It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints).
See also [how to read/write length-delimited protobuf messages in popular languages](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages).
## Avro { #data-format-avro}
## Avro {#data-format-avro}
[Apache Avro](http://avro.apache.org/) is a row-oriented data serialization framework developed within Apache's Hadoop project.
@ -997,7 +997,7 @@ The root schema of input Avro file must be of `record` type.
To find the correspondence between table columns and fields of Avro schema ClickHouse compares their names. This comparison is case-sensitive.
Unused fields are skipped.
Data types of a ClickHouse table columns can differ from the corresponding fields of the Avro data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to corresponding column type.
Data types of ClickHouse table columns can differ from the corresponding fields of the Avro data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [casts](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to corresponding column type.
### Selecting Data
@ -1014,7 +1014,7 @@ Column names must:
Output Avro file compression and sync interval can be configured with [output_format_avro_codec](../operations/settings/settings.md#settings-output_format_avro_codec) and [output_format_avro_sync_interval](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectively.
## AvroConfluent { #data-format-avro-confluent}
## AvroConfluent {#data-format-avro-confluent}
AvroConfluent supports decoding single-object Avro messages commonly used with [Kafka](https://kafka.apache.org/) and [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html).
@ -1062,7 +1062,7 @@ SELECT * FROM topic1_stream;
Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` to maintain it's value after a restart.
## Parquet { #data-format-parquet}
## Parquet {#data-format-parquet}
[Apache Parquet](http://parquet.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. ClickHouse supports read and write operations for this format.
@ -1092,7 +1092,7 @@ ClickHouse supports configurable precision of `Decimal` type. The `INSERT` query
Unsupported Parquet data types: `DATE32`, `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`.
Data types of a ClickHouse table columns can differ from the corresponding fields of the Parquet data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [cast](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to that data type which is set for the ClickHouse table column.
Data types of ClickHouse table columns can differ from the corresponding fields of the Parquet data inserted. When inserting data, ClickHouse interprets data types according to the table above and then [cast](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) the data to that data type which is set for the ClickHouse table column.
### Inserting and Selecting Data
@ -1110,7 +1110,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_
To exchange data with Hadoop, you can use [HDFS table engine](../operations/table_engines/hdfs.md).
## ORC { #data-format-orc}
## ORC {#data-format-orc}
[Apache ORC](https://orc.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. You can only insert data in this format to ClickHouse.
@ -1151,14 +1151,14 @@ $ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT OR
To exchange data with Hadoop, you can use [HDFS table engine](../operations/table_engines/hdfs.md).
## Format Schema { #formatschema}
## Format Schema {#formatschema}
The file name containing the format schema is set by the setting `format_schema`.
It's required to set this setting when it is used one of the formats `Cap'n Proto` and `Protobuf`.
The format schema is a combination of a file name and the name of a message type in this file, delimited by colon,
The format schema is a combination of a file name and the name of a message type in this file, delimited by a colon,
e.g. `schemafile.proto:MessageType`.
If the file has the standard extension for the format (for example, `.proto` for `Protobuf`),
it can be omitted and in this case the format schema looks like `schemafile:MessageType`.
it can be omitted and in this case, the format schema looks like `schemafile:MessageType`.
If you input or output data via the [client](../interfaces/cli.md) in the [interactive mode](../interfaces/cli.md#cli_usage), the file name specified in the format schema
can contain an absolute path or a path relative to the current directory on the client.
@ -1168,12 +1168,12 @@ If you input or output data via the [HTTP interface](../interfaces/http.md) the
should be located in the directory specified in [format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path)
in the server configuration.
[Original article](https://clickhouse.tech/docs/en/interfaces/formats/) <!--hide-->
## Skipping Errors { #skippingerrors}
## Skipping Errors {#skippingerrors}
Some formats such as `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` and `Protobuf` can skip broken row if parsing error occurred and continue parsing from the beginning of next row. See [input_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) and
[input_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) settings.
Limitations:
- In case of parsing error `JSONEachRow` skips all data until the new line (or EOF), so rows must be delimited by `\n` to count errors correctly.
- `Template` and `CustomSeparated` use delimiter after the last column and delimiter between rows to find the beginning of next row, so skipping errors works only if at least one of them is not empty.
[Original article](https://clickhouse.tech/docs/en/interfaces/formats/) <!--hide-->

View File

@ -1,4 +1,4 @@
# HTTP Interface { #http_interface}
# HTTP Interface {#http_interface}
The HTTP interface lets you use ClickHouse on any platform from any programming language. We use it for working from Java and Perl, as well as shell scripts. In other departments, the HTTP interface is used from Perl, Python, and Go. The HTTP interface is more limited than the native interface, but it has better compatibility.
@ -10,7 +10,7 @@ $ curl 'http://localhost:8123/'
Ok.
```
Use GET /ping request in health-check scripts. This handler always return "Ok." (with a line feed at the end). Available from version 18.12.13.
Use GET /ping request in health-check scripts. This handler always returns "Ok." (with a line feed at the end). Available from version 18.12.13.
```bash
$ curl 'http://localhost:8123/ping'
Ok.
@ -22,7 +22,7 @@ Send the request as a URL 'query' parameter, or as a POST. Or send the beginning
If successful, you receive the 200 response code and the result in the response body.
If an error occurs, you receive the 500 response code and an error description text in the response body.
When using the GET method, 'readonly' is set. In other words, for queries that modify data, you can only use the POST method. You can send the query itself either in the POST body, or in the URL parameter.
When using the GET method, 'readonly' is set. In other words, for queries that modify data, you can only use the POST method. You can send the query itself either in the POST body or in the URL parameter.
Examples:
@ -200,7 +200,7 @@ $ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: pass
```
If the user name is not specified, the `default` name is used. If the password is not specified, the empty password is used.
You can also use the URL parameters to specify any settings for processing a single query, or entire profiles of settings. Example:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1
You can also use the URL parameters to specify any settings for processing a single query or entire profiles of settings. Example:http://localhost:8123/?profile=web&max_rows_to_read=1000000000&query=SELECT+1
For more information, see the [Settings](../operations/settings/index.md) section.
@ -238,7 +238,7 @@ Possible header fields:
- `written_rows` — Number of rows written.
- `written_bytes` — Volume of data written in bytes.
Running requests don't stop automatically if the HTTP connection is lost. Parsing and data formatting are performed on the server side, and using the network might be ineffective.
Running requests don't stop automatically if the HTTP connection is lost. Parsing and data formatting are performed on the server-side, and using the network might be ineffective.
The optional 'query_id' parameter can be passed as the query ID (any string). For more information, see the section "Settings, replace_running_query".
The optional 'quota_key' parameter can be passed as the quota key (any string). For more information, see the section "Quotas".
@ -247,9 +247,9 @@ The HTTP interface allows passing external data (external temporary tables) for
## Response Buffering
You can enable response buffering on the server side. The `buffer_size` and `wait_end_of_query` URL parameters are provided for this purpose.
You can enable response buffering on the server-side. The `buffer_size` and `wait_end_of_query` URL parameters are provided for this purpose.
`buffer_size` determines the number of bytes in the result to buffer in the server memory. If the result body is larger than this threshold, the buffer is written to the HTTP channel, and the remaining data is sent directly to the HTTP channel.
`buffer_size` determines the number of bytes in the result to buffer in the server memory. If a result body is larger than this threshold, the buffer is written to the HTTP channel, and the remaining data is sent directly to the HTTP channel.
To ensure that the entire response is buffered, set `wait_end_of_query=1`. In this case, the data that is not stored in memory will be buffered in a temporary server file.
@ -259,9 +259,9 @@ Example:
$ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wait_end_of_query=1' -d 'SELECT toUInt8(number) FROM system.numbers LIMIT 9000000 FORMAT RowBinary'
```
Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client side, the error can only be detected at the parsing stage.
Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client-side, the error can only be detected at the parsing stage.
### Queries with Parameters { #cli-queries-with-parameters}
### Queries with Parameters {#cli-queries-with-parameters}
You can create a query with parameters and pass values for them from the corresponding HTTP request parameters. For more information, see [Queries with Parameters for CLI](cli.md#cli-queries-with-parameters).

View File

@ -1,4 +1,4 @@
# Interfaces { #interfaces}
# Interfaces {#interfaces}
ClickHouse provides two network interfaces (both can be optionally wrapped in TLS for additional security):

View File

@ -1,16 +1,16 @@
# MySQL interface { #mysql_interface}
# MySQL interface {#mysql_interface}
ClickHouse supports MySQL wire protocol. It can be enabled by [mysql_port](../operations/server_settings/settings.md#server_settings-mysql_port) setting in configuration file:
```xml
<mysql_port>9004</mysql_port>
```
Example of connecting using command-line tool mysql:
Example of connecting using command-line tool `mysql`:
```bash
$ mysql --protocol tcp -u default -P 9004
```
Output if connection succeeded:
Output if a connection succeeded:
```text
Welcome to the MySQL monitor. Commands end with ; or \g.
Your MySQL connection id is 4
@ -35,3 +35,5 @@ Restrictions:
- prepared queries are not supported
- some data types are sent as strings
[Original article](https://clickhouse.tech/docs/en/interfaces/mysql/) <!--hide-->

View File

@ -1,5 +1,5 @@
# Native Interface (TCP)
The native protocol is used in the [command-line client](cli.md), for interserver communication during distributed query processing, and also in other C++ programs. Unfortunately, native ClickHouse protocol does not have formal specification yet, but it can be reverse engineered from ClickHouse source code (starting [around here](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/src/Client)) and/or by intercepting and analyzing TCP traffic.
The native protocol is used in the [command-line client](cli.md), for inter-server communication during distributed query processing, and also in other C++ programs. Unfortunately, native ClickHouse protocol does not have formal specification yet, but it can be reverse-engineered from ClickHouse source code (starting [around here](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/src/Client)) and/or by intercepting and analyzing TCP traffic.
[Original article](https://clickhouse.tech/docs/en/interfaces/tcp/) <!--hide-->

View File

@ -12,7 +12,7 @@ Features:
- Query editor with syntax highlighting.
- Auto-completion of commands.
- Tools for graphical analysis of query execution.
- Color scheme options.
- Colour scheme options.
[Tabix documentation](https://tabix.io/doc/).
@ -26,7 +26,7 @@ Features:
- Export query results as CSV or JSON.
- List of processes with descriptions. Write mode. Ability to stop (`KILL`) a process.
- Database graph. Shows all tables and their columns with additional information.
- Quick view of the column size.
- A quick view of the column size.
- Server configuration.
The following features are planned for development:
@ -69,11 +69,11 @@ Features:
- Query development with syntax highlight and autocompletion.
- Table list with filters and metadata search.
- Table data preview.
- Full text search.
- Full-text search.
### clickhouse-cli
[clickhouse-cli](https://github.com/hatarist/clickhouse-cli) is an alternative command line client for ClickHouse, written in Python 3.
[clickhouse-cli](https://github.com/hatarist/clickhouse-cli) is an alternative command-line client for ClickHouse, written in Python 3.
Features:
@ -90,18 +90,18 @@ Features:
### DataGrip
[DataGrip](https://www.jetbrains.com/datagrip/) is a database IDE from JetBrains with dedicated support for ClickHouse. It is also embedded into other IntelliJ-based tools: PyCharm, IntelliJ IDEA, GoLand, PhpStorm and others.
[DataGrip](https://www.jetbrains.com/datagrip/) is a database IDE from JetBrains with dedicated support for ClickHouse. It is also embedded in other IntelliJ-based tools: PyCharm, IntelliJ IDEA, GoLand, PhpStorm and others.
Features:
- Very fast code completion.
- ClickHouse syntax highlighting.
- Support for features specific to ClickHouse, for example nested columns, table engines.
- Support for features specific to ClickHouse, for example, nested columns, table engines.
- Data Editor.
- Refactorings.
- Search and Navigation.
### Yandex DataLens { #yandex-datalens}
### Yandex DataLens {#yandex-datalens}
[Yandex DataLens](https://cloud.yandex.ru/services/datalens) is a service of data visualization and analytics.
@ -127,7 +127,7 @@ Features:
- SQL editor with visualizations, version control, auto-completion, reusable query components and dynamic filters.
- Embedded analytics of reports and dashboards via iframe.
- Data preparation and ETL capabilities.
- SQL data modeling support for relational mapping of data.
- SQL data modelling support for relational mapping of data.
### Looker
[Looker](https://looker.com) is a data platform and business intelligence tool with support for 50+ database dialects including ClickHouse. Looker is available as a SaaS platform and self-hosted. Users can use Looker via the browser to explore data, build visualizations and dashboards, schedule reports, and share their insights with colleagues. Looker provides a rich set of tools to embed these features in other applications, and an API
@ -135,8 +135,8 @@ to integrate data with other applications.
Features:
- Easy and agile development using LookML, a language which supports currated
[Data Modeling](https://looker.com/platform/data-modeling) to support report writers and end users.
- Easy and agile development using LookML, a language which supports curated
[Data Modeling](https://looker.com/platform/data-modeling) to support report writers and end-users.
- Powerful workflow integration via Looker's [Data Actions](https://looker.com/platform/actions).

View File

@ -2,13 +2,13 @@
## chproxy
[chproxy](https://github.com/Vertamedia/chproxy), is an http proxy and load balancer for ClickHouse database.
[chproxy](https://github.com/Vertamedia/chproxy), is an HTTP proxy and load balancer for ClickHouse database.
Features:
* Per-user routing and response caching.
* Flexible limits.
* Automatic SSL cerificate renewal.
* Automatic SSL certificate renewal.
Implemented in Go.

View File

@ -1,7 +1,7 @@
# ClickHouse Adopters
!!! warning "Disclaimer"
The following list of companies using ClickHouse and their success stories is assembled from public sources, thus might differ from current reality. We'd really appreciate if you share the story of adopting ClickHouse in your company and [add it to the list](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md), but please make sure you won't have any NDA issues by doing so. Providing updates with publications by other companies is also useful.
The following list of companies using ClickHouse and their success stories is assembled from public sources, thus might differ from current reality. We'd really appreciate if you share the story of adopting ClickHouse in your company and [add it to the list](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md), but please make sure you won't have any NDA issues by doing so. Providing updates with publications from other companies is also useful.
| Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size<abbr title="of single replica"><sup>*</sup></abbr> | Reference |
| --- | --- | --- | --- | --- | --- |

View File

@ -2,6 +2,6 @@
1. No full-fledged transactions.
2. Lack of ability to modify or delete already inserted data with high rate and low latency. There are batch deletes and updates available to clean up or modify data, for example to comply with [GDPR](https://gdpr-info.eu).
3. The sparse index makes ClickHouse not really suitable for point queries retrieving single rows by their keys.
3. The sparse index makes ClickHouse not so suitable for point queries retrieving single rows by their keys.
[Original article](https://clickhouse.tech/docs/en/introduction/features_considered_disadvantages/) <!--hide-->

View File

@ -2,9 +2,9 @@
ClickHouse was originally developed to power [Yandex.Metrica](https://metrica.yandex.com/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all), and continues to be the core component of this system. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows generating custom reports on the fly directly from non-aggregated data. This article briefly covers the goals of ClickHouse in the early stages of its development.
Yandex.Metrica builds customized reports on the fly based on hits and sessions, with arbitrary segments defined by the user. This often requires building complex aggregates, such as the number of unique users. New data for building a report is received in real time.
Yandex.Metrica builds customized reports on the fly based on hits and sessions, with arbitrary segments defined by the user. This often requires building complex aggregates, such as the number of unique users. New data for building a report is received in real-time.
As of April 2014, Yandex.Metrica was tracking about 12 billion events (page views and clicks) daily. All these events must be stored in order to build custom reports. A single query may require scanning millions of rows within a few hundred milliseconds, or hundreds of millions of rows in just a few seconds.
As of April 2014, Yandex.Metrica was tracking about 12 billion events (page views and clicks) daily. All these events must be stored to build custom reports. A single query may require scanning millions of rows within a few hundred milliseconds, or hundreds of millions of rows in just a few seconds.
## Usage in Yandex.Metrica and Other Yandex Services
@ -23,20 +23,20 @@ ClickHouse has at least a dozen installations in other Yandex services: in searc
## Aggregated and Non-aggregated Data
There is a popular opinion that in order to effectively calculate statistics, you must aggregate data, since this reduces the volume of data.
There is a popular opinion that to effectively calculate statistics, you must aggregate data since this reduces the volume of data.
But data aggregation is a very limited solution, for the following reasons:
- You must have a pre-defined list of reports the user will need.
- The user can't make custom reports.
- When aggregating a large quantity of keys, the volume of data is not reduced, and aggregation is useless.
- When aggregating a large number of keys, the volume of data is not reduced, and aggregation is useless.
- For a large number of reports, there are too many aggregation variations (combinatorial explosion).
- When aggregating keys with high cardinality (such as URLs), the volume of data is not reduced by much (less than twofold).
- For this reason, the volume of data with aggregation might grow instead of shrink.
- Users do not view all the reports we generate for them. A large portion of calculations are useless.
- Users do not view all the reports we generate for them. A large portion of those calculations is useless.
- The logical integrity of data may be violated for various aggregations.
If we do not aggregate anything and work with non-aggregated data, this might actually reduce the volume of calculations.
If we do not aggregate anything and work with non-aggregated data, this might reduce the volume of calculations.
However, with aggregation, a significant part of the work is taken offline and completed relatively calmly. In contrast, online calculations require calculating as fast as possible, since the user is waiting for the result.

View File

@ -1,4 +1,4 @@
# Configuration Files { #configuration_files}
# Configuration Files {#configuration_files}
ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml`. Other files must be in the `/etc/clickhouse-server/config.d` directory.

View File

@ -1,25 +1,25 @@
# Sampling Query Profiler
ClickHouse runs sampling profiler that allows to analyze query execution. Using profiler you can find source code routines that used the most frequently during a query execution. You can trace CPU time and wall-clock time spent including idle time.
ClickHouse runs sampling profiler that allows analyzing query execution. Using profiler you can find source code routines that used the most frequently during query execution. You can trace CPU time and wall-clock time spent including idle time.
To use profiler:
- Setup the [trace_log](../server_settings/settings.md#server_settings-trace_log) section of the server configuration.
This section configures the [trace_log](../system_tables.md#system_tables-trace_log) system table containing the results of the profiler functioning. It is configured by default. Remember that data in this table is valid only for running server. After the server restart, ClickHouse doesn't clean up the table and all the stored virtual memory address may become invalid.
This section configures the [trace_log](../system_tables.md#system_tables-trace_log) system table containing the results of the profiler functioning. It is configured by default. Remember that data in this table is valid only for a running server. After the server restart, ClickHouse doesn't clean up the table and all the stored virtual memory address may become invalid.
- Setup the [query_profiler_cpu_time_period_ns](../settings/settings.md#query_profiler_cpu_time_period_ns) or [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) settings. Both settings can be used simultaneously.
These settings allow you to configure profiler timers. As these are the session settings, you can get different sampling frequency for the whole server, individual users or user profiles, for your interactive session, and for each individual query.
Default sampling frequency is one sample per second and both CPU and real timers are enabled. This frequency allows to collect enough information about ClickHouse cluster. At the same time, working with this frequency, profiler doesn't affect ClickHouse server's performance. If you need to profile each individual query try to use higher sampling frequency.
The default sampling frequency is one sample per second and both CPU and real timers are enabled. This frequency allows collecting enough information about ClickHouse cluster. At the same time, working with this frequency, profiler doesn't affect ClickHouse server's performance. If you need to profile each individual query try to use higher sampling frequency.
To analyze the `trace_log` system table:
- Install the `clickhouse-common-static-dbg` package. See [Install from DEB Packages](../../getting_started/install.md#install-from-deb-packages).
- Allow introspection functions by the [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) setting.
For security reasons introspection functions are disabled by default.
For security reasons, introspection functions are disabled by default.
- Use the `addressToLine`, `addressToSymbol` and `demangle` [introspection functions](../../query_language/functions/introspection.md) to get function names and their positions in ClickHouse code. To get a profile for some query, you need to aggregate data from the `trace_log` table. You can aggregate data by individual functions or by the whole stack traces.
@ -30,7 +30,7 @@ If you need to visualize `trace_log` info, try [flamegraph](../../interfaces/thi
In this example we:
- Filtering `trace_log` data by a query identifier and current date.
- Filtering `trace_log` data by a query identifier and the current date.
- Aggregating by stack trace.
- Using introspection functions, we will get a report of:

View File

@ -2,13 +2,13 @@
With this instruction you can run basic ClickHouse performance test on any server without installation of ClickHouse packages.
\1. Go to "commits" page: [https://github.com/ClickHouse/ClickHouse/commits/master](https://github.com/ClickHouse/ClickHouse/commits/master)
1. Go to "commits" page: https://github.com/ClickHouse/ClickHouse/commits/master
\2. Click on the first green check mark or red cross with green "ClickHouse Build Check" and click on the "Details" link near "ClickHouse Build Check".
2. Click on the first green check mark or red cross with green "ClickHouse Build Check" and click on the "Details" link near "ClickHouse Build Check".
\3. Copy the link to "clickhouse" binary for amd64 or aarch64.
3. Copy the link to "clickhouse" binary for amd64 or aarch64.
\4. ssh to the server and download it with wget:
4. ssh to the server and download it with wget:
```
# For amd64:
wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578163263_binary/clickhouse
@ -18,7 +18,7 @@ wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f7907
chmod a+x clickhouse
```
\5. Download configs:
5. Download configs:
```
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/dbms/programs/server/config.xml
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/dbms/programs/server/users.xml
@ -27,20 +27,14 @@ wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/dbms/program
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/dbms/programs/server/config.d/log_to_console.xml -O config.d/log_to_console.xml
```
\6. Download benchmark files:
6. Download benchmark files:
```
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/dbms/benchmark/clickhouse/benchmark-new.sh
chmod a+x benchmark-new.sh
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/dbms/benchmark/clickhouse/queries.sql
```
\7. Download test data:
According to the instruction:
[https://clickhouse.tech/docs/en/getting_started/example_datasets/metrica/](https://clickhouse.yandex/docs/en/getting_started/example_datasets/metrica/)
("hits" table containing 100 million rows)
7. Download test data according to the [Yandex.Metrica dataset](../getting_started/example_datasets/metrica.md) instruction ("hits" table containing 100 million rows).
```
wget https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz
@ -48,29 +42,27 @@ tar xvf hits_100m_obfuscated_v1.tar.xz -C .
mv hits_100m_obfuscated_v1/* .
```
\8. Run the server:
8. Run the server:
```
./clickhouse server
```
\9. Check the data:
ssh to the server in another terminal
9. Check the data: ssh to the server in another terminal
```
./clickhouse client --query "SELECT count() FROM hits_100m_obfuscated"
100000000
```
\10. Edit the benchmark-new.sh, change "clickhouse-client" to "./clickhouse client" and add "--max_memory_usage 100000000000" parameter.
10. Edit the benchmark-new.sh, change "clickhouse-client" to "./clickhouse client" and add "--max_memory_usage 100000000000" parameter.
```
mcedit benchmark-new.sh
```
\11. Run the benchmark:
11. Run the benchmark:
```
./benchmark-new.sh hits_100m_obfuscated
```
\12. Send the numbers and the info about your hardware configuration to clickhouse-feedback@yandex-team.com
12. Send the numbers and the info about your hardware configuration to clickhouse-feedback@yandex-team.com
All the results are published here: [https://clickhouse.tech/benchmark_hardware.html](https://clickhouse.yandex/benchmark_hardware.html)
All the results are published here: https://clickhouse.tech/benchmark_hardware.html

View File

@ -1,4 +1,4 @@
# Quotas { #quotas}
# Quotas {#quotas}
Quotas allow you to limit resource usage over a period of time, or simply track the use of resources.
Quotas are set up in the user config. This is usually 'users.xml'.

View File

@ -1,4 +1,4 @@
# Server configuration parameters { #server_settings}
# Server configuration parameters {#server_settings}
This section contains descriptions of server settings that cannot be changed at the session or query level.

View File

@ -1,5 +1,4 @@
# Server settings
# Server Settings
## builtin_dictionaries_reload_interval
@ -16,7 +15,7 @@ Default value: 3600.
```
## compression { #server-settings-compression}
## compression {#server-settings-compression}
Data compression settings for [MergeTree](../table_engines/mergetree.md)-engine tables.
@ -90,7 +89,7 @@ Settings profiles are located in the file specified in the parameter `user_confi
```
## dictionaries_config { #server_settings-dictionaries_config}
## dictionaries_config {#server_settings-dictionaries_config}
The path to the config file for external dictionaries.
@ -108,7 +107,7 @@ See also "[External dictionaries](../../query_language/dicts/external_dicts.md)"
```
## dictionaries_lazy_load { #server_settings-dictionaries_lazy_load}
## dictionaries_lazy_load {#server_settings-dictionaries_lazy_load}
Lazy loading of dictionaries.
@ -125,7 +124,7 @@ The default is `true`.
```
## format_schema_path { #server_settings-format_schema_path}
## format_schema_path {#server_settings-format_schema_path}
The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#capnproto) format.
@ -136,7 +135,7 @@ The path to the directory with the schemes for the input data, such as schemas f
<format_schema_path>format_schemas/</format_schema_path>
```
## graphite { #server_settings-graphite}
## graphite {#server_settings-graphite}
Sending data to [Graphite](https://github.com/graphite-project).
@ -171,7 +170,7 @@ You can configure multiple `<graphite>` clauses. For instance, you can use this
```
## graphite_rollup { #server_settings-graphite_rollup}
## graphite_rollup {#server_settings-graphite_rollup}
Settings for thinning data for Graphite.
@ -206,7 +205,7 @@ The port for connecting to the server over HTTP(s).
If `https_port` is specified, [openSSL](#server_settings-openssl) must be configured.
If `http_port` is specified, the openSSL configuration is ignored even if it is set.
If `http_port` is specified, the OpenSSL configuration is ignored even if it is set.
**Example**
@ -215,10 +214,10 @@ If `http_port` is specified, the openSSL configuration is ignored even if it is
```
## http_server_default_response { #server_settings-http_server_default_response}
## http_server_default_response {#server_settings-http_server_default_response}
The page that is shown by default when you access the ClickHouse HTTP(s) server.
Default value is "Ok." (with a line feed at the end)
The default value is "Ok." (with a line feed at the end)
**Example**
@ -230,7 +229,7 @@ Opens `https://tabix.io/` when accessing ` http://localhost: http_port`.
</http_server_default_response>
```
## include_from { #server_settings-include_from}
## include_from {#server_settings-include_from}
The path to the file with substitutions.
@ -256,7 +255,7 @@ Port for exchanging data between ClickHouse servers.
## interserver_http_host
The host name that can be used by other servers to access this server.
The hostname that can be used by other servers to access this server.
If omitted, it is defined in the same way as the `hostname-f` command.
@ -268,7 +267,7 @@ Useful for breaking away from a specific network interface.
<interserver_http_host>example.yandex.ru</interserver_http_host>
```
## interserver_http_credentials { #server-settings-interserver_http_credentials}
## interserver_http_credentials {#server-settings-interserver_http_credentials}
The username and password used to authenticate during [replication](../table_engines/replication.md) with the Replicated* engines. These credentials are used only for communication between replicas and are unrelated to credentials for ClickHouse clients. The server is checking these credentials for connecting replicas and use the same credentials when connecting to other replicas. So, these credentials should be set the same for all replicas in a cluster.
By default, the authentication is not used.
@ -299,7 +298,7 @@ The number of seconds that ClickHouse waits for incoming requests before closing
```
## listen_host { #server_settings-listen_host}
## listen_host {#server_settings-listen_host}
Restriction on hosts that requests can come from. If you want the server to answer all of them, specify `::`.
@ -311,7 +310,7 @@ Examples:
```
## logger { #server_settings-logger}
## logger {#server_settings-logger}
Logging settings.
@ -374,7 +373,7 @@ For more information, see the section "[Creating replicated tables](../../operat
```
## mark_cache_size { #server-mark-cache-size}
## mark_cache_size {#server-mark-cache-size}
Approximate size (in bytes) of the cache of marks used by table engines of the [MergeTree](../table_engines/mergetree.md) family.
@ -415,7 +414,7 @@ The maximum number of open files.
By default: `maximum`.
We recommend using this option in Mac OS X, since the `getrlimit()` function returns an incorrect value.
We recommend using this option in Mac OS X since the `getrlimit()` function returns an incorrect value.
**Example**
@ -443,7 +442,7 @@ The value 0 means that you can delete all tables without any restrictions.
```
## merge_tree { #server_settings-merge_tree}
## merge_tree {#server_settings-merge_tree}
Fine tuning for tables in the [MergeTree](../table_engines/mergetree.md).
@ -458,7 +457,7 @@ For more information, see the MergeTreeSettings.h header file.
```
## openSSL { #server_settings-openssl}
## openSSL {#server_settings-openssl}
SSL client/server configuration.
@ -474,7 +473,7 @@ Keys for server/client settings:
- loadDefaultCAFile Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. |
- cipherList Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`.
- cacheSessions Enables or disables caching sessions. Must be used in combination with ``sessionIdContext``. Acceptable values: `true`, `false`.
- sessionIdContext A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed ``SSL_MAX_SSL_SESSION_ID_LENGTH``. This parameter is always recommended, since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: ``${application.name}``.
- sessionIdContext A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed ``SSL_MAX_SSL_SESSION_ID_LENGTH``. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: ``${application.name}``.
- sessionCacheSize The maximum number of sessions that the server caches. Default value: 1024\*20. 0 Unlimited sessions.
- sessionTimeout Time for caching the session on the server.
- extendedVerification Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`.
@ -483,7 +482,7 @@ Keys for server/client settings:
- requireTLSv1 Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
- fips Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS.
- privateKeyPassphraseHandler Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``<privateKeyPassphraseHandler>``, ``<name>KeyFileHandler</name>``, ``<options><password>test</password></options>``, ``</privateKeyPassphraseHandler>``.
- invalidCertificateHandler Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` <invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>`` .
- invalidCertificateHandler Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `` <invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>`` .
- disableProtocols Protocols that are not allowed to use.
- preferServerCiphers Preferred server ciphers on the client.
@ -518,7 +517,7 @@ Keys for server/client settings:
```
## part_log { #server_settings-part-log}
## part_log {#server_settings-part-log}
Logging events that are associated with [MergeTree](../table_engines/mergetree.md). For instance, adding or merging data. You can use the log to simulate merge algorithms and compare their characteristics. You can visualize the merge process.
@ -543,7 +542,7 @@ Use the following parameters to configure logging:
```
## path { #server_settings-path}
## path {#server_settings-path}
The path to the directory containing data.
@ -557,7 +556,7 @@ The path to the directory containing data.
```
## query_log { #server_settings-query-log}
## query_log {#server_settings-query-log}
Setting for logging queries received with the [log_queries=1](../settings/settings.md) setting.
@ -583,7 +582,7 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q
</query_log>
```
## query_thread_log { #server_settings-query-thread-log}
## query_thread_log {#server_settings-query-thread-log}
Setting for logging threads of queries received with the [log_query_threads=1](../settings/settings.md#settings-log-query-threads) setting.
@ -609,7 +608,7 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q
</query_thread_log>
```
## trace_log { #server_settings-trace_log}
## trace_log {#server_settings-trace_log}
Settings for the [trace_log](../system_tables.md#system_tables-trace_log) system table operation.
@ -634,9 +633,9 @@ The default server configuration file `config.xml` contains the following settin
## query_masking_rules
Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs,
`system.query_log`, `system.text_log`, `system.processes` table, and in logs sent to client. That allows preventing
sensitive data leakage from SQL queries (like names / emails / personal
identifiers / credit card numbers etc) to logs.
`system.query_log`, `system.text_log`, `system.processes` table, and in logs sent to the client. That allows preventing
sensitive data leakage from SQL queries (like names, emails, personal
identifiers or credit card numbers) to logs.
**Example**
@ -655,14 +654,14 @@ Config fields:
- `regexp` - RE2 compatible regular expression (mandatory)
- `replace` - substitution string for sensitive data (optional, by default - six asterisks)
The masking rules are applied on whole query (to prevent leaks of sensitive data from malformed / non parsable queries).
The masking rules are applied to the whole query (to prevent leaks of sensitive data from malformed / non-parsable queries).
`system.events` table have counter `QueryMaskingRulesMatch` which have overall number of query masking rules matches.
`system.events` table have counter `QueryMaskingRulesMatch` which have an overall number of query masking rules matches.
For distributed queries each server have to be configured separately, otherwise subquries passed to other
For distributed queries each server have to be configured separately, otherwise, subqueries passed to other
nodes will be stored without masking.
## remote_servers { #server_settings_remote_servers}
## remote_servers {#server_settings_remote_servers}
Configuration of clusters used by the [Distributed](../../operations/table_engines/distributed.md) table engine and by the `cluster` table function.
@ -678,13 +677,13 @@ For the value of the `incl` attribute, see the section "[Configuration files](..
- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards)
## timezone { #server_settings-timezone}
## timezone {#server_settings-timezone}
The server's time zone.
Specified as an IANA identifier for the UTC time zone or geographic location (for example, Africa/Abidjan).
Specified as an IANA identifier for the UTC timezone or geographic location (for example, Africa/Abidjan).
The time zone is necessary for conversions between String and DateTime formats when DateTime fields are output to text format (printed on the screen or in a file), and when getting DateTime from a string. In addition, the time zone is used in functions that work with the time and date if they didn't receive the time zone in the input parameters.
The time zone is necessary for conversions between String and DateTime formats when DateTime fields are output to text format (printed on the screen or in a file), and when getting DateTime from a string. Besides, the time zone is used in functions that work with the time and date if they didn't receive the time zone in the input parameters.
**Example**
@ -693,7 +692,7 @@ The time zone is necessary for conversions between String and DateTime formats w
```
## tcp_port { #server_settings-tcp_port}
## tcp_port {#server_settings-tcp_port}
Port for communicating with clients over the TCP protocol.
@ -703,7 +702,7 @@ Port for communicating with clients over the TCP protocol.
<tcp_port>9000</tcp_port>
```
## tcp_port_secure { #server_settings-tcp_port_secure}
## tcp_port_secure {#server_settings-tcp_port_secure}
TCP port for secure communication with clients. Use it with [OpenSSL](#server_settings-openssl) settings.
@ -717,7 +716,7 @@ Positive integer.
<tcp_port_secure>9440</tcp_port_secure>
```
## mysql_port { #server_settings-mysql_port}
## mysql_port {#server_settings-mysql_port}
Port for communicating with clients over MySQL protocol.
@ -731,7 +730,7 @@ Example
<mysql_port>9004</mysql_port>
```
## tmp_path { #server-settings-tmp_path}
## tmp_path {#server-settings-tmp_path}
Path to temporary data for processing large queries.
@ -745,7 +744,7 @@ Path to temporary data for processing large queries.
```
## tmp_policy { #server-settings-tmp_policy}
## tmp_policy {#server-settings-tmp_policy}
Policy from [`storage_configuration`](../table_engines/mergetree.md#table_engine-mergetree-multiple-volumes) to store temporary files.
If not set [`tmp_path`](#server-settings-tmp_path) is used, otherwise it is ignored.
@ -756,7 +755,7 @@ If not set [`tmp_path`](#server-settings-tmp_path) is used, otherwise it is igno
- `max_data_part_size_bytes` is ignored
- you must have exactly one volume in that policy
## uncompressed_cache_size { #server-settings-uncompressed_cache_size}
## uncompressed_cache_size {#server-settings-uncompressed_cache_size}
Cache size (in bytes) for uncompressed data used by table engines from the [MergeTree](../table_engines/mergetree.md).
@ -770,7 +769,7 @@ The uncompressed cache is advantageous for very short queries in individual case
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
```
## user_files_path { #server_settings-user_files_path}
## user_files_path {#server_settings-user_files_path}
The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md).
@ -797,7 +796,7 @@ Path to the file that contains:
```
## zookeeper { #server-settings_zookeeper}
## zookeeper {#server-settings_zookeeper}
Contains settings that allow ClickHouse to interact with a [ZooKeeper](http://zookeeper.apache.org/) cluster.
@ -848,7 +847,7 @@ This section contains the following parameters:
- [Replication](../../operations/table_engines/replication.md)
- [ZooKeeper Programmer's Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html)
## use_minimalistic_part_header_in_zookeeper { #server-settings-use_minimalistic_part_header_in_zookeeper}
## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper}
Storage method for data part headers in ZooKeeper.
@ -856,11 +855,11 @@ This setting only applies to the `MergeTree` family. It can be specified:
- Globally in the [merge_tree](#server_settings-merge_tree) section of the `config.xml` file.
ClickHouse uses the setting for all the tables on the server. You can change the setting at any time. Existing tables change their behavior when the setting changes.
ClickHouse uses the setting for all the tables on the server. You can change the setting at any time. Existing tables change their behaviour when the setting changes.
- For each individual table.
- For each table.
When creating a table, specify the corresponding [engine setting](../table_engines/mergetree.md#table_engine-mergetree-creating-a-table). The behavior of an existing table with this setting does not change, even if the global setting changes.
When creating a table, specify the corresponding [engine setting](../table_engines/mergetree.md#table_engine-mergetree-creating-a-table). The behaviour of an existing table with this setting does not change, even if the global setting changes.
**Possible values**
@ -876,14 +875,14 @@ If `use_minimalistic_part_header_in_zookeeper = 1`, then [replicated](../table_e
**Default value:** 0.
## disable_internal_dns_cache { #server-settings-disable_internal_dns_cache}
## disable_internal_dns_cache {#server-settings-disable_internal_dns_cache}
Disables the internal DNS cache. Recommended for operating ClickHouse in systems
with frequently changing infrastructure such as Kubernetes.
**Default value:** 0.
## dns_cache_update_period { #server-settings-dns_cache_update_period}
## dns_cache_update_period {#server-settings-dns_cache_update_period}
The period of updating IP addresses stored in the ClickHouse internal DNS cache (in seconds).
The update is performed asynchronously, in a separate system thread.

View File

@ -1,7 +1,7 @@
# Constraints on Settings
The constraints on settings can be defined in the `profiles` section of the `user.xml` configuration file and prohibit users from changing some of the settings with the `SET` query.
The constraints are defined as following:
The constraints are defined as the following:
```xml
<profiles>
@ -25,8 +25,8 @@ The constraints are defined as following:
</profiles>
```
If user tries to violate the constraints an exception is thrown and the setting isn't actually changed.
There are supported three types of constraints: `min`, `max`, `readonly`. The `min` and `max` constraints specify upper and lower boundaries for a numeric setting and can be used in combination. The `readonly` constraint specify that the user cannot change the corresponding setting at all.
If the user tries to violate the constraints an exception is thrown and the setting isn't changed.
There are supported three types of constraints: `min`, `max`, `readonly`. The `min` and `max` constraints specify upper and lower boundaries for a numeric setting and can be used in combination. The `readonly` constraint specifies that the user cannot change the corresponding setting at all.
**Example:** Let `users.xml` includes lines:
@ -63,6 +63,6 @@ Code: 452, e.displayText() = DB::Exception: Setting max_memory_usage should not
Code: 452, e.displayText() = DB::Exception: Setting force_index_by_date should not be changed.
```
**Note:** the `default` profile has a special handling: all the constraints defined for the `default` profile become the default constraints, so they restrict all the users until they're overriden explicitly for these users.
**Note:** the `default` profile has special handling: all the constraints defined for the `default` profile become the default constraints, so they restrict all the users until they're overridden explicitly for these users.
[Original article](https://clickhouse.tech/docs/en/operations/settings/constraints_on_settings/) <!--hide-->

View File

@ -1,4 +1,4 @@
# Settings { #settings}
# Settings {#settings}
There are multiple ways to make all the settings described below.
Settings are configured in layers, so each subsequent layer redefines the previous settings.

View File

@ -1,12 +1,12 @@
# Permissions for queries { #permissions_for_queries}
# Permissions for Queries {#permissions_for_queries}
Queries in ClickHouse can be divided into several types:
1. Read data queries: `SELECT`, `SHOW`, `DESCRIBE`, `EXISTS`.
1. Write data queries: `INSERT`, `OPTIMIZE`.
1. Change settings queries: `SET`, `USE`.
1. [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries: `CREATE`, `ALTER`, `RENAME`, `ATTACH`, `DETACH`, `DROP` `TRUNCATE`.
1. `KILL QUERY`.
2. Write data queries: `INSERT`, `OPTIMIZE`.
3. Change settings query: `SET`, `USE`.
4. [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries: `CREATE`, `ALTER`, `RENAME`, `ATTACH`, `DETACH`, `DROP` `TRUNCATE`.
5. `KILL QUERY`.
The following settings regulate user permissions by the type of query:
@ -15,9 +15,9 @@ The following settings regulate user permissions by the type of query:
`KILL QUERY` can be performed with any settings.
## readonly { #settings_readonly}
## readonly {#settings_readonly}
Restricts permissions for read data, write data and change settings queries.
Restricts permissions for reading data, write data and change settings queries.
See how the queries are divided into types [above](#permissions_for_queries).
@ -36,7 +36,7 @@ from changing only specific settings, for details see [constraints on settings](
Default value: 0
## allow_ddl { #settings_allow_ddl}
## allow_ddl {#settings_allow_ddl}
Allows or denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries.

View File

@ -1,10 +1,10 @@
# Restrictions on query complexity
# Restrictions on Query Complexity
Restrictions on query complexity are part of the settings.
They are used in order to provide safer execution from the user interface.
They are used to provide safer execution from the user interface.
Almost all the restrictions only apply to `SELECT`. For distributed query processing, restrictions are applied on each server separately.
ClickHouse checks the restrictions for data parts, not for each row. It means that you can exceed the value of restriction with a size of the data part.
ClickHouse checks the restrictions for data parts, not for each row. It means that you can exceed the value of restriction with the size of the data part.
Restrictions on the "maximum amount of something" can take the value 0, which means "unrestricted".
Most restrictions also have an 'overflow_mode' setting, meaning what to do when the limit is exceeded.
@ -16,7 +16,7 @@ It can take one of two values: `throw` or `break`. Restrictions on aggregation (
`any (only for group_by_overflow_mode)` Continuing aggregation for the keys that got into the set, but don't add new keys to the set.
## max_memory_usage { #settings_max_memory_usage}
## max_memory_usage {#settings_max_memory_usage}
The maximum amount of RAM to use for running a query on a single server.
@ -25,7 +25,7 @@ In the default configuration file, the maximum is 10 GB.
The setting doesn't consider the volume of available memory or the total volume of memory on the machine.
The restriction applies to a single query within a single server.
You can use `SHOW PROCESSLIST` to see the current memory consumption for each query.
In addition, the peak memory consumption is tracked for each query and written to the log.
Besides, the peak memory consumption is tracked for each query and written to the log.
Memory usage is not monitored for the states of certain aggregate functions.
@ -54,26 +54,26 @@ See also the description of [max_memory_usage](#settings_max_memory_usage).
The following restrictions can be checked on each block (instead of on each row). That is, the restrictions can be broken a little.
When running a query in multiple threads, the following restrictions apply to each thread separately.
Maximum number of rows that can be read from a table when running a query.
A maximum number of rows that can be read from a table when running a query.
## max_bytes_to_read
Maximum number of bytes (uncompressed data) that can be read from a table when running a query.
A maximum number of bytes (uncompressed data) that can be read from a table when running a query.
## read_overflow_mode
What to do when the volume of data read exceeds one of the limits: 'throw' or 'break'. By default, throw.
## max_rows_to_group_by { #settings-max_rows_to_group_by}
## max_rows_to_group_by {#settings-max_rows_to_group_by}
Maximum number of unique keys received from aggregation. This setting lets you limit memory consumption when aggregating.
A maximum number of unique keys received from aggregation. This setting lets you limit memory consumption when aggregating.
## group_by_overflow_mode
What to do when the number of unique keys for aggregation exceeds the limit: 'throw', 'break', or 'any'. By default, throw.
Using the 'any' value lets you run an approximation of GROUP BY. The quality of this approximation depends on the statistical nature of the data.
## max_bytes_before_external_group_by { #settings-max_bytes_before_external_group_by}
## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by}
Enables or disables execution of `GROUP BY` clauses in external memory. See [GROUP BY in external memory](../../query_language/select.md#select-group-by-in-external-memory).
@ -86,17 +86,17 @@ Default value: 0.
## max_rows_to_sort
Maximum number of rows before sorting. This allows you to limit memory consumption when sorting.
A maximum number of rows before sorting. This allows you to limit memory consumption when sorting.
## max_bytes_to_sort
Maximum number of bytes before sorting.
A maximum number of bytes before sorting.
## sort_overflow_mode
What to do if the number of rows received before sorting exceeds one of the limits: 'throw' or 'break'. By default, throw.
## max_result_rows { #setting-max_result_rows}
## max_result_rows {#setting-max_result_rows}
Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query.
@ -142,15 +142,15 @@ Minimal execution speed in rows per second. Checked on every data block when 'ti
## min_execution_speed_bytes
Minimum number of execution bytes per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is lower, an exception is thrown.
A minimum number of execution bytes per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is lower, an exception is thrown.
## max_execution_speed
Maximum number of execution rows per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is high, the execution speed will be reduced.
A maximum number of execution rows per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is high, the execution speed will be reduced.
## max_execution_speed_bytes
Maximum number of execution bytes per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is high, the execution speed will be reduced.
A maximum number of execution bytes per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is high, the execution speed will be reduced.
## timeout_before_checking_execution_speed
@ -158,11 +158,11 @@ Checks that execution speed is not too slow (no less than 'min_execution_speed')
## max_columns_to_read
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.
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.
## max_temporary_columns
Maximum number of temporary columns that must be kept in RAM at the same time when running a query, including constant columns. If there are more temporary columns than this, it throws an exception.
A maximum number of temporary columns that must be kept in RAM at the same time when running a query, including constant columns. If there are more temporary columns than this, it throws an exception.
## max_temporary_non_const_columns
@ -184,16 +184,16 @@ At this time, it isn't checked during parsing, but only after parsing the query.
## max_ast_elements
Maximum number of elements in a query syntactic tree. If exceeded, an exception is thrown.
A maximum number of elements in a query syntactic tree. If exceeded, an exception is thrown.
In the same way as the previous setting, it is checked only after parsing the query. By default, 50,000.
## max_rows_in_set
Maximum number of rows for a data set in the IN clause created from a subquery.
A maximum number of rows for a data set in the IN clause created from a subquery.
## max_bytes_in_set
Maximum number of bytes (uncompressed data) used by a set in the IN clause created from a subquery.
A maximum number of bytes (uncompressed data) used by a set in the IN clause created from a subquery.
## set_overflow_mode
@ -201,11 +201,11 @@ What to do when the amount of data exceeds one of the limits: 'throw' or 'break'
## max_rows_in_distinct
Maximum number of different rows when using DISTINCT.
A maximum number of different rows when using DISTINCT.
## max_bytes_in_distinct
Maximum number of bytes used by a hash table when using DISTINCT.
A maximum number of bytes used by a hash table when using DISTINCT.
## distinct_overflow_mode
@ -213,17 +213,17 @@ What to do when the amount of data exceeds one of the limits: 'throw' or 'break'
## max_rows_to_transfer
Maximum number of rows that can be passed to a remote server or saved in a temporary table when using GLOBAL IN.
A maximum number of rows that can be passed to a remote server or saved in a temporary table when using GLOBAL IN.
## max_bytes_to_transfer
Maximum number of bytes (uncompressed data) that can be passed to a remote server or saved in a temporary table when using GLOBAL IN.
A maximum number of bytes (uncompressed data) that can be passed to a remote server or saved in a temporary table when using GLOBAL IN.
## transfer_overflow_mode
What to do when the amount of data exceeds one of the limits: 'throw' or 'break'. By default, throw.
## max_rows_in_join { #settings-max_rows_in_join}
## max_rows_in_join {#settings-max_rows_in_join}
Limits the number of rows in the hash table that is used when joining tables.
@ -240,7 +240,7 @@ Possible values:
Default value: 0.
## max_bytes_in_join { #settings-max_bytes_in_join}
## max_bytes_in_join {#settings-max_bytes_in_join}
Limits the size in bytes of the hash table used when joining tables.
@ -257,7 +257,7 @@ Possible values:
Default value: 0.
## join_overflow_mode { #settings-join_overflow_mode}
## join_overflow_mode {#settings-join_overflow_mode}
Defines what action ClickHouse performs when any of the following join limits is reached:
@ -290,6 +290,6 @@ Default value: 100.
When inserting data, ClickHouse calculates the number of partitions in the inserted block. If the number of partitions is more than `max_partitions_per_insert_block`, ClickHouse throws an exception with the following text:
> "Too many partitions for single INSERT block (more than " + toString(max_parts) + "). The limit is controlled by 'max_partitions_per_insert_block' setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc)."
> "Too many partitions for single INSERT block (more than " + toString(max_parts) + "). The limit is controlled by 'max_partitions_per_insert_block' setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc)."
[Original article](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) <!--hide-->

View File

@ -41,11 +41,11 @@ Consider the following queries:
1. `SELECT count() FROM test_table WHERE date = '2018-10-10'`
2. `SELECT count() FROM (SELECT * FROM test_table) WHERE date = '2018-10-10'`
If `enable_optimize_predicate_expression = 1`, then the execution time of these queries is equal, because ClickHouse applies `WHERE` to the subquery when processing it.
If `enable_optimize_predicate_expression = 1`, then the execution time of these queries is equal because ClickHouse applies `WHERE` to the subquery when processing it.
If `enable_optimize_predicate_expression = 0`, then the execution time of the second query is much longer, because the `WHERE` clause applies to all the data after the subquery finishes.
## fallback_to_stale_replicas_for_distributed_queries { #settings-fallback_to_stale_replicas_for_distributed_queries}
## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries}
Forces a query to an out-of-date replica if updated data is not available. See [Replication](../table_engines/replication.md).
@ -55,13 +55,13 @@ Used when performing `SELECT` from a distributed table that points to replicated
By default, 1 (enabled).
## force_index_by_date { #settings-force_index_by_date}
## force_index_by_date {#settings-force_index_by_date}
Disables query execution if the index can't be used by date.
Works with tables in the MergeTree family.
If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see [MergeTree](../table_engines/mergetree.md).
If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see [MergeTree](../table_engines/mergetree.md).
## force_primary_key
@ -70,7 +70,7 @@ Disables query execution if indexing by the primary key is not possible.
Works with tables in the MergeTree family.
If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see [MergeTree](../table_engines/mergetree.md).
If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For more information about data ranges in MergeTree tables, see [MergeTree](../table_engines/mergetree.md).
## format_schema
@ -82,7 +82,7 @@ Enables or disables [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/func
It makes sense to disable it if the server has millions of tiny tables that are constantly being created and destroyed.
## enable_http_compression { #settings-enable_http_compression}
## enable_http_compression {#settings-enable_http_compression}
Enables or disables data compression in the response to an HTTP request.
@ -95,7 +95,7 @@ Possible values:
Default value: 0.
## http_zlib_compression_level { #settings-http_zlib_compression_level}
## http_zlib_compression_level {#settings-http_zlib_compression_level}
Sets the level of data compression in the response to an HTTP request if [enable_http_compression = 1](#settings-enable_http_compression).
@ -104,7 +104,7 @@ Possible values: Numbers from 1 to 9.
Default value: 3.
## http_native_compression_disable_checksumming_on_decompress { #settings-http_native_compression_disable_checksumming_on_decompress}
## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress}
Enables or disables checksum verification when decompressing the HTTP POST data from the client. Used only for ClickHouse native compression format (not used with `gzip` or `deflate`).
@ -117,7 +117,7 @@ Possible values:
Default value: 0.
## send_progress_in_http_headers { #settings-send_progress_in_http_headers}
## send_progress_in_http_headers {#settings-send_progress_in_http_headers}
Enables or disables `X-ClickHouse-Progress` HTTP response headers in `clickhouse-server` responses.
@ -130,7 +130,7 @@ Possible values:
Default value: 0.
## max_http_get_redirects { #setting-max_http_get_redirects}
## max_http_get_redirects {#setting-max_http_get_redirects}
Limits the maximum number of HTTP GET redirect hops for [URL](../table_engines/url.md)-engine tables. The setting applies to both types of tables: those created by the [CREATE TABLE](../../query_language/create/#create-table-query) query and by the [url](../../query_language/table_functions/url.md) table function.
@ -141,7 +141,7 @@ Possible values:
Default value: 0.
## input_format_allow_errors_num { #settings-input_format_allow_errors_num}
## input_format_allow_errors_num {#settings-input_format_allow_errors_num}
Sets the maximum number of acceptable errors when reading from text formats (CSV, TSV, etc.).
@ -153,7 +153,7 @@ If an error occurred while reading rows but the error counter is still less than
If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` are exceeded, ClickHouse throws an exception.
## input_format_allow_errors_ratio { #settings-input_format_allow_errors_ratio}
## input_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio}
Sets the maximum percentage of errors allowed when reading from text formats (CSV, TSV, etc.).
The percentage of errors is set as a floating-point number between 0 and 1.
@ -167,7 +167,7 @@ If an error occurred while reading rows but the error counter is still less than
If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` are exceeded, ClickHouse throws an exception.
## input_format_values_interpret_expressions { #settings-input_format_values_interpret_expressions}
## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions}
Enables or disables the full SQL parser if the fast stream parser can't parse the data. This setting is used only for the [Values](../../interfaces/formats.md#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../query_language/syntax.md) section.
@ -217,9 +217,9 @@ INSERT INTO datetime_t SELECT now()
Ok.
```
## input_format_values_deduce_templates_of_expressions { #settings-input_format_values_deduce_templates_of_expressions}
## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions}
Enables or disables template deduction for an SQL expressions in [Values](../../interfaces/formats.md#data-format-values) format. It allows to parse and interpret expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse will try to deduce template of an expression, parse the following rows using this template and evaluate the expression on batch of successfully parsed rows. For the following query:
Enables or disables template deduction for an SQL expressions in [Values](../../interfaces/formats.md#data-format-values) format. It allows to parse and interpret expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse will try to deduce template of an expression, parse the following rows using this template and evaluate the expression on a batch of successfully parsed rows. For the following query:
```sql
INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), (upper('Values')), ...
@ -231,7 +231,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), (
Enabled by default.
## input_format_values_accurate_types_of_literals { #settings-input_format_values_accurate_types_of_literals}
## input_format_values_accurate_types_of_literals {#settings-input_format_values_accurate_types_of_literals}
This setting is used only when `input_format_values_deduce_templates_of_expressions = 1`. It can happen, that expressions for some column have the same structure, but contain numeric literals of different types, e.g
```sql
@ -240,11 +240,11 @@ This setting is used only when `input_format_values_deduce_templates_of_expressi
(..., abs(-1), ...), -- Int64 literal
```
When this setting is enabled, ClickHouse will check actual type of literal and will use expression template of the corresponding type. In some cases it may significantly slow down expression evaluation in `Values`.
When this setting is enabled, ClickHouse will check the actual type of literal and will use an expression template of the corresponding type. In some cases, it may significantly slow down expression evaluation in `Values`.
When disabled, ClickHouse may use more general type for some literals (e.g. `Float64` or `Int64` instead of `UInt64` for `42`), but it may cause overflow and precision issues.
Enabled by default.
## input_format_defaults_for_omitted_fields { #session_settings-input_format_defaults_for_omitted_fields}
## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields}
When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) and [TabSeparated](../../interfaces/formats.md#tabseparated) formats.
@ -258,18 +258,18 @@ Possible values:
Default value: 1.
## input_format_tsv_empty_as_default { #settings-input_format_tsv_empty_as_default}
## input_format_tsv_empty_as_default {#settings-input_format_tsv_empty_as_default}
When enabled, replace empty input fields in TSV with default values. For complex default expressions `input_format_defaults_for_omitted_fields` must be enabled too.
Disabled by default.
## input_format_null_as_default { #settings-input_format_null_as_default}
## input_format_null_as_default {#settings-input_format_null_as_default}
Enables or disables using default values if input data contain `NULL`, but data type of corresponding column in not `Nullable(T)` (for text input formats).
Enables or disables using default values if input data contain `NULL`, but data type of the corresponding column in not `Nullable(T)` (for text input formats).
## input_format_skip_unknown_fields { #settings-input_format_skip_unknown_fields}
## input_format_skip_unknown_fields {#settings-input_format_skip_unknown_fields}
Enables or disables skipping insertion of extra data.
@ -289,7 +289,7 @@ Possible values:
Default value: 0.
## input_format_import_nested_json { #settings-input_format_import_nested_json}
## input_format_import_nested_json {#settings-input_format_import_nested_json}
Enables or disables the insertion of JSON data with nested objects.
@ -308,7 +308,7 @@ See also:
- [Usage of Nested Structures](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format.
## input_format_with_names_use_header { #settings-input_format_with_names_use_header}
## input_format_with_names_use_header {#settings-input_format_with_names_use_header}
Enables or disables checking the column order when inserting data.
@ -326,9 +326,9 @@ Possible values:
Default value: 1.
## date_time_input_format { #settings-date_time_input_format}
## date_time_input_format {#settings-date_time_input_format}
Allows to choose a parser of text representation of date and time.
Allows choosing a parser of the text representation of date and time.
The setting doesn't apply to [date and time functions](../../query_language/functions/date_time_functions.md).
@ -349,22 +349,22 @@ See also:
- [DateTime data type.](../../data_types/datetime.md)
- [Functions for working with dates and times.](../../query_language/functions/date_time_functions.md)
## join_default_strictness { #settings-join_default_strictness}
## join_default_strictness {#settings-join_default_strictness}
Sets default strictness for [JOIN clauses](../../query_language/select.md#select-join).
Possible values:
- `ALL` — If the right table has several matching rows, ClickHouse creates a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from matching rows. This is the normal `JOIN` behavior from standard SQL.
- `ALL` — If the right table has several matching rows, ClickHouse creates a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from matching rows. This is the normal `JOIN` behaviour from standard SQL.
- `ANY` — If the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` and `ALL` are the same.
- `ASOF` — For joining sequences with an uncertain match.
- `Empty string` — If `ALL` or `ANY` is not specified in the query, ClickHouse throws an exception.
Default value: `ALL`.
## join_any_take_last_row { #settings-join_any_take_last_row}
## join_any_take_last_row {#settings-join_any_take_last_row}
Changes behavior of join operations with `ANY` strictness.
Changes behaviour of join operations with `ANY` strictness.
!!! warning "Attention"
This setting applies only for `JOIN` operations with [Join](../table_engines/join.md) engine tables.
@ -382,7 +382,7 @@ See also:
- [Join table engine](../table_engines/join.md)
- [join_default_strictness](#settings-join_default_strictness)
## join_use_nulls { #join_use_nulls}
## join_use_nulls {#join_use_nulls}
Sets the type of [JOIN](../../query_language/select.md) behavior. When merging tables, empty cells may appear. ClickHouse fills them differently based on this setting.
@ -393,9 +393,9 @@ Possible values:
Default value: 0.
## max_block_size { #setting-max_block_size}
## max_block_size {#setting-max_block_size}
In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of block (in number of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large, so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads, and to preserve at least some cache locality.
In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of the block (in a count of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads and to preserve at least some cache locality.
Default value: 65,536.
@ -407,7 +407,7 @@ Used for the same purpose as `max_block_size`, but it sets the recommended block
However, the block size cannot be more than `max_block_size` rows.
By default: 1,000,000. It only works when reading from MergeTree engines.
## merge_tree_min_rows_for_concurrent_read { #setting-merge_tree_min_rows_for_concurrent_read}
## merge_tree_min_rows_for_concurrent_read {#setting-merge_tree_min_rows_for_concurrent_read}
If the number of rows to be read from a file of a [MergeTree](../table_engines/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads.
@ -417,7 +417,7 @@ Possible values:
Default value: 163840.
## merge_tree_min_bytes_for_concurrent_read { #setting-merge_tree_min_bytes_for_concurrent_read}
## merge_tree_min_bytes_for_concurrent_read {#setting-merge_tree_min_bytes_for_concurrent_read}
If the number of bytes to read from one file of a [MergeTree](../table_engines/mergetree.md)-engine table exceeds `merge_tree_min_bytes_for_concurrent_read`, then ClickHouse tries to concurrently read from this file in several threads.
@ -427,9 +427,9 @@ Possible value:
Default value: 251658240.
## merge_tree_min_rows_for_seek { #setting-merge_tree_min_rows_for_seek}
## merge_tree_min_rows_for_seek {#setting-merge_tree_min_rows_for_seek}
If the distance between two data blocks to be read in one file is less than `merge_tree_min_rows_for_seek` rows, then ClickHouse does not seek through the file, but reads the data sequentially.
If the distance between two data blocks to be read in one file is less than `merge_tree_min_rows_for_seek` rows, then ClickHouse does not seek through the file but reads the data sequentially.
Possible values:
@ -437,9 +437,9 @@ Possible values:
Default value: 0.
## merge_tree_min_bytes_for_seek { #setting-merge_tree_min_bytes_for_seek}
## merge_tree_min_bytes_for_seek {#setting-merge_tree_min_bytes_for_seek}
If the distance between two data blocks to be read in one file is less than `merge_tree_min_bytes_for_seek` bytes, then ClickHouse sequentially reads range of file that contains both blocks, thus avoiding extra seek.
If the distance between two data blocks to be read in one file is less than `merge_tree_min_bytes_for_seek` bytes, then ClickHouse sequentially reads a range of file that contains both blocks, thus avoiding extra seek.
Possible values:
@ -448,9 +448,9 @@ Possible values:
Default value: 0.
## merge_tree_coarse_index_granularity { #setting-merge_tree_coarse_index_granularity}
## merge_tree_coarse_index_granularity {#setting-merge_tree_coarse_index_granularity}
When searching data, ClickHouse checks the data marks in the index file. If ClickHouse finds that required keys are in some range, it divides this range into `merge_tree_coarse_index_granularity` subranges and searches the required keys there recursively.
When searching for data, ClickHouse checks the data marks in the index file. If ClickHouse finds that required keys are in some range, it divides this range into `merge_tree_coarse_index_granularity` subranges and searches the required keys there recursively.
Possible values:
@ -458,7 +458,7 @@ Possible values:
Default value: 8.
## merge_tree_max_rows_to_use_cache { #setting-merge_tree_max_rows_to_use_cache}
## merge_tree_max_rows_to_use_cache {#setting-merge_tree_max_rows_to_use_cache}
If ClickHouse should read more than `merge_tree_max_rows_to_use_cache` rows in one query, it doesn't use the cache of uncompressed blocks.
@ -470,7 +470,7 @@ Possible values:
Default value: 128 ✕ 8192.
## merge_tree_max_bytes_to_use_cache { #setting-merge_tree_max_bytes_to_use_cache}
## merge_tree_max_bytes_to_use_cache {#setting-merge_tree_max_bytes_to_use_cache}
If ClickHouse should read more than `merge_tree_max_bytes_to_use_cache` bytes in one query, it doesn't use the cache of uncompressed blocks.
@ -482,7 +482,7 @@ Possible value:
Default value: 2013265920.
## min_bytes_to_use_direct_io { #settings-min_bytes_to_use_direct_io}
## min_bytes_to_use_direct_io {#settings-min_bytes_to_use_direct_io}
The minimum data volume required for using direct I/O access to the storage disk.
@ -495,7 +495,7 @@ Possible values:
Default value: 0.
## log_queries { #settings-log-queries}
## log_queries {#settings-log-queries}
Setting up query logging.
@ -507,7 +507,7 @@ Example:
log_queries=1
```
## log_query_threads { #settings-log-query-threads}
## log_query_threads {#settings-log-query-threads}
Setting up query threads logging.
@ -519,7 +519,7 @@ Example:
log_query_threads=1
```
## max_insert_block_size { #settings-max_insert_block_size}
## max_insert_block_size {#settings-max_insert_block_size}
The size of blocks to form for insertion into a table.
This setting only applies in cases when the server forms the blocks.
@ -529,9 +529,9 @@ The setting also doesn't have a purpose when using INSERT SELECT, since data is
Default value: 1,048,576.
The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allows sorting more data in RAM.
The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion and a large enough block size allow sorting more data in RAM.
## max_replica_delay_for_distributed_queries { #settings-max_replica_delay_for_distributed_queries}
## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries}
Disables lagging replicas for distributed queries. See [Replication](../../operations/table_engines/replication.md).
@ -541,7 +541,7 @@ Default value: 300.
Used when performing `SELECT` from a distributed table that points to replicated tables.
## max_threads { #settings-max_threads}
## max_threads {#settings-max_threads}
The maximum number of query processing threads, excluding threads for retrieving data from remote servers (see the 'max_distributed_connections' parameter).
@ -556,7 +556,7 @@ For queries that are completed quickly because of a LIMIT, you can set a lower '
The smaller the `max_threads` value, the less memory is consumed.
## max_insert_threads { #settings-max_insert_threads}
## max_insert_threads {#settings-max_insert_threads}
The maximum number of threads to execute the `INSERT SELECT` query.
@ -590,7 +590,7 @@ We are writing a URL column with the String type (average size of 60 bytes per v
There usually isn't any reason to change this setting.
## max_query_size { #settings-max_query_size}
## max_query_size {#settings-max_query_size}
The maximum part of a query that can be taken to RAM for parsing with the SQL parser.
The INSERT query also contains data for INSERT that is processed by a separate stream parser (that consumes O(1) RAM), which is not included in this restriction.
@ -599,9 +599,9 @@ Default value: 256 KiB.
## interactive_delay
The interval in microseconds for checking whether request execution has been canceled and sending the progress.
The interval in microseconds for checking whether request execution has been cancelled and sending the progress.
Default value: 100,000 (checks for canceling and sends the progress ten times per second).
Default value: 100,000 (checks for cancelling and sends the progress ten times per second).
## connect_timeout, receive_timeout, send_timeout
@ -611,7 +611,7 @@ Default value: 10, 300, 300.
## cancel_http_readonly_queries_on_client_close
Cancels HTTP readonly queries (e.g. SELECT) when a client closes the connection without waiting for response.
Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response.
Default value: 0
@ -654,23 +654,23 @@ Default value: 3.
Whether to count extreme values (the minimums and maximums in columns of a query result). Accepts 0 or 1. By default, 0 (disabled).
For more information, see the section "Extreme values".
## use_uncompressed_cache { #setting-use_uncompressed_cache}
## use_uncompressed_cache {#setting-use_uncompressed_cache}
Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled).
Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../server_settings/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted.
For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically in order to save space for truly small queries. This means that you can keep the 'use_uncompressed_cache' setting always set to 1.
For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically to save space for truly small queries. This means that you can keep the 'use_uncompressed_cache' setting always set to 1.
## replace_running_query
When using the HTTP interface, the 'query_id' parameter can be passed. This is any string that serves as the query identifier.
If a query from the same user with the same 'query_id' already exists at this time, the behavior depends on the 'replace_running_query' parameter.
If a query from the same user with the same 'query_id' already exists at this time, the behaviour depends on the 'replace_running_query' parameter.
`0` (default) Throw an exception (don't allow the query to run if a query with the same 'query_id' is already running).
`1` Cancel the old query and start running the new one.
Yandex.Metrica uses this parameter set to 1 for implementing suggestions for segmentation conditions. After entering the next character, if the old query hasn't finished yet, it should be canceled.
Yandex.Metrica uses this parameter set to 1 for implementing suggestions for segmentation conditions. After entering the next character, if the old query hasn't finished yet, it should be cancelled.
## stream_flush_interval_ms
@ -682,7 +682,7 @@ The default value is 7500.
The smaller the value, the more often data is flushed into the table. Setting the value too low leads to poor performance.
## load_balancing { #settings-load_balancing}
## load_balancing {#settings-load_balancing}
Specifies the algorithm of replicas selection that is used for distributed query processing.
@ -693,22 +693,22 @@ ClickHouse supports the following algorithms of choosing replicas:
- [In order](#load_balancing-in_order)
- [First or random](#load_balancing-first_or_random)
### Random (by default) { #load_balancing-random}
### Random (by default) {#load_balancing-random}
```sql
load_balancing = random
```
The number of errors is counted for each replica. The query is sent to the replica with the fewest errors, and if there are several of these, to any one of them.
The number of errors is counted for each replica. The query is sent to the replica with the fewest errors, and if there are several of these, to anyone of them.
Disadvantages: Server proximity is not accounted for; if the replicas have different data, you will also get different data.
### Nearest Hostname { #load_balancing-nearest_hostname}
### Nearest Hostname {#load_balancing-nearest_hostname}
```sql
load_balancing = nearest_hostname
```
The number of errors is counted for each replica. Every 5 minutes, the number of errors is integrally divided by 2. Thus, the number of errors is calculated for a recent time with exponential smoothing. If there is one replica with a minimal number of errors (i.e. errors occurred recently on the other replicas), the query is sent to it. If there are multiple replicas with the same minimal number of errors, the query is sent to the replica with a host name that is most similar to the server's host name in the config file (for the number of different characters in identical positions, up to the minimum length of both host names).
The number of errors is counted for each replica. Every 5 minutes, the number of errors is integrally divided by 2. Thus, the number of errors is calculated for a recent time with exponential smoothing. If there is one replica with a minimal number of errors (i.e. errors occurred recently on the other replicas), the query is sent to it. If there are multiple replicas with the same minimal number of errors, the query is sent to the replica with a hostname that is most similar to the server's hostname in the config file (for the number of different characters in identical positions, up to the minimum length of both hostnames).
For instance, example01-01-1 and example01-01-2.yandex.ru are different in one position, while example01-01-1 and example01-02-2 differ in two places.
This method might seem primitive, but it doesn't require external data about network topology, and it doesn't compare IP addresses, which would be complicated for our IPv6 addresses.
@ -716,17 +716,17 @@ This method might seem primitive, but it doesn't require external data about net
Thus, if there are equivalent replicas, the closest one by name is preferred.
We can also assume that when sending a query to the same server, in the absence of failures, a distributed query will also go to the same servers. So even if different data is placed on the replicas, the query will return mostly the same results.
### In Order { #load_balancing-in_order}
### In Order {#load_balancing-in_order}
```sql
load_balancing = in_order
```
Replicas with the same number of errors are accessed in the same order as they are specified in configuration.
Replicas with the same number of errors are accessed in the same order as they are specified in the configuration.
This method is appropriate when you know exactly which replica is preferable.
### First or Random { #load_balancing-first_or_random}
### First or Random {#load_balancing-first_or_random}
```sql
load_balancing = first_or_random
@ -734,9 +734,9 @@ load_balancing = first_or_random
This algorithm chooses the first replica in the set or a random replica if the first is unavailable. It's effective in cross-replication topology setups, but useless in other configurations.
The `first_or_random` algorithm solves the problem of the `in_order` algorithm. With `in_order`, if one replica goes down, the next one gets a double load while the remaining replicas handle the usual amount of traffic. When using the `first_or_random` algorithm, load is evenly distributed among replicas that are still available.
The `first_or_random` algorithm solves the problem of the `in_order` algorithm. With `in_order`, if one replica goes down, the next one gets a double load while the remaining replicas handle the usual amount of traffic. When using the `first_or_random` algorithm, the load is evenly distributed among replicas that are still available.
## prefer_localhost_replica { #settings-prefer_localhost_replica}
## prefer_localhost_replica {#settings-prefer_localhost_replica}
Enables/disables preferable using the localhost replica when processing distributed queries.
@ -760,7 +760,7 @@ See the section "WITH TOTALS modifier".
The threshold for `totals_mode = 'auto'`.
See the section "WITH TOTALS modifier".
## max_parallel_replicas { #settings-max_parallel_replicas}
## max_parallel_replicas {#settings-max_parallel_replicas}
The maximum number of replicas for each shard when executing a query.
For consistency (to get different parts of the same data split), this option only works when the sampling key is set.
@ -770,7 +770,7 @@ Replica lag is not controlled.
Enable compilation of queries. By default, 0 (disabled).
Compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY).
The compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY).
If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution.
## min_count_to_compile
@ -780,31 +780,31 @@ For testing, the value can be set to 0: compilation runs synchronously and the q
If the value is 1 or more, compilation occurs asynchronously in a separate thread. The result will be used as soon as it is ready, including queries that are currently running.
Compiled code is required for each different combination of aggregate functions used in the query and the type of keys in the GROUP BY clause.
The results of compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results, since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade in this case, the old results are deleted.
The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade in this case, the old results are deleted.
## output_format_json_quote_64bit_integers { #session_settings-output_format_json_quote_64bit_integers}
## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers}
If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes.
## format_csv_delimiter { #settings-format_csv_delimiter}
## format_csv_delimiter {#settings-format_csv_delimiter}
The character interpreted as a delimiter in the CSV data. By default, the delimiter is `,`.
## input_format_csv_unquoted_null_literal_as_null { #settings-input_format_csv_unquoted_null_literal_as_null}
## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null}
For CSV input format enables or disables parsing of unquoted `NULL` as literal (synonym for `\N`).
## output_format_csv_crlf_end_of_line { #settings-output_format_csv_crlf_end_of_line}
## output_format_csv_crlf_end_of_line {#settings-output_format_csv_crlf_end_of_line}
Use DOS/Windows style line separator (CRLF) in CSV instead of Unix style (LF).
Use DOS/Windows-style line separator (CRLF) in CSV instead of Unix style (LF).
## output_format_tsv_crlf_end_of_line { #settings-output_format_tsv_crlf_end_of_line}
## output_format_tsv_crlf_end_of_line {#settings-output_format_tsv_crlf_end_of_line}
Use DOC/Windows style line separator (CRLF) in TSV instead of Unix style (LF).
Use DOC/Windows-style line separator (CRLF) in TSV instead of Unix style (LF).
## insert_quorum { #settings-insert_quorum}
## insert_quorum {#settings-insert_quorum}
Enables quorum writes.
Enables the quorum writes.
- If `insert_quorum < 2`, the quorum writes are disabled.
- If `insert_quorum >= 2`, the quorum writes are enabled.
@ -829,9 +829,9 @@ See also:
- [insert_quorum_timeout](#settings-insert_quorum_timeout)
- [select_sequential_consistency](#settings-select_sequential_consistency)
## insert_quorum_timeout { #settings-insert_quorum_timeout}
## insert_quorum_timeout {#settings-insert_quorum_timeout}
Quorum write timeout in seconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica.
Write to quorum timeout in seconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica.
Default value: 60 seconds.
@ -841,7 +841,7 @@ See also:
- [select_sequential_consistency](#settings-select_sequential_consistency)
## select_sequential_consistency { #settings-select_sequential_consistency}
## select_sequential_consistency {#settings-select_sequential_consistency}
Enables or disables sequential consistency for `SELECT` queries:
@ -861,7 +861,7 @@ See also:
- [insert_quorum](#settings-insert_quorum)
- [insert_quorum_timeout](#settings-insert_quorum_timeout)
## insert_deduplicate { #settings-insert_deduplicate}
## insert_deduplicate {#settings-insert_deduplicate}
Enables or disables block deduplication of `INSERT` (for Replicated* tables).
@ -874,7 +874,7 @@ Default value: 1.
By default, blocks inserted into replicated tables by the `INSERT` statement are deduplicated (see [Data Replication] (../ table_engines/replication.md).
## deduplicate_blocks_in_dependent_materialized_views { #settings-deduplicate_blocks_in_dependent_materialized_views}
## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate_blocks_in_dependent_materialized_views}
Enables or disables the deduplication check for materialized views that receive data from Replicated* tables.
@ -887,12 +887,12 @@ Default value: 0.
Usage
By default, deduplication is not performed for materialized views, but is done upstream, in the source table.
If an INSERTed block is skipped due to deduplication in the source table, there will be no insertion into attached materialized views. This behavior exists to enable insertion of highly aggregated data into materialized views, for cases where inserted blocks are the same after materialized view aggregation but derived from different INSERTs into the source table.
At the same time, this behavior "breaks" `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows to change this behavior. On retry a materialized view will receive the repeat insert and will perform deduplication check by itself,
ignoring check result for the source table, and will insert rows lost because of first failure.
By default, deduplication is not performed for materialized views but is done upstream, in the source table.
If an INSERTed block is skipped due to deduplication in the source table, there will be no insertion into attached materialized views. This behaviour exists to enable insertion of highly aggregated data into materialized views, for cases where inserted blocks are the same after materialized view aggregation but derived from different INSERTs into the source table.
At the same time, this behaviour "breaks" `INSERT` idempotency. If an `INSERT` into the main table was successful and `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` allows for changing this behaviour. On retry, a materialized view will receive the repeat insert and will perform deduplication check by itself,
ignoring check result for the source table, and will insert rows lost because of the first failure.
## max_network_bytes { #settings-max_network_bytes}
## max_network_bytes {#settings-max_network_bytes}
Limits the data volume (in bytes) that is received or transmitted over the network when executing a query. This setting applies to every individual query.
Possible values:
@ -902,7 +902,7 @@ Possible values:
Default value: 0.
## max_network_bandwidth { #settings-max_network_bandwidth}
## max_network_bandwidth {#settings-max_network_bandwidth}
Limits the speed of the data exchange over the network in bytes per second. This setting applies to every query.
@ -913,7 +913,7 @@ Possible values:
Default value: 0.
## max_network_bandwidth_for_user { #settings-max_network_bandwidth_for_user}
## max_network_bandwidth_for_user {#settings-max_network_bandwidth_for_user}
Limits the speed of the data exchange over the network in bytes per second. This setting applies to all concurrently running queries performed by a single user.
@ -924,7 +924,7 @@ Possible values:
Default value: 0.
## max_network_bandwidth_for_all_users { #settings-max_network_bandwidth_for_all_users}
## max_network_bandwidth_for_all_users {#settings-max_network_bandwidth_for_all_users}
Limits the speed that data is exchanged at over the network in bytes per second. This setting applies to all concurrently running queries on the server.
@ -935,7 +935,7 @@ Possible values:
Default value: 0.
## count_distinct_implementation { #settings-count_distinct_implementation}
## count_distinct_implementation {#settings-count_distinct_implementation}
Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT ...)](../../query_language/agg_functions/reference.md#agg_function-count) construction.
@ -949,7 +949,7 @@ Possible values:
Default value: `uniqExact`.
## skip_unavailable_shards { #settings-skip_unavailable_shards}
## skip_unavailable_shards {#settings-skip_unavailable_shards}
Enables or disables silently skipping of unavailable shards.
@ -979,13 +979,13 @@ Possible values:
Default value: 0.
## optimize_skip_unused_shards { #settings-optimize_skip_unused_shards}
## optimize_skip_unused_shards {#settings-optimize_skip_unused_shards}
Enables or disables skipping of unused shards for SELECT queries that has sharding key condition in PREWHERE/WHERE (assumes that the data is distributed by sharding key, otherwise do nothing).
Enables or disables skipping of unused shards for SELECT queries that have sharding key condition in PREWHERE/WHERE (assumes that the data is distributed by sharding key, otherwise do nothing).
Default value: 0
## force_optimize_skip_unused_shards { #settings-force_optimize_skip_unused_shards}
## force_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards}
Enables or disables query execution if [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) enabled and skipping of unused shards is not possible. If the skipping is not possible and the setting is enabled exception will be thrown.
@ -997,7 +997,7 @@ Possible values:
Default value: 0
## optimize_throw_if_noop { #setting-optimize_throw_if_noop}
## optimize_throw_if_noop {#setting-optimize_throw_if_noop}
Enables or disables throwing an exception if an [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) query didn't perform a merge.
@ -1011,19 +1011,19 @@ Possible values:
Default value: 0.
## distributed_replica_error_half_life { #settings-distributed_replica_error_half_life}
## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life}
- Type: seconds
- Default value: 60 seconds
Controls how fast errors in distributed tables are zeroed. If a replica is unavailabe for some time, accumulates 5 errors, and distributed_replica_error_half_life is set to 1 second, then the replica is considered normal 3 seconds after last error.
Controls how fast errors in distributed tables are zeroed. If a replica is unavailable for some time, accumulates 5 errors, and distributed_replica_error_half_life is set to 1 second, then the replica is considered normal 3 seconds after last error.
See also:
- [Table engine Distributed](../../operations/table_engines/distributed.md)
- [distributed_replica_error_cap](#settings-distributed_replica_error_cap)
## distributed_replica_error_cap { #settings-distributed_replica_error_cap}
## distributed_replica_error_cap {#settings-distributed_replica_error_cap}
- Type: unsigned int
- Default value: 1000
@ -1035,31 +1035,31 @@ See also:
- [Table engine Distributed](../../operations/table_engines/distributed.md)
- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life)
## distributed_directory_monitor_sleep_time_ms { #distributed_directory_monitor_sleep_time_ms}
## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms}
Base interval for the [Distributed](../table_engines/distributed.md) table engine to send data. The actual interval grows exponentially in the event of errors.
Possible values:
- Positive integer number of milliseconds.
- A positive integer number of milliseconds.
Default value: 100 milliseconds.
## distributed_directory_monitor_max_sleep_time_ms { #distributed_directory_monitor_max_sleep_time_ms}
## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms}
Maximum interval for the [Distributed](../table_engines/distributed.md) table engine to send data. Limits exponential growth of the interval set in the [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) setting.
Possible values:
- Positive integer number of milliseconds.
- A positive integer number of milliseconds.
Default value: 30000 milliseconds (30 seconds).
## distributed_directory_monitor_batch_inserts { #distributed_directory_monitor_batch_inserts}
## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts}
Enables/disables sending of inserted data in batches.
When batch sending is enabled, the [Distributed](../table_engines/distributed.md) table engine tries to send multiple files of inserted data in one operation instead of sending them separately. Batch sending improves cluster performance by better utilizing server and network resources.
When batch sending is enabled, the [Distributed](../table_engines/distributed.md) table engine tries to send multiple files of inserted data in one operation instead of sending them separately. Batch sending improves cluster performance by better-utilizing server and network resources.
Possible values:
@ -1068,7 +1068,7 @@ Possible values:
Default value: 0.
## os_thread_priority { #setting-os_thread_priority}
## os_thread_priority {#setting-os_thread_priority}
Sets the priority ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) for threads that execute queries. The OS scheduler considers this priority when choosing the next thread to run on each available CPU core.
@ -1079,11 +1079,11 @@ Possible values:
- You can set values in the range `[-20, 19]`.
Lower values mean higher priority. Threads with low `nice` priority values are executed more frequently than threads with high values. High values are preferable for long running non-interactive queries because it allows them to quickly give up resources in favor of short interactive queries when they arrive.
Lower values mean higher priority. Threads with low `nice` priority values are executed more frequently than threads with high values. High values are preferable for long-running non-interactive queries because it allows them to quickly give up resources in favour of short interactive queries when they arrive.
Default value: 0.
## query_profiler_real_time_period_ns { #query_profiler_real_time_period_ns}
## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns}
Sets the period for a real clock timer of the [query profiler](../../operations/performance/sampling_query_profiler.md). Real clock timer counts wall-clock time.
@ -1106,17 +1106,17 @@ See also:
- System table [trace_log](../system_tables.md#system_tables-trace_log)
## query_profiler_cpu_time_period_ns { #query_profiler_cpu_time_period_ns}
## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns}
Sets the period for a CPU clock timer of the [query profiler](../../operations/performance/sampling_query_profiler.md). This timer counts only CPU time.
Possible values:
- Positive integer number of nanoseconds.
- A positive integer number of nanoseconds.
Recommended values:
- 10000000 (100 times a second) nanosecods and more for for single queries.
- 10000000 (100 times a second) nanoseconds and more for single queries.
- 1000000000 (once a second) for cluster-wide profiling.
- 0 for turning off the timer.
@ -1129,7 +1129,7 @@ See also:
- System table [trace_log](../system_tables.md#system_tables-trace_log)
## allow_introspection_functions { #settings-allow_introspection_functions}
## allow_introspection_functions {#settings-allow_introspection_functions}
Enables of disables [introspections functions](../../query_language/functions/introspection.md) for query profiling.
@ -1159,7 +1159,7 @@ Enable order-preserving parallel parsing of data formats. Supported only for TSV
The minimum chunk size in bytes, which each thread will parse in parallel.
## output_format_avro_codec { #settings-output_format_avro_codec}
## output_format_avro_codec {#settings-output_format_avro_codec}
Sets the compression codec used for output Avro file.
@ -1173,7 +1173,7 @@ Possible values:
Default value: `snappy` (if available) or `deflate`.
## output_format_avro_sync_interval { #settings-output_format_avro_sync_interval}
## output_format_avro_sync_interval {#settings-output_format_avro_sync_interval}
Sets minimum data size (in bytes) between synchronization markers for output Avro file.
@ -1183,7 +1183,7 @@ Possible values: 32 (32 bytes) - 1073741824 (1 GiB)
Default value: 32768 (32 KiB)
## format_avro_schema_registry_url { #settings-format_avro_schema_registry_url}
## format_avro_schema_registry_url {#settings-format_avro_schema_registry_url}
Sets Confluent Schema Registry URL to use with [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format

View File

@ -1,5 +1,4 @@
# Settings profiles
# Settings Profiles
A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile.
To apply all the settings in a profile, set the `profile` setting.

View File

@ -1,4 +1,4 @@
# User settings
# User Settings
The `users` section of the `user.xml` configuration file contains user settings.

View File

@ -6,7 +6,7 @@ System tables don't have files with data on the disk or files with metadata. The
System tables are read-only.
They are located in the 'system' database.
## system.asynchronous_metrics { #system_tables-asynchronous_metrics}
## system.asynchronous_metrics {#system_tables-asynchronous_metrics}
Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.
@ -92,7 +92,7 @@ The `system.columns` table contains the following columns (the column type is sh
- `is_in_primary_key` (UInt8) — Flag that indicates whether the column is in the primary key expression.
- `is_in_sampling_key` (UInt8) — Flag that indicates whether the column is in the sampling key expression.
## system.contributors { #system_contributors}
## system.contributors {#system_contributors}
Contains information about contributors. All constributors in random order. The order is random at query execution time.
@ -138,7 +138,7 @@ This table contains a single String column called 'name' the name of a datab
Each database that the server knows about has a corresponding entry in the table.
This system table is used for implementing the `SHOW DATABASES` query.
## system.detached_parts { #system_tables-detached_parts}
## system.detached_parts {#system_tables-detached_parts}
Contains information about detached parts of [MergeTree](table_engines/mergetree.md) tables. The `reason` column specifies why the part was detached. For user-detached parts, the reason is empty. Such parts can be attached with [ALTER TABLE ATTACH PARTITION|PART](../query_language/query_language/alter/#alter_attach-partition) command. For the description of other columns, see [system.parts](#system_tables-parts). If part name is invalid, values of some columns may be `NULL`. Such parts can be deleted with [ALTER TABLE DROP DETACHED PART](../query_language/query_language/alter/#alter_drop-detached).
@ -164,7 +164,7 @@ Columns:
Note that the amount of memory used by the dictionary is not proportional to the number of items stored in it. So for flat and cached dictionaries, all the memory cells are pre-assigned, regardless of how full the dictionary actually is.
## system.events { #system_tables-events}
## system.events {#system_tables-events}
Contains information about the number of events that have occurred in the system. For example, in the table, you can find how many `SELECT` queries were processed since the ClickHouse server started.
@ -243,7 +243,7 @@ Columns:
- `bytes_written_uncompressed` (UInt64) — Number of bytes written, uncompressed.
- `rows_written` (UInt64) — Number of rows written.
## system.metrics { #system_tables-metrics}
## system.metrics {#system_tables-metrics}
Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date.
@ -283,7 +283,7 @@ SELECT * FROM system.metrics LIMIT 10
- [system.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`.
- [Monitoring](monitoring.md) — Base concepts of ClickHouse monitoring.
## system.metric_log { #system_tables-metric_log}
## system.metric_log {#system_tables-metric_log}
Contains history of metrics values from tables `system.metrics` and `system.events`, periodically flushed to disk.
To turn on metrics history collection on `system.metric_log`, create `/etc/clickhouse-server/config.d/metric_log.xml` with following content:
@ -356,7 +356,7 @@ This table contains a single row with a single 'dummy' UInt8 column containing t
This table is used if a SELECT query doesn't specify the FROM clause.
This is similar to the DUAL table found in other DBMSs.
## system.parts { #system_tables-parts}
## system.parts {#system_tables-parts}
Contains information about parts of [MergeTree](table_engines/mergetree.md) tables.
@ -406,7 +406,7 @@ Columns:
- `marks_size` (`UInt64`) Alias for `marks_bytes`.
## system.part_log { #system_tables-part-log}
## system.part_log {#system_tables-part-log}
The `system.part_log` table is created only if the [part_log](server_settings/settings.md#server_settings-part-log) server setting is specified.
@ -439,7 +439,7 @@ The `system.part_log` table contains the following columns:
The `system.part_log` table is created after the first inserting data to the `MergeTree` table.
## system.processes { #system_tables-processes}
## system.processes {#system_tables-processes}
This system table is used for implementing the `SHOW PROCESSLIST` query.
@ -455,7 +455,7 @@ Columns:
- `query` (String) The query text. For `INSERT`, it doesn't include the data to insert.
- `query_id` (String) Query ID, if defined.
## system.text_log { #system_tables-text_log}
## system.text_log {#system_tables-text_log}
Contains logging entries. Logging level which goes to this table can be limited with `text_log.level` server setting.
@ -483,7 +483,7 @@ Columns:
- `source_line` (`UInt64`) - Source line from which the logging was done.
## system.query_log { #system_tables-query_log}
## system.query_log {#system_tables-query_log}
Contains information about execution of queries. For each query, you can see processing start time, duration of processing, error messages and other information.
@ -569,7 +569,7 @@ When the table is deleted manually, it will be automatically created on the fly.
You can specify an arbitrary partitioning key for the `system.query_log` table in the [query_log](server_settings/settings.md#server_settings-query-log) server setting (see the `partition_by` parameter).
## system.query_thread_log { #system_tables-query-thread-log}
## system.query_thread_log {#system_tables-query-thread-log}
The table contains information about each query execution thread.
@ -634,7 +634,7 @@ When the table is deleted manually, it will be automatically created on the fly.
You can specify an arbitrary partitioning key for the `system.query_thread_log` table in the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server setting (see the `partition_by` parameter).
## system.trace_log { #system_tables-trace_log}
## system.trace_log {#system_tables-trace_log}
Contains stack traces collected by the sampling query profiler.
@ -677,7 +677,7 @@ query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915
trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935]
```
## system.replicas { #system_tables-replicas}
## system.replicas {#system_tables-replicas}
Contains information and status for replicated tables residing on the local server.
This table can be used for monitoring. The table contains a row for every Replicated\* table.
@ -960,7 +960,7 @@ pzxid: 987021252247
path: /clickhouse/tables/01-08/visits/replicas
```
## system.mutations { #system_tables-mutations}
## system.mutations {#system_tables-mutations}
The table contains information about [mutations](../query_language/alter.md#alter-mutations) of MergeTree tables and their progress. Each mutation command is represented by a single row. The table has the following columns:
@ -987,7 +987,7 @@ If there were problems with mutating some parts, the following columns contain a
**latest_fail_reason** - The exception message that caused the most recent part mutation failure.
## system.disks { #system_tables-disks}
## system.disks {#system_tables-disks}
Contains information about disks defined in the [server configuration](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure).
@ -1000,7 +1000,7 @@ Columns:
- `keep_free_space` ([UInt64](../data_types/int_uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` parameter of disk configuration.
## system.storage_policies { #system_tables-storage_policies}
## system.storage_policies {#system_tables-storage_policies}
Contains information about storage policies and volumes defined in the [server configuration](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure).

View File

@ -1,4 +1,4 @@
# CollapsingMergeTree { #table_engine-collapsingmergetree}
# CollapsingMergeTree {#table_engine-collapsingmergetree}
The engine inherits from [MergeTree](mergetree.md) and adds the logic of rows collapsing to data parts merge algorithm.
@ -55,7 +55,7 @@ All of the parameters excepting `sign` have the same meaning as in `MergeTree`.
</details>
## Collapsing { #table_engine-collapsingmergetree-collapsing}
## Collapsing {#table_engine-collapsingmergetree-collapsing}
### Data
@ -103,7 +103,7 @@ Why we need 2 rows for each change read in the [Algorithm](#table_engine-collaps
2. Long growing arrays in columns reduce the efficiency of the engine due to load for writing. The more straightforward data, the higher efficiency.
3. The `SELECT` results depend strongly on the consistency of object changes history. Be accurate when preparing data for inserting. You can get unpredictable results in inconsistent data, for example, negative values for non-negative metrics such as session depth.
### Algorithm { #table_engine-collapsingmergetree-collapsing-algorithm}
### Algorithm {#table_engine-collapsingmergetree-collapsing-algorithm}
When ClickHouse merges data parts, each group of consecutive rows with the same sorting key (`ORDER BY`) is reduced to not more than two rows, one with `Sign = 1` ("state" row) and another with `Sign = -1` ("cancel" row). In other words, entries collapse.

View File

@ -1,4 +1,4 @@
# File { #table_engines-file}
# File {#table_engines-file}
The File table engine keeps the data in a file in one of the supported [file
formats](../../interfaces/formats.md#formats) (TabSeparated, Native, etc.).

View File

@ -1,4 +1,4 @@
# GenerateRandom { #table_engines-generate}
# GenerateRandom {#table_engines-generate}
The GenerateRandom table engine produces random data for given table schema.

View File

@ -1,4 +1,4 @@
# GraphiteMergeTree { #graphitemergetree}
# GraphiteMergeTree {#graphitemergetree}
This engine is designed for thinning and aggregating/averaging (rollup) [Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite.
@ -6,7 +6,7 @@ You can use any ClickHouse table engine to store the Graphite data if you don't
The engine inherits properties from [MergeTree](mergetree.md).
## Creating a Table { #creating-table}
## Creating a Table {#creating-table}
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
@ -66,7 +66,7 @@ All of the parameters excepting `config_section` have the same meaning as in `Me
- `config_section` — Name of the section in the configuration file, where are the rules of rollup set.
</details>
## Rollup configuration { #rollup-configuration}
## Rollup configuration {#rollup-configuration}
The settings for rollup are defined by the [graphite_rollup](../server_settings/settings.md#server_settings-graphite_rollup) parameter in the server configuration. The name of the parameter could be any. You can create several configurations and use them for different tables.
@ -77,14 +77,14 @@ required-columns
patterns
```
### Required Columns { #required-columns}
### Required Columns {#required-columns}
- `path_column_name` — The name of the column storing the metric name (Graphite sensor). Default value: `Path`.
- `time_column_name` — The name of the column storing the time of measuring the metric. Default value: `Time`.
- `value_column_name` — The name of the column storing the value of the metric at the time set in `time_column_name`. Default value: `Value`.
- `version_column_name` — The name of the column storing the version of the metric. Default value: `Timestamp`.
### Patterns { #patterns}
### Patterns {#patterns}
Structure of the `patterns` section:
@ -126,7 +126,7 @@ Fields for `pattern` and `default` sections:
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`.
### Configuration Example { #configuration-example}
### Configuration Example {#configuration-example}
```xml
<graphite_rollup>

View File

@ -1,4 +1,4 @@
# HDFS { #table_engines-hdfs}
# HDFS {#table_engines-hdfs}
This engine provides integration with [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)via ClickHouse. This engine is similar
to the [File](file.md) and [URL](url.md) engines, but provides Hadoop-specific features.

View File

@ -1,4 +1,4 @@
# Table engines { #table_engines}
# Table engines {#table_engines}
The table engine (type of table) determines:
@ -64,7 +64,7 @@ Engines in the family:
- [Memory](memory.md)
- [Buffer](buffer.md)
## Virtual columns { #table_engines-virtual_columns}
## Virtual columns {#table_engines-virtual_columns}
Virtual column is an integral table engine attribute that is defined in the engine source code.

View File

@ -1,4 +1,4 @@
# JDBC { #table_engine-jdbc}
# JDBC {#table_engine-jdbc}
Allows ClickHouse to connect to external databases via [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity).

View File

@ -2,7 +2,7 @@
Prepared data structure for using in [JOIN](../../query_language/select.md#select-join) operations.
## Creating a Table { #creating-a-table}
## Creating a Table {#creating-a-table}
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
@ -77,7 +77,7 @@ You cannot perform a `SELECT` query directly from the table. Instead, use one of
- Place the table to the right side in a `JOIN` clause.
- Call the [joinGet](../../query_language/functions/other_functions.md#joinget) function, which lets you extract data from the table the same way as from a dictionary.
### Limitations and Settings { #join-limitations-and-settings}
### Limitations and Settings {#join-limitations-and-settings}
When creating a table, the following settings are applied:

View File

@ -9,7 +9,7 @@ Kafka lets you:
- Process streams as they become available.
## Creating a Table { #table_engine-kafka-creating-a-table}
## Creating a Table {#table_engine-kafka-creating-a-table}
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]

View File

@ -1,4 +1,4 @@
# MergeTree { #table_engines-mergetree}
# MergeTree {#table_engines-mergetree}
The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most robust ClickHouse table engines.
@ -26,7 +26,7 @@ Main features:
The [Merge](merge.md) engine does not belong to the `*MergeTree` family.
## Creating a Table { #table_engine-mergetree-creating-a-table}
## Creating a Table {#table_engine-mergetree-creating-a-table}
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
@ -50,7 +50,7 @@ For a description of parameters, see the [CREATE query description](../../query_
!!!note "Note"
`INDEX` is an experimental feature, see [Data Skipping Indexes](#table_engine-mergetree-data_skipping-indexes).
### Query Clauses { #mergetree-query-clauses}
### Query Clauses {#mergetree-query-clauses}
- `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters.
@ -134,7 +134,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)
The `MergeTree` engine is configured in the same way as in the example above for the main engine configuration method.
</details>
## Data Storage { #mergetree-data-storage}
## Data Storage {#mergetree-data-storage}
A table consists of data parts sorted by primary key.
@ -146,7 +146,7 @@ Each data part is logically divided into granules. A granule is the smallest ind
The granule size is restricted by the `index_granularity` and `index_granularity_bytes` settings of the table engine. The number of rows in a granule lays in the `[1, index_granularity]` range, depending on the size of the rows. The size of a granule can exceed `index_granularity_bytes` if the size of a single row is greater than the value of the setting. In this case, the size of the granule equals the size of the row.
## Primary Keys and Indexes in Queries { #primary-keys-and-indexes-in-queries}
## Primary Keys and Indexes in Queries {#primary-keys-and-indexes-in-queries}
Take the `(CounterID, Date)` primary key as an example. In this case, the sorting and index can be illustrated as follows:
@ -248,7 +248,7 @@ ClickHouse cannot use an index if the values of the primary key in the query par
ClickHouse uses this logic not only for days of the month sequences, but for any primary key that represents a partially-monotonic sequence.
### Data Skipping Indexes (Experimental) { #table_engine-mergetree-data_skipping-indexes}
### Data Skipping Indexes (Experimental) {#table_engine-mergetree-data_skipping-indexes}
The index declaration is in the columns section of the `CREATE` query.
```sql
@ -368,7 +368,7 @@ For concurrent table access, we use multi-versioning. In other words, when a tab
Reading from a table is automatically parallelized.
## TTL for Columns and Tables { #table_engine-mergetree-ttl}
## TTL for Columns and Tables {#table_engine-mergetree-ttl}
Determines the lifetime of values.
@ -390,7 +390,7 @@ TTL date_time + INTERVAL 1 MONTH
TTL date_time + INTERVAL 15 HOUR
```
### Column TTL { #mergetree-column-ttl}
### Column TTL {#mergetree-column-ttl}
When the values in the column expire, ClickHouse replaces them with the default values for the column data type. If all the column values in the data part expire, ClickHouse deletes this column from the data part in a filesystem.
@ -429,7 +429,7 @@ ALTER TABLE example_table
c String TTL d + INTERVAL 1 MONTH;
```
### Table TTL { #mergetree-table-ttl}
### Table TTL {#mergetree-table-ttl}
Table can have an expression for removal of expired rows, and multiple expressions for automatic move of parts between [disks or volumes](#table_engine-mergetree-multiple-volumes). When rows in the table expire, ClickHouse deletes all corresponding rows. For parts moving feature, all rows of a part must satisfy the movement expression criteria.
@ -479,7 +479,7 @@ If you perform the `SELECT` query between merges, you may get expired data. To a
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/mergetree/) <!--hide-->
## Using Multiple Block Devices for Data Storage { #table_engine-mergetree-multiple-volumes}
## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes}
### Introduction
@ -496,7 +496,7 @@ Data part is the minimum movable unit for `MergeTree`-engine tables. The data be
The names given to the described entities can be found in the system tables, [system.storage_policies](../system_tables.md#system_tables-storage_policies) and [system.disks](../system_tables.md#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables.
### Configuration { #table_engine-mergetree-multiple-volumes_configure}
### Configuration {#table_engine-mergetree-multiple-volumes_configure}
Disks, volumes and storage policies should be declared inside the `<storage_configuration>` tag either in the main file `config.xml` or in a distinct file in the `config.d` directory.

View File

@ -1,4 +1,4 @@
# ODBC { #table_engine-odbc}
# ODBC {#table_engine-odbc}
Allows ClickHouse to connect to external databases via [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity).

View File

@ -1,4 +1,4 @@
# Data Replication { #table_engines-replication}
# Data Replication {#table_engines-replication}
Replication is only supported for tables in the MergeTree family:
@ -73,7 +73,7 @@ You can have any number of replicas of the same data. Yandex.Metrica uses double
The system monitors data synchronicity on replicas and is able to recover after a failure. Failover is automatic (for small differences in data) or semi-automatic (when data differs too much, which may indicate a configuration error).
## Creating Replicated Tables { #creating-replicated-tables}
## Creating Replicated Tables {#creating-replicated-tables}
The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`.

View File

@ -4,7 +4,7 @@ This engine belongs to the family of log engines. See the common properties of l
Use this engine in scenarios when you need to write many tables with a small amount of data (less than 1 million rows).
## Creating a Table { #table_engines-stripelog-creating-a-table}
## Creating a Table {#table_engines-stripelog-creating-a-table}
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
@ -17,7 +17,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
See the detailed description of the [CREATE TABLE](../../query_language/create.md#create-table-query) query.
## Writing the Data { #table_engines-stripelog-writing-the-data}
## Writing the Data {#table_engines-stripelog-writing-the-data}
The `StripeLog` engine stores all the columns in one file. For each `INSERT` query, ClickHouse appends the data block to the end of a table file, writing columns one by one.
@ -28,11 +28,11 @@ For each table ClickHouse writes the files:
The `StripeLog` engine does not support the `ALTER UPDATE` and `ALTER DELETE` operations.
## Reading the Data { #table_engines-stripelog-reading-the-data}
## Reading the Data {#table_engines-stripelog-reading-the-data}
The file with marks allows ClickHouse to parallelize the reading of data. This means that a `SELECT` query returns rows in an unpredictable order. Use the `ORDER BY` clause to sort rows.
## Example of Use { #table_engines-stripelog-example-of-use}
## Example of Use {#table_engines-stripelog-example-of-use}
Creating a table:

View File

@ -86,7 +86,7 @@ SELECT key, sum(value) FROM summtt GROUP BY key
```
## Data Processing { #data-processing}
## Data Processing {#data-processing}
When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data.

View File

@ -1,4 +1,4 @@
# URL(URL, Format) { #table_engines-url}
# URL(URL, Format) {#table_engines-url}
Manages data on a remote HTTP/HTTPS server. This engine is similar
to the [File](file.md) engine.

View File

@ -71,7 +71,7 @@ All of the parameters except `sign` and `version` have the same meaning as in `M
</details>
## Collapsing { #table_engines_versionedcollapsingmergetree}
## Collapsing {#table_engines_versionedcollapsingmergetree}
### Data
@ -119,7 +119,7 @@ To find out why we need two rows for each change, see [Algorithm](#table_engines
2. Long growing arrays in columns reduce the efficiency of the engine due to the load for writing. The more straightforward the data, the better the efficiency.
3. `SELECT` results depend strongly on the consistency of the history of object changes. Be accurate when preparing data for inserting. You can get unpredictable results with inconsistent data, such as negative values for non-negative metrics like session depth.
### Algorithm { #table_engines-versionedcollapsingmergetree-algorithm}
### Algorithm {#table_engines-versionedcollapsingmergetree-algorithm}
When ClickHouse merges data parts, it deletes each pair of rows that have the same primary key and version and different `Sign`. The order of rows does not matter.

View File

@ -1,4 +1,4 @@
# View { #table_engines-view}
# View {#table_engines-view}
Used for implementing views (for more information, see the `CREATE VIEW query`). It does not store data, but only stores the specified `SELECT` query. When reading from a table, it runs this query (and deletes all unnecessary columns from the query).

View File

@ -5,7 +5,7 @@
- [Query processing](#troubleshooting-does-not-process-queries)
- [Efficiency of query processing](#troubleshooting-too-slow)
## Installation { #troubleshooting-installation-errors}
## Installation {#troubleshooting-installation-errors}
### You Cannot Get Deb Packages from ClickHouse Repository With apt-get
@ -13,7 +13,7 @@
- If you cannot access the repository for any reason, download packages as described in the [Getting started](../getting_started/index.md) article and install them manually using the `sudo dpkg -i <packages>` command. You will also need the `tzdata` package.
## Connecting to the Server { #troubleshooting-accepts-no-connections}
## Connecting to the Server {#troubleshooting-accepts-no-connections}
Possible issues:
@ -120,7 +120,7 @@ Check:
You might be using the wrong user name or password.
## Query Processing { #troubleshooting-does-not-process-queries}
## Query Processing {#troubleshooting-does-not-process-queries}
If ClickHouse is not able to process the query, it sends an error description to the client. In the `clickhouse-client` you get a description of the error in the console. If you are using the HTTP interface, ClickHouse sends the error description in the response body. For example:
@ -133,7 +133,7 @@ If you start `clickhouse-client` with the `stack-trace` parameter, ClickHouse re
You might see a message about a broken connection. In this case, you can repeat the query. If the connection breaks every time you perform the query, check the server logs for errors.
## Efficiency of Query Processing { #troubleshooting-too-slow}
## Efficiency of Query Processing {#troubleshooting-too-slow}
If you see that ClickHouse is working too slowly, you need to profile the load on the server resources and network for your queries.

View File

@ -25,7 +25,7 @@ Then pass this file to a standard input of `clickhouse-benchmark`.
clickhouse-benchmark [keys] < queries_file
```
## Keys { #clickhouse-benchmark-keys}
## Keys {#clickhouse-benchmark-keys}
- `-c N`, `--concurrency=N` — Number of queries that `clickhouse-benchmark` sends simultaneously. Default value: 1.
- `-d N`, `--delay=N` — Interval in seconds between intermediate reports (set 0 to disable reports). Default value: 1.
@ -47,7 +47,7 @@ clickhouse-benchmark [keys] < queries_file
If you want to apply some [settings](../../operations/settings/index.md) for queries, pass them as a key `--<session setting name>= SETTING_VALUE`. For example, `--max_memory_usage=1048576`.
## Output { #clickhouse-benchmark-output}
## Output {#clickhouse-benchmark-output}
By default, `clickhouse-benchmark` reports for each `--delay` interval.
@ -90,13 +90,13 @@ In the report you can find:
- Percentiles of queries execution time.
## Comparison mode { #clickhouse-benchmark-comparison-mode}
## Comparison mode {#clickhouse-benchmark-comparison-mode}
`clickhouse-benchmark` can compare performances for two running ClickHouse servers.
To use the comparison mode, specify endpoints of both servers by two pairs of `--host`, `--port` keys. Keys matched together by position in arguments list, the first `--host` is matched with the first `--port` and so on. `clickhouse-benchmark` establishes connections to both servers, then sends queries. Each query addressed to a randomly selected server. The results are shown for each server separately.
## Example { #clickhouse-benchmark-example}
## Example {#clickhouse-benchmark-example}
```bash
$ echo "SELECT * FROM system.numbers LIMIT 10000000 OFFSET 10000000" | clickhouse-benchmark -i 10

View File

@ -1,8 +1,8 @@
# Aggregate function combinators { #aggregate_functions_combinators}
# Aggregate function combinators {#aggregate_functions_combinators}
The name of an aggregate function can have a suffix appended to it. This changes the way the aggregate function works.
## -If { #agg-functions-combinator-if}
## -If {#agg-functions-combinator-if}
The suffix -If can be appended to the name of any aggregate function. In this case, the aggregate function accepts an extra argument a condition (Uint8 type). The aggregate function processes only the rows that trigger the condition. If the condition was not triggered even once, it returns a default value (usually zeros or empty strings).
@ -10,7 +10,7 @@ Examples: `sumIf(column, cond)`, `countIf(cond)`, `avgIf(x, cond)`, `quantilesTi
With conditional aggregate functions, you can calculate aggregates for several conditions at once, without using subqueries and `JOIN`s. For example, in Yandex.Metrica, conditional aggregate functions are used to implement the segment comparison functionality.
## -Array { #agg-functions-combinator-array}
## -Array {#agg-functions-combinator-array}
The -Array suffix can be appended to any aggregate function. In this case, the aggregate function takes arguments of the 'Array(T)' type (arrays) instead of 'T' type arguments. If the aggregate function accepts multiple arguments, this must be arrays of equal lengths. When processing arrays, the aggregate function works like the original aggregate function across all array elements.
@ -20,7 +20,7 @@ Example 2: `uniqArray(arr)` Counts the number of unique elements in all 'arr
-If and -Array can be combined. However, 'Array' must come first, then 'If'. Examples: `uniqArrayIf(arr, cond)`, `quantilesTimingArrayIf(level1, level2)(arr, cond)`. Due to this order, the 'cond' argument won't be an array.
## -State { #agg-functions-combinator-state}
## -State {#agg-functions-combinator-state}
If you apply this combinator, the aggregate function doesn't return the resulting value (such as the number of unique values for the [uniq](reference.md#agg_function-uniq) function), but an intermediate state of the aggregation (for `uniq`, this is the hash table for calculating the number of unique values). This is an `AggregateFunction(...)` that can be used for further processing or stored in a table to finish aggregating later.
@ -32,19 +32,19 @@ To work with these states, use:
- [-Merge](#aggregate_functions_combinators_merge) combinator.
- [-MergeState](#aggregate_functions_combinators_mergestate) combinator.
## -Merge { #aggregate_functions_combinators_merge}
## -Merge {#aggregate_functions_combinators_merge}
If you apply this combinator, the aggregate function takes the intermediate aggregation state as an argument, combines the states to finish aggregation, and returns the resulting value.
## -MergeState { #aggregate_functions_combinators_mergestate}
## -MergeState {#aggregate_functions_combinators_mergestate}
Merges the intermediate aggregation states in the same way as the -Merge combinator. However, it doesn't return the resulting value, but an intermediate aggregation state, similar to the -State combinator.
## -ForEach { #agg-functions-combinator-foreach}
## -ForEach {#agg-functions-combinator-foreach}
Converts an aggregate function for tables into an aggregate function for arrays that aggregates the corresponding array items and returns an array of results. For example, `sumForEach` for the arrays `[1, 2]`, `[3, 4, 5]`and`[6, 7]`returns the result `[10, 13, 5]` after adding together the corresponding array items.
## -OrDefault { #agg-functions-combinator-ordefault}
## -OrDefault {#agg-functions-combinator-ordefault}
Fills the default value of the aggregate function's return type if there is nothing to aggregate.
@ -57,7 +57,7 @@ SELECT avg(number), avgOrDefault(number) FROM numbers(0)
└─────────────┴──────────────────────┘
```
## -OrNull { #agg-functions-combinator-ornull}
## -OrNull {#agg-functions-combinator-ornull}
Fills `null` if there is nothing to aggregate. The return column will be nullable.
@ -85,7 +85,7 @@ FROM
└────────────────────────────────┘
```
## -Resample { #agg_functions-combinator-resample}
## -Resample {#agg_functions-combinator-resample}
Lets you divide data into groups, and then separately aggregates the data in those groups. Groups are created by splitting the values from one column into intervals.

View File

@ -1,4 +1,4 @@
# Aggregate functions { #aggregate_functions}
# Aggregate functions {#aggregate_functions}
Aggregate functions work in the [normal](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) way as expected by database experts.

View File

@ -1,4 +1,4 @@
# Parametric aggregate functions { #aggregate_functions_parametric}
# Parametric aggregate functions {#aggregate_functions_parametric}
Some aggregate functions can accept not only argument columns (used for compression), but a set of parameters constants for initialization. The syntax is two pairs of brackets instead of one. The first is for parameters, and the second is for arguments.
@ -71,7 +71,7 @@ FROM
In this case, you should remember that you don't know the histogram bin borders.
## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) { #function-sequencematch}
## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) {#function-sequencematch}
Checks whether the sequence contains an event chain that matches the pattern.
@ -161,7 +161,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM
- [sequenceCount](#function-sequencecount)
## sequenceCount(pattern)(time, cond1, cond2, ...) { #function-sequencecount}
## sequenceCount(pattern)(time, cond1, cond2, ...) {#function-sequencecount}
Counts the number of event chains that matched the pattern. The function searches event chains that don't overlap. It starts to search for the next chain after the current chain is matched.
@ -219,7 +219,7 @@ SELECT sequenceCount('(?1).*(?2)')(time, number = 1, number = 2) FROM t
- [sequenceMatch](#function-sequencematch)
## windowFunnel { #windowfunnel}
## windowFunnel {#windowfunnel}
Searches for event chains in a sliding time window and calculates the maximum number of events that occurred from the chain.
@ -309,7 +309,7 @@ Result:
└───────┴───┘
```
## retention { #retention}
## retention {#retention}
The function takes as arguments a set of conditions from 1 to 32 arguments of type `UInt8` that indicate whether a certain condition was met for the event.
Any condition can be specified as an argument (as in [WHERE](../../query_language/select.md#select-where)).

View File

@ -1,6 +1,6 @@
# Function Reference
## count { #agg_function-count}
## count {#agg_function-count}
Counts the number of rows or not-NULL values.
@ -65,7 +65,7 @@ SELECT count(DISTINCT num) FROM t
This example shows that `count(DISTINCT num)` is performed by the `uniqExact` function according to the `count_distinct_implementation` setting value.
## any(x) { #agg_function-any}
## any(x) {#agg_function-any}
Selects the first encountered value.
The query can be executed in any order and even in a different order each time, so the result of this function is indeterminate.
@ -277,15 +277,15 @@ num
3
```
## min(x) { #agg_function-min}
## min(x) {#agg_function-min}
Calculates the minimum.
## max(x) { #agg_function-max}
## max(x) {#agg_function-max}
Calculates the maximum.
## argMin(arg, val) { #agg_function-argMin}
## argMin(arg, val) {#agg_function-argMin}
Calculates the 'arg' value for a minimal 'val' value. If there are several different values of 'arg' for minimal values of 'val', the first of these values encountered is output.
@ -306,12 +306,12 @@ SELECT argMin(user, salary) FROM salary
└──────────────────────┘
```
## argMax(arg, val) { #agg_function-argMax}
## argMax(arg, val) {#agg_function-argMax}
Calculates the 'arg' value for a maximum 'val' value. If there are several different values of 'arg' for maximum values of 'val', the first of these values encountered is output.
## sum(x) { #agg_function-sum}
## sum(x) {#agg_function-sum}
Calculates the sum.
Only works for numbers.
@ -323,7 +323,7 @@ Computes the sum of the numbers, using the same data type for the result as for
Only works for numbers.
## sumMap(key, value) { #agg_functions-summap}
## sumMap(key, value) {#agg_functions-summap}
Totals the 'value' array according to the keys specified in the 'key' array.
The number of elements in 'key' and 'value' must be the same for each row that is totaled.
@ -451,7 +451,7 @@ The kurtosis of the given distribution. Type — [Float64](../../data_types/floa
SELECT kurtSamp(value) FROM series_with_value_column
```
## timeSeriesGroupSum(uid, timestamp, value) { #agg_function-timeseriesgroupsum}
## timeSeriesGroupSum(uid, timestamp, value) {#agg_function-timeseriesgroupsum}
`timeSeriesGroupSum` can aggregate different time series that sample timestamp not alignment.
It will use linear interpolation between two sample timestamp and then sum time-series together.
@ -498,7 +498,7 @@ And the result will be:
[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)]
```
## timeSeriesGroupRateSum(uid, ts, val) { #agg_function-timeseriesgroupratesum}
## timeSeriesGroupRateSum(uid, ts, val) {#agg_function-timeseriesgroupratesum}
Similarly timeSeriesGroupRateSum, timeSeriesGroupRateSum will Calculate the rate of time-series and then sum rates together.
Also, timestamp should be in ascend order before use this function.
@ -507,13 +507,13 @@ Use this function, the result above case will be:
[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)]
```
## avg(x) { #agg_function-avg}
## avg(x) {#agg_function-avg}
Calculates the average.
Only works for numbers.
The result is always Float64.
## uniq { #agg_function-uniq}
## uniq {#agg_function-uniq}
Calculates the approximate number of different values of the argument.
@ -549,7 +549,7 @@ We recommend using this function in almost all scenarios.
- [uniqHLL12](#agg_function-uniqhll12)
- [uniqExact](#agg_function-uniqexact)
## uniqCombined { #agg_function-uniqcombined}
## uniqCombined {#agg_function-uniqcombined}
Calculates the approximate number of different argument values.
@ -596,11 +596,11 @@ Compared to the [uniq](#agg_function-uniq) function, the `uniqCombined`:
- [uniqHLL12](#agg_function-uniqhll12)
- [uniqExact](#agg_function-uniqexact)
## uniqCombined64 { #agg_function-uniqcombined64}
## uniqCombined64 {#agg_function-uniqcombined64}
Same as [uniqCombined](#agg_function-uniqcombined), but uses 64-bit hash for all data types.
## uniqHLL12 { #agg_function-uniqhll12}
## uniqHLL12 {#agg_function-uniqhll12}
Calculates the approximate number of different argument values, using the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algorithm.
@ -636,7 +636,7 @@ We don't recommend using this function. In most cases, use the [uniq](#agg_funct
- [uniqExact](#agg_function-uniqexact)
## uniqExact { #agg_function-uniqexact}
## uniqExact {#agg_function-uniqexact}
Calculates the exact number of different argument values.
@ -659,7 +659,7 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, `
- [uniqHLL12](#agg_function-uniqhll12)
## groupArray(x), groupArray(max_size)(x) { #agg_function-grouparray}
## groupArray(x), groupArray(max_size)(x) {#agg_function-grouparray}
Creates an array of argument values.
Values can be added to the array in any (indeterminate) order.
@ -684,7 +684,7 @@ Optional parameters:
- The default value for substituting in empty positions.
- The length of the resulting array. This allows you to receive arrays of the same size for all the aggregate keys. When using this parameter, the default value must be specified.
## groupArrayMovingSum { #agg_function-grouparraymovingsum}
## groupArrayMovingSum {#agg_function-grouparraymovingsum}
Calculates the moving sum of input values.
@ -753,7 +753,7 @@ FROM t
└────────────┴─────────────────────────────────┴────────────────────────┘
```
## groupArrayMovingAvg { #agg_function-grouparraymovingavg}
## groupArrayMovingAvg {#agg_function-grouparraymovingavg}
Calculates the moving average of input values.
@ -831,7 +831,7 @@ Creates an array from different argument values. Memory consumption is the same
The second version (with the `max_size` parameter) limits the size of the resulting array to `max_size` elements.
For example, `groupUniqArray(1)(x)` is equivalent to `[any(x)]`.
## quantile { #quantile}
## quantile {#quantile}
Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence.
@ -896,7 +896,7 @@ Result:
- [quantiles](#quantiles)
## quantileDeterministic { #quantiledeterministic}
## quantileDeterministic {#quantiledeterministic}
Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence.
@ -962,7 +962,7 @@ Result:
- [quantiles](#quantiles)
## quantileExact { #quantileexact}
## quantileExact {#quantileexact}
Exactly computes the [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence.
@ -1014,7 +1014,7 @@ Result:
- [median](#median)
- [quantiles](#quantiles)
## quantileExactWeighted { #quantileexactweighted}
## quantileExactWeighted {#quantileexactweighted}
Exactly computes the [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence, taking into account the weight of each element.
@ -1078,7 +1078,7 @@ Result:
- [median](#median)
- [quantiles](#quantiles)
## quantileTiming { #quantiletiming}
## quantileTiming {#quantiletiming}
With the determined precision computes the [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence.
@ -1160,7 +1160,7 @@ Result:
- [median](#median)
- [quantiles](#quantiles)
## quantileTimingWeighted { #quantiletimingweighted}
## quantileTimingWeighted {#quantiletimingweighted}
With the determined precision computes the [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence according to the weight of each sequence member.
@ -1243,7 +1243,7 @@ Result:
- [quantiles](#quantiles)
## quantileTDigest { #quantiletdigest}
## quantileTDigest {#quantiletdigest}
Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm.
@ -1298,7 +1298,7 @@ Result:
- [median](#median)
- [quantiles](#quantiles)
## quantileTDigestWeighted { #quantiletdigestweighted}
## quantileTDigestWeighted {#quantiletdigestweighted}
Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm. The function takes into account the weight of each sequence member. The maximum error is 1%. Memory consumption is `log(n)`, where `n` is a number of values.
@ -1354,7 +1354,7 @@ Result:
- [quantiles](#quantiles)
## median { #median}
## median {#median}
The `median*` functions are the aliases for the corresponding `quantile*` functions. They calculate median of a numeric data sample.
@ -1397,7 +1397,7 @@ Result:
```
## quantiles(level1, level2, ...)(x) { #quantiles}
## quantiles(level1, level2, ...)(x) {#quantiles}
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
@ -1462,7 +1462,7 @@ FROM ontime
└─────────────────────┘
```
## topKWeighted { #topkweighted}
## topKWeighted {#topkweighted}
Similar to `topK` but takes one additional argument of integer type - `weight`. Every value is accounted `weight` times for frequency calculation.
@ -1562,12 +1562,12 @@ SELECT arrayReduce('simpleLinearRegression', [0, 1, 2, 3], [3, 4, 5, 6])
└───────────────────────────────────────────────────────────────────┘
```
## stochasticLinearRegression { #agg_functions-stochasticlinearregression}
## stochasticLinearRegression {#agg_functions-stochasticlinearregression}
This function implements stochastic linear regression. It supports custom parameters for learning rate, L2 regularization coefficient, mini-batch size and has few methods for updating weights ([Adam](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Adam) (used by default), [simple SGD](https://en.wikipedia.org/wiki/Stochastic_gradient_descent), [Momentum](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Momentum), [Nesterov](https://mipt.ru/upload/medialibrary/d7e/41-91.pdf)).
### Parameters { #agg_functions-stochasticlinearregression-parameters}
### Parameters {#agg_functions-stochasticlinearregression-parameters}
There are 4 customizable parameters. They are passed to the function sequentially, but there is no need to pass all four - default values will be used, however good model required some parameter tuning.
@ -1581,7 +1581,7 @@ stochasticLinearRegression(1.0, 1.0, 10, 'SGD')
4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergance and stability of stochastic gradient methods.
### Usage { #agg_functions-stochasticlinearregression-usage}
### Usage {#agg_functions-stochasticlinearregression-usage}
`stochasticLinearRegression` is used in two steps: fitting the model and predicting on new data. In order to fit the model and save its state for later usage we use `-State` combinator, which basically saves the state (model weights, etc).
To predict we use function [evalMLMethod](../functions/machine_learning_functions.md#machine_learning_methods-evalmlmethod), which takes a state as an argument as well as features to predict on.
@ -1622,7 +1622,7 @@ The query will return a column of predicted values. Note that first argument of
`test_data` is a table like `train_data` but may not contain target value.
### Notes { #agg_functions-stochasticlinearregression-notes}
### Notes {#agg_functions-stochasticlinearregression-notes}
1. To merge two models user may create such query:
```sql
@ -1642,12 +1642,12 @@ The query will return a column of predicted values. Note that first argument of
- [Difference between linear and logistic regressions](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression)
## stochasticLogisticRegression { #agg_functions-stochasticlogisticregression}
## stochasticLogisticRegression {#agg_functions-stochasticlogisticregression}
This function implements stochastic logistic regression. It can be used for binary classification problem, supports the same custom parameters as stochasticLinearRegression and works the same way.
### Parameters { #agg_functions-stochasticlogisticregression-parameters}
### Parameters {#agg_functions-stochasticlogisticregression-parameters}
Parameters are exactly the same as in stochasticLinearRegression:
`learning rate`, `l2 regularization coefficient`, `mini-batch size`, `method for updating weights`.

View File

@ -1,4 +1,4 @@
## ALTER { #query_language_queries_alter}
## ALTER {#query_language_queries_alter}
The `ALTER` query is only supported for `*MergeTree` tables, as well as `Merge`and`Distributed`. The query has several variations.
@ -23,7 +23,7 @@ The following actions are supported:
These actions are described in detail below.
#### ADD COLUMN { #alter_add-column}
#### ADD COLUMN {#alter_add-column}
```sql
ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after]
@ -43,7 +43,7 @@ Example:
ALTER TABLE visits ADD COLUMN browser String AFTER user_id
```
#### DROP COLUMN { #alter_drop-column}
#### DROP COLUMN {#alter_drop-column}
```sql
DROP COLUMN [IF EXISTS] name
@ -59,7 +59,7 @@ Example:
ALTER TABLE visits DROP COLUMN browser
```
#### CLEAR COLUMN { #alter_clear-column}
#### CLEAR COLUMN {#alter_clear-column}
```sql
CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name
@ -75,7 +75,7 @@ Example:
ALTER TABLE visits CLEAR COLUMN browser IN PARTITION tuple()
```
#### COMMENT COLUMN { #alter_comment-column}
#### COMMENT COLUMN {#alter_comment-column}
```sql
COMMENT COLUMN [IF EXISTS] name 'comment'
@ -93,7 +93,7 @@ Example:
ALTER TABLE visits COMMENT COLUMN browser 'The table shows the browser used for accessing the site.'
```
#### MODIFY COLUMN { #alter_modify-column}
#### MODIFY COLUMN {#alter_modify-column}
```sql
MODIFY COLUMN [IF EXISTS] name [type] [default_expr] [TTL]
@ -191,7 +191,7 @@ Constraint check *will not be executed* on existing data if it was added.
All changes on replicated tables are broadcasting to ZooKeeper so will be applied on other replicas.
### Manipulations With Partitions and Parts { #alter_manipulations-with-partitions}
### Manipulations With Partitions and Parts {#alter_manipulations-with-partitions}
The following operations with [partitions](../operations/table_engines/custom_partitioning_key.md) are available:
@ -207,7 +207,7 @@ The following operations with [partitions](../operations/table_engines/custom_pa
- [FREEZE PARTITION](#alter_freeze-partition) Creates a backup of a partition.
- [FETCH PARTITION](#alter_fetch-partition) Downloads a partition from another server.
- [MOVE PARTITION|PART](#alter_move-partition) Move partition/data part to another disk or volume.
#### DETACH PARTITION { #alter_detach-partition}
#### DETACH PARTITION {#alter_detach-partition}
```sql
ALTER TABLE table_name DETACH PARTITION partition_expr
@ -227,7 +227,7 @@ After the query is executed, you can do whatever you want with the data in the `
This query is replicated it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../operations/system_tables.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replica.
#### DROP PARTITION { #alter_drop-partition}
#### DROP PARTITION {#alter_drop-partition}
```sql
ALTER TABLE table_name DROP PARTITION partition_expr
@ -239,7 +239,7 @@ Read about setting the partition expression in a section [How to specify the par
The query is replicated it deletes data on all replicas.
#### DROP DETACHED PARTITION|PART { #alter_drop-detached}
#### DROP DETACHED PARTITION|PART {#alter_drop-detached}
```sql
ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr
@ -249,7 +249,7 @@ Removes the specified part or all parts of the specified partition from `detache
Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
#### ATTACH PARTITION|PART { #alter_attach-partition}
#### ATTACH PARTITION|PART {#alter_attach-partition}
``` sql
ALTER TABLE table_name ATTACH PARTITION|PART partition_expr
@ -268,7 +268,7 @@ This query is replicated. The replica-initiator checks whether there is data in
So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas.
#### ATTACH PARTITION FROM { #alter_attach-partition-from}
#### ATTACH PARTITION FROM {#alter_attach-partition-from}
```sql
ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1
@ -281,7 +281,7 @@ For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
#### REPLACE PARTITION { #alter_replace-partition}
#### REPLACE PARTITION {#alter_replace-partition}
```sql
ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1
@ -294,7 +294,7 @@ For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
#### MOVE PARTITION TO TABLE { #alter_move_to_table-partition}
#### MOVE PARTITION TO TABLE {#alter_move_to_table-partition}
``` sql
ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest
@ -311,7 +311,7 @@ For the query to run successfully, the following conditions must be met:
#### CLEAR COLUMN IN PARTITION { #alter_clear-column-partition}
#### CLEAR COLUMN IN PARTITION {#alter_clear-column-partition}
```sql
ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr
@ -325,7 +325,7 @@ Example:
ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902
```
#### FREEZE PARTITION { #alter_freeze-partition}
#### FREEZE PARTITION {#alter_freeze-partition}
```sql
ALTER TABLE table_name FREEZE [PARTITION partition_expr]
@ -364,7 +364,7 @@ Restoring from a backup doesn't require stopping the server.
For more information about backups and restoring data, see the [Data Backup](../operations/backup.md) section.
#### CLEAR INDEX IN PARTITION { #alter_clear-index-partition}
#### CLEAR INDEX IN PARTITION {#alter_clear-index-partition}
```sql
ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr
@ -372,7 +372,7 @@ ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr
The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column data.
#### FETCH PARTITION { #alter_fetch-partition}
#### FETCH PARTITION {#alter_fetch-partition}
```sql
ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'path-in-zookeeper'
@ -400,7 +400,7 @@ Before downloading, the system checks if the partition exists and the table stru
Although the query is called `ALTER TABLE`, it does not change the table structure and does not immediately change the data available in the table.
#### MOVE PARTITION|PART { #alter_move-partition}
#### MOVE PARTITION|PART {#alter_move-partition}
Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes).
@ -421,7 +421,7 @@ ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow'
ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd'
```
#### How To Set Partition Expression { #alter-how-to-specify-part-expr}
#### How To Set Partition Expression {#alter-how-to-specify-part-expr}
You can specify the partition expression in `ALTER ... PARTITION` queries in different ways:
@ -458,7 +458,7 @@ For non-replicatable tables, all `ALTER` queries are performed synchronously. Fo
For `ALTER ... ATTACH|DETACH|DROP` queries, you can use the `replication_alter_partitions_sync` setting to set up waiting.
Possible values: `0` do not wait; `1` only wait for own execution (default); `2` wait for all.
### Mutations { #alter-mutations}
### Mutations {#alter-mutations}
Mutations are an ALTER query variant that allows changing or deleting rows in a table. In contrast to standard `UPDATE` and `DELETE` queries that are intended for point data changes, mutations are intended for heavy operations that change a lot of rows in a table. Supported for the `MergeTree` family of table engines including the engines with replication support.

View File

@ -1,6 +1,6 @@
# CREATE Queries
## CREATE DATABASE { #query_language-create-database}
## CREATE DATABASE {#query_language-create-database}
Creates database.
@ -29,7 +29,7 @@ CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(..
By default, ClickHouse uses its own [database engine](../database_engines/index.md).
## CREATE TABLE { #create-table-query}
## CREATE TABLE {#create-table-query}
The `CREATE TABLE` query can have several forms.
@ -70,7 +70,7 @@ In all cases, if `IF NOT EXISTS` is specified, the query won't return an error i
There can be other clauses after the `ENGINE` clause in the query. See detailed documentation on how to create tables in the descriptions of [table engines](../operations/table_engines/index.md#table_engines).
### Default Values { #create-default-values}
### Default Values {#create-default-values}
The column description can specify an expression for a default value, in one of the following ways:`DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`.
Example: `URLDomain String DEFAULT domain(URL)`.
@ -105,7 +105,7 @@ If you add a new column to a table but later change its default expression, the
It is not possible to set default values for elements in nested data structures.
### Constraints { #constraints}
### Constraints {#constraints}
Along with columns descriptions constraints could be defined:
@ -127,7 +127,7 @@ Adding large amount of constraints can negatively affect performance of big `INS
Defines storage time for values. Can be specified only for MergeTree-family tables. For the detailed description, see [TTL for columns and tables](../operations/table_engines/mergetree.md#table_engine-mergetree-ttl).
### Column Compression Codecs { #codecs}
### Column Compression Codecs {#codecs}
By default, ClickHouse applies the `lz4` compression method. For `MergeTree`-engine family you can change the default compression method in the [compression](../operations/server_settings/settings.md#server-settings-compression) section of a server configuration. You can also define the compression method for each individual column in the `CREATE TABLE` query.
@ -158,7 +158,7 @@ Compression is supported for the following table engines:
ClickHouse supports common purpose codecs and specialized codecs.
#### Specialized Codecs { #create-query-specialized-codecs}
#### Specialized Codecs {#create-query-specialized-codecs}
These codecs are designed to make compression more effective by using specific features of data. Some of these codecs don't compress data themself. Instead, they prepare the data for a common purpose codec, which compresses it better than without this preparation.
@ -180,7 +180,7 @@ CREATE TABLE codec_example
ENGINE = MergeTree()
```
#### Common purpose codecs { #create-query-common-purpose-codecs}
#### Common purpose codecs {#create-query-common-purpose-codecs}
Codecs:
@ -275,7 +275,7 @@ Views look the same as normal tables. For example, they are listed in the result
There isn't a separate query for deleting views. To delete a view, use `DROP TABLE`.
## CREATE DICTIONARY { #create-dictionary-query}
## CREATE DICTIONARY {#create-dictionary-query}
```sql
CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name

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